diff --git a/.gitignore b/.gitignore
index 31c5061e33..76a4fcb904 100644
--- a/.gitignore
+++ b/.gitignore
@@ -20,6 +20,7 @@ SOURCES
gmon.out
*.gz
*.tgz
+!tests/trivup/trivup*.tar.gz
*.bz2
*.deb
*.rpm
@@ -31,3 +32,4 @@ cov-int
gdbrun*.gdb
TAGS
vcpkg_installed
+*tmp-KafkaCluster*
diff --git a/.semaphore/project.yml b/.semaphore/project.yml
deleted file mode 100644
index 4ba05ab89b..0000000000
--- a/.semaphore/project.yml
+++ /dev/null
@@ -1,43 +0,0 @@
-# This file is managed by ServiceBot plugin - Semaphore. The content in this file is created using a common
-# template and configurations in service.yml.
-# Modifications in this file will be overwritten by generated content in the nightly run.
-# For more information, please refer to the page:
-# https://confluentinc.atlassian.net/wiki/spaces/Foundations/pages/2871296194/Add+SemaphoreCI
-apiVersion: v1alpha
-kind: Project
-metadata:
- name: librdkafka
- description: ""
-spec:
- visibility: private
- repository:
- url: git@github.com:confluentinc/librdkafka.git
- run_on:
- - tags
- - branches
- pipeline_file: .semaphore/semaphore.yml
- integration_type: github_app
- status:
- pipeline_files:
- - path: .semaphore/semaphore.yml
- level: pipeline
- whitelist:
- branches:
- - master
- - /semaphore.*/
- - /dev_.*/
- - /feature\/.*/
- custom_permissions: true
- debug_permissions:
- - empty
- - default_branch
- - non_default_branch
- - pull_request
- - forked_pull_request
- - tag
- attach_permissions:
- - default_branch
- - non_default_branch
- - pull_request
- - forked_pull_request
- - tag
diff --git a/.semaphore/project_public.yml b/.semaphore/project_public.yml
deleted file mode 100644
index 7e095c94d9..0000000000
--- a/.semaphore/project_public.yml
+++ /dev/null
@@ -1,20 +0,0 @@
-# This file is managed by ServiceBot plugin - Semaphore. The content in this file is created using a common
-# template and configurations in service.yml.
-# Modifications in this file will be overwritten by generated content in the nightly run.
-# For more information, please refer to the page:
-# https://confluentinc.atlassian.net/wiki/spaces/Foundations/pages/2871296194/Add+SemaphoreCI
-apiVersion: v1alpha
-kind: Project
-metadata:
- name: librdkafka
- description: ""
-spec:
- visibility: private
- repository:
- url: git@github.com:confluentinc/librdkafka.git
- pipeline_file: .semaphore/semaphore.yml
- integration_type: github_app
- status:
- pipeline_files:
- - path: .semaphore/semaphore.yml
- level: pipeline
diff --git a/.semaphore/run-all-tests.yml b/.semaphore/run-all-tests.yml
new file mode 100644
index 0000000000..18d90becd1
--- /dev/null
+++ b/.semaphore/run-all-tests.yml
@@ -0,0 +1,77 @@
+version: v1.0
+name: run-all-tests
+
+agent:
+ machine:
+ type: s1-prod-ubuntu24-04-amd64-00
+
+execution_time_limit:
+ hours: 3
+
+global_job_config:
+ prologue:
+ commands:
+ - checkout
+ - '[[ -z "$GIT_REF" ]] || git checkout $GIT_REF'
+ - wget -O rapidjson-dev.deb https://launchpad.net/ubuntu/+archive/primary/+files/rapidjson-dev_1.1.0+dfsg2-3_all.deb
+ - sudo dpkg -i rapidjson-dev.deb
+ - sudo apt update
+ - sudo apt remove -y needrestart
+ - sudo apt install -y valgrind
+ - python3 -m pip install -U pip
+ - python3 -m pip -V
+ - (cd tests && python3 -m pip install -r requirements.txt)
+ - ./configure --install-deps --enable-werror --enable-devel
+ - make -j all
+ - make -j -C tests build
+ - sem-version java 17
+
+blocks:
+ - name: "Run all tests (x86_64)"
+ dependencies: []
+ task:
+ agent:
+ machine:
+ type: s1-prod-ubuntu24-04-amd64-1
+ prologue:
+ commands:
+ - if [[ "$TEST_ARCHES" != *"x86_64"* ]]; then exit 0; fi
+ jobs:
+ - name: "PLAINTEXT cluster (x86_64)"
+ env_vars:
+ - name: TEST_SSL
+ value: "False"
+ commands:
+ - if [[ "$TEST_TYPE" != *"plaintext"* ]]; then exit 0; fi
+ - ./tests/run-all-tests.sh
+ - name: "SSL cluster (x86_64)"
+ env_vars:
+ - name: TEST_SSL
+ value: "True"
+ commands:
+ - if [[ "$TEST_TYPE" != *"ssl"* ]]; then exit 0; fi
+ - ./tests/run-all-tests.sh
+ - name: "Run all tests (aarch64)"
+ dependencies: []
+ task:
+ agent:
+ machine:
+ type: s1-prod-ubuntu24-04-arm64-1
+ prologue:
+ commands:
+ - if [[ "$TEST_ARCHES" != *"aarch64"* ]]; then exit 0; fi
+ jobs:
+ - name: "PLAINTEXT cluster (aarch64)"
+ env_vars:
+ - name: TEST_SSL
+ value: "False"
+ commands:
+ - if [[ "$TEST_TYPE" != *"plaintext"* ]]; then exit 0; fi
+ - ./tests/run-all-tests.sh
+ - name: "SSL cluster (aarch64)"
+ env_vars:
+ - name: TEST_SSL
+ value: "True"
+ commands:
+ - if [[ "$TEST_TYPE" != *"ssl"* ]]; then exit 0; fi
+ - ./tests/run-all-tests.sh
diff --git a/.semaphore/semaphore-integration.yml b/.semaphore/semaphore-integration.yml
new file mode 100644
index 0000000000..7f7368f055
--- /dev/null
+++ b/.semaphore/semaphore-integration.yml
@@ -0,0 +1,252 @@
+version: v1.0
+name: 'librdkafka integration and all architectures tests pipeline'
+agent:
+ machine:
+ type: s1-prod-ubuntu24-04-arm64-00
+execution_time_limit:
+ hours: 3
+
+auto_cancel:
+ running:
+ when: "branch != 'master'"
+
+global_job_config:
+ prologue:
+ commands:
+ - checkout
+ - mkdir artifacts
+ - mkdir dest
+ env_vars:
+ - name: CACHE_TAG
+ value: '7'
+ - name: CI
+ value: 'true'
+ - name: KAFKA_VERSION
+ value: '4.0.0'
+ - name: CP_VERSION
+ value: '8.0.0'
+ # Shows plain output from docker build (no progress bars)
+ - name: BUILDKIT_PROGRESS
+ value: plain
+ - name: MACOSX_DEPLOYMENT_TARGET
+ value: '13'
+
+blocks:
+ - name: 'OSX arm64/m1'
+ dependencies: []
+ task:
+ agent:
+ machine:
+ type: s1-macos-15-arm64-8
+ env_vars:
+ - name: ARTIFACT_KEY
+ value: p-librdkafka__plat-osx__arch-arm64__lnk-all
+ epilogue:
+ commands:
+ - '[[ -z $SEMAPHORE_GIT_TAG_NAME ]] || artifact push workflow artifacts/ --destination artifacts/${ARTIFACT_KEY}/'
+ jobs:
+ - name: 'Build'
+ commands:
+ - ./configure --install-deps --source-deps-only --enable-static --disable-lz4-ext --enable-strip
+ - make -j all examples check
+ - examples/rdkafka_example -X builtin.features
+ - otool -L src/librdkafka.dylib
+ - otool -L src-cpp/librdkafka++.dylib
+ - make -j -C tests build
+ - make -C tests run_local_quick
+ - DESTDIR="$PWD/dest" make install
+ - (cd dest && tar cvzf ../artifacts/librdkafka.tgz .)
+
+
+ - name: 'OSX x64'
+ dependencies: []
+ task:
+ agent:
+ machine:
+ type: s1-macos-15-amd64-6
+ env_vars:
+ - name: ARTIFACT_KEY
+ value: p-librdkafka__plat-osx__arch-x64__lnk-all
+ epilogue:
+ commands:
+ - '[[ -z $SEMAPHORE_GIT_TAG_NAME ]] || artifact push workflow artifacts/ --destination artifacts/${ARTIFACT_KEY}/'
+ jobs:
+ - name: 'Build'
+ commands:
+ - ./configure --install-deps --source-deps-only --enable-static --disable-lz4-ext --enable-strip
+ - make -j all examples check
+ - examples/rdkafka_example -X builtin.features
+ - otool -L src/librdkafka.dylib
+ - otool -L src-cpp/librdkafka++.dylib
+ - make -j -C tests build
+ - make -C tests run_local_quick
+ - DESTDIR="$PWD/dest" make install
+ - (cd dest && tar cvzf ../artifacts/librdkafka.tgz .)
+
+ - name: 'Linux Ubuntu amd64: integration tests'
+ dependencies: []
+ task:
+ agent:
+ machine:
+ type: s1-prod-ubuntu24-04-amd64-1
+ env_vars:
+ - name: CFLAGS
+ value: -std=gnu90 # Test minimum C standard, default in CentOS 7
+ prologue:
+ commands:
+ - '[[ -z $DOCKERHUB_APIKEY ]] || docker login --username $DOCKERHUB_USER --password $DOCKERHUB_APIKEY'
+ jobs:
+ - name: 'Build and integration tests with "classic" protocol'
+ commands:
+ - cache restore trivup-kafka-${KAFKA_VERSION}-${CACHE_TAG}
+ - packaging/tools/run-in-docker.sh test-runner-amd64-${CACHE_TAG}
+ ./packaging/tools/run-integration-tests.sh
+ ${KAFKA_VERSION} ${CP_VERSION} classic
+ - cache store trivup-kafka-${KAFKA_VERSION}-${CACHE_TAG} tests/tmp-KafkaCluster/KafkaCluster/KafkaBrokerApp/kafka/${KAFKA_VERSION}
+ - name: 'Build and integration tests with "consumer" protocol'
+ commands:
+ - cache restore trivup-kafka-${KAFKA_VERSION}-${CACHE_TAG}
+ - packaging/tools/run-in-docker.sh test-runner-amd64-${CACHE_TAG}
+ ./packaging/tools/run-integration-tests.sh
+ ${KAFKA_VERSION} ${CP_VERSION} consumer
+ - cache store trivup-kafka-${KAFKA_VERSION}-${CACHE_TAG} tests/tmp-KafkaCluster/KafkaCluster/KafkaBrokerApp/kafka/${KAFKA_VERSION}
+
+
+ - name: 'Linux arm64: local quick tests'
+ dependencies: []
+ task:
+ agent:
+ machine:
+ type: s1-prod-ubuntu24-04-arm64-1
+ prologue:
+ commands:
+ - '[[ -z $DOCKERHUB_APIKEY ]] || docker login --username $DOCKERHUB_USER --password $DOCKERHUB_APIKEY'
+ epilogue:
+ commands:
+ - '[[ -z $SEMAPHORE_GIT_TAG_NAME ]] || artifact push workflow artifacts/ --destination artifacts/${ARTIFACT_KEY}/'
+ jobs:
+ - name: 'Build: centos8 glibc +gssapi'
+ env_vars:
+ - name: ARTIFACT_KEY
+ value: p-librdkafka__plat-linux__dist-centos8__arch-arm64__lnk-std__extra-gssapi
+ commands:
+ - packaging/tools/build-release-artifacts.sh quay.io/pypa/manylinux_2_28_aarch64:2024.07.01-1 artifacts/librdkafka.tgz
+
+ - name: 'Build: centos8 glibc'
+ env_vars:
+ - name: ARTIFACT_KEY
+ value: p-librdkafka__plat-linux__dist-centos8__arch-arm64__lnk-all
+ commands:
+ - packaging/tools/build-release-artifacts.sh --disable-gssapi quay.io/pypa/manylinux_2_28_aarch64:2024.07.01-1 artifacts/librdkafka.tgz
+
+ - name: 'Build: alpine musl +gssapi'
+ env_vars:
+ - name: ARTIFACT_KEY
+ value: p-librdkafka__plat-linux__dist-alpine__arch-arm64__lnk-all__extra-gssapi
+ commands:
+ - packaging/tools/build-release-artifacts.sh alpine:3.16.9 artifacts/librdkafka.tgz
+
+ - name: 'Build: alpine musl'
+ env_vars:
+ - name: ARTIFACT_KEY
+ value: p-librdkafka__plat-linux__dist-alpine__arch-arm64__lnk-all
+ commands:
+ - packaging/tools/build-release-artifacts.sh --disable-gssapi alpine:3.16.9 artifacts/librdkafka.tgz
+
+
+ - name: 'Windows x64: MinGW-w64'
+ dependencies: []
+ task:
+ agent:
+ machine:
+ type: s1-prod-windows
+ env_vars:
+ - name: CHERE_INVOKING
+ value: 'yes'
+ - name: MSYSTEM
+ value: UCRT64
+ prologue:
+ commands:
+ # Set up msys2
+ - "& .\\win32\\setup-msys2.ps1"
+ epilogue:
+ commands:
+ - if ($env:SEMAPHORE_GIT_TAG_NAME -ne "") { artifact push workflow artifacts/ --destination artifacts/$Env:ARTIFACT_KEY/ }
+ jobs:
+ - name: 'Build: MinGW-w64 Dynamic'
+ env_vars:
+ - name: ARTIFACT_KEY
+ value: p-librdkafka__plat-windows__dist-mingw__arch-x64__lnk-std
+ commands:
+ - C:\msys64\usr\bin\bash -lc './packaging/mingw-w64/semaphoreci-build.sh ./artifacts/librdkafka.tgz'
+
+ - name: 'Build: MinGW-w64 Static'
+ env_vars:
+ - name: ARTIFACT_KEY
+ value: p-librdkafka__plat-windows__dist-mingw__arch-x64__lnk-static
+ commands:
+ - C:\msys64\usr\bin\bash -lc './packaging/mingw-w64/semaphoreci-build.sh --static ./artifacts/librdkafka.tgz'
+
+ - name: 'Windows x64: Windows SDK 10.0 / MSVC v142 / VS 2019'
+ dependencies: []
+ task:
+ agent:
+ machine:
+ type: s1-prod-windows
+ env_vars:
+ # Disable vcpkg telemetry
+ - name: VCPKG_DISABLE_METRICS
+ value: 'yes'
+ prologue:
+ commands:
+ # install vcpkg in the parent directory.
+ - cd ..
+ # Setup vcpkg
+ - "& .\\librdkafka\\win32\\setup-vcpkg.ps1"
+ - cd librdkafka
+ epilogue:
+ commands:
+ - Get-ChildItem . -include *.dll -recurse
+ - Get-ChildItem . -include *.lib -recurse
+ - if ($env:SEMAPHORE_GIT_TAG_NAME -ne "") { artifact push workflow artifacts/ --destination artifacts/$Env:ARTIFACT_KEY/ }
+ jobs:
+ - name: 'Build: MSVC x64'
+ env_vars:
+ - name: triplet
+ value: x64-windows
+ - name: ARTIFACT_KEY
+ value: p-librdkafka__plat-windows__dist-msvc__arch-x64__lnk-std
+ commands:
+ - "& .\\win32\\msbuild.ps1 -platform x64"
+ - "& .\\win32\\package-zip.ps1 -platform x64"
+ - name: 'Build: MSVC x86'
+ env_vars:
+ - name: triplet
+ value: x86-windows
+ - name: ARTIFACT_KEY
+ value: p-librdkafka__plat-windows__dist-msvc__arch-x86__lnk-std
+ commands:
+ - "& .\\win32\\msbuild.ps1 -platform Win32"
+ - "& .\\win32\\package-zip.ps1 -platform Win32"
+
+promotions:
+ - name: Run all tests on master commits
+ pipeline_file: run-all-tests.yml
+ parameters:
+ env_vars:
+ - required: true
+ name: TEST_KAFKA_GIT_REF
+ default_value: 4.0.0
+ - required: true
+ name: TEST_TYPE
+ default_value: plaintext,ssl
+ - required: true
+ name: TEST_ARCHES
+ default_value: x86_64,aarch64
+ - required: true
+ name: TEST_PARALLEL
+ default_value: "1"
+ auto_promote_on:
+ - result: passed
+ branch:
+ - "master"
diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml
index d346dbb47c..4930428329 100644
--- a/.semaphore/semaphore.yml
+++ b/.semaphore/semaphore.yml
@@ -2,22 +2,44 @@ version: v1.0
name: 'librdkafka build and release artifact pipeline'
agent:
machine:
- type: s1-prod-macos-13-5-arm64
+ type: s1-prod-ubuntu24-04-arm64-00
execution_time_limit:
hours: 3
+
+auto_cancel:
+ running:
+ when: "branch != 'master'"
+
global_job_config:
prologue:
commands:
- checkout
- mkdir artifacts
- mkdir dest
+ env_vars:
+ - name: CACHE_TAG
+ value: '7'
+ - name: CI
+ value: 'true'
+ - name: KAFKA_VERSION
+ value: '4.0.0'
+ - name: CP_VERSION
+ value: '8.0.0'
+ # Shows plain output from docker build (no progress bars)
+ - name: BUILDKIT_PROGRESS
+ value: plain
+ - name: MACOSX_DEPLOYMENT_TARGET
+ value: '13'
+
blocks:
- name: 'OSX arm64/m1'
dependencies: []
+ run:
+ when: "tag =~ '^v[0-9]\\.'"
task:
agent:
machine:
- type: s1-prod-macos-13-5-arm64
+ type: s1-macos-15-arm64-8
env_vars:
- name: ARTIFACT_KEY
value: p-librdkafka__plat-osx__arch-arm64__lnk-all
@@ -40,10 +62,12 @@ blocks:
- name: 'OSX x64'
dependencies: []
+ run:
+ when: "tag =~ '^v[0-9]\\.'"
task:
agent:
machine:
- type: s1-prod-macos-13-5-amd64
+ type: s1-macos-15-amd64-6
env_vars:
- name: ARTIFACT_KEY
value: p-librdkafka__plat-osx__arch-x64__lnk-all
@@ -73,31 +97,27 @@ blocks:
task:
agent:
machine:
- type: s1-prod-ubuntu20-04-amd64-2
+ type: s1-prod-ubuntu24-04-arm64-00
jobs:
- name: 'Style check'
commands:
- - sudo apt install -y clang-format-10 python3 python3-pip python3-setuptools
- - python3 -m pip install -r packaging/tools/requirements.txt
- - CLANG_FORMAT=clang-format-10 make style-check
-
+ - packaging/tools/run-in-docker.sh test-runner-arm64-${CACHE_TAG} packaging/tools/run-style-check.sh
- name: 'Build documentation'
dependencies: []
task:
agent:
machine:
- type: s1-prod-ubuntu20-04-amd64-2
+ type: s1-prod-ubuntu24-04-arm64-1
jobs:
- name: 'Generate documentation'
commands:
- - sudo apt install -y doxygen graphviz
- - make docs
+ - packaging/tools/run-in-docker.sh test-runner-arm64-${CACHE_TAG} make docs
- (cd staging-docs && tar cvzf ../artifacts/librdkafka-docs.tgz .)
- '[[ -z $SEMAPHORE_GIT_TAG_NAME ]] || artifact push workflow artifacts/librdkafka-docs.tgz --destination artifacts/librdkafka-docs.tgz'
- - name: 'Linux Ubuntu x64: source build'
+ - name: 'Linux Ubuntu amd64: source build'
dependencies: []
skip:
# Skip for release tags, we don't want flaky CI tests
@@ -106,7 +126,7 @@ blocks:
task:
agent:
machine:
- type: s1-prod-ubuntu20-04-amd64-2
+ type: s1-prod-ubuntu24-04-amd64-1
env_vars:
- name: CFLAGS
value: -std=gnu90 # Test minimum C standard, default in CentOS 7
@@ -122,25 +142,7 @@ blocks:
- ./packaging/tools/build-configurations-checks.sh
- name: 'Build and integration tests'
commands:
- - wget -O rapidjson-dev.deb https://launchpad.net/ubuntu/+archive/primary/+files/rapidjson-dev_1.1.0+dfsg2-3_all.deb
- - sudo dpkg -i rapidjson-dev.deb
- - python3 -m pip install -U pip
- - python3 -m pip -V
- - (cd tests && python3 -m pip install -r requirements.txt)
- - ./configure --install-deps
- # split these up
- - ./packaging/tools/rdutcoverage.sh
- - make copyright-check
- - make -j all examples check
- - echo "Verifying that CONFIGURATION.md does not have manual changes"
- - git diff --exit-code CONFIGURATION.md
- - examples/rdkafka_example -X builtin.features
- - ldd src/librdkafka.so.1
- - ldd src-cpp/librdkafka++.so.1
- - make -j -C tests build
- - make -C tests run_local_quick
- - DESTDIR="$PWD/dest" make install
- - (cd tests && python3 -m trivup.clusters.KafkaCluster --version 3.4.0 --cmd 'make quick')
+ - packaging/tools/run-in-docker.sh test-runner-amd64-${CACHE_TAG} ./packaging/tools/build-configurations-checks.sh
- name: 'Linux x64: release artifact docker builds'
@@ -150,7 +152,7 @@ blocks:
task:
agent:
machine:
- type: s1-prod-ubuntu20-04-amd64-2
+ type: s1-prod-ubuntu24-04-amd64-1
prologue:
commands:
- '[[ -z $DOCKERHUB_APIKEY ]] || docker login --username $DOCKERHUB_USER --password $DOCKERHUB_APIKEY'
@@ -189,10 +191,12 @@ blocks:
- name: 'Linux arm64: release artifact docker builds'
dependencies: []
+ run:
+ when: "tag =~ '^v[0-9]\\.'"
task:
agent:
machine:
- type: s1-prod-ubuntu20-04-arm64-1
+ type: s1-prod-ubuntu24-04-arm64-1
prologue:
commands:
- '[[ -z $DOCKERHUB_APIKEY ]] || docker login --username $DOCKERHUB_USER --password $DOCKERHUB_APIKEY'
@@ -231,6 +235,8 @@ blocks:
- name: 'Windows x64: MinGW-w64'
dependencies: []
+ run:
+ when: "tag =~ '^v[0-9]\\.'"
task:
agent:
machine:
@@ -264,6 +270,8 @@ blocks:
- name: 'Windows x64: Windows SDK 10.0 / MSVC v142 / VS 2019'
dependencies: []
+ run:
+ when: "tag =~ '^v[0-9]\\.'"
task:
agent:
machine:
@@ -275,20 +283,10 @@ blocks:
prologue:
commands:
# install vcpkg in the parent directory.
- - pwd
- cd ..
# Setup vcpkg
- "& .\\librdkafka\\win32\\setup-vcpkg.ps1"
- cd librdkafka
- - ..\vcpkg\vcpkg integrate install
- # Install required packages.
- - ..\vcpkg\vcpkg --feature-flags=versions install --triplet $Env:triplet
- - cd ..
- - pwd
- - ls vcpkg/
- - echo $Env:VCPKG_ROOT
- - pwd
- - cd librdkafka
epilogue:
commands:
- Get-ChildItem . -include *.dll -recurse
@@ -328,7 +326,7 @@ blocks:
task:
agent:
machine:
- type: s1-prod-ubuntu20-04-amd64-2
+ type: s1-prod-ubuntu24-04-amd64-2
jobs:
- name: 'Build NuGet and static packages'
commands:
@@ -360,3 +358,29 @@ blocks:
# Upload all packages to project artifact store
- artifact push project packages --destination librdkafka-packages-${SEMAPHORE_GIT_TAG_NAME}-${SEMAPHORE_WORKFLOW_ID}
- echo Thank you
+
+promotions:
+ - name: Run local tests on all architectures and integration tests
+ pipeline_file: semaphore-integration.yml
+ auto_promote_on:
+ - result: passed
+ branch:
+ - "master"
+ # Manual promotion only, the auto promotion on master happens after
+ # integration tests pass.
+ - name: Run all tests
+ pipeline_file: run-all-tests.yml
+ parameters:
+ env_vars:
+ - required: true
+ name: TEST_KAFKA_GIT_REF
+ default_value: 4.0.0
+ - required: true
+ name: TEST_TYPE
+ default_value: plaintext,ssl
+ - required: true
+ name: TEST_ARCHES
+ default_value: x86_64,aarch64
+ - required: true
+ name: TEST_PARALLEL
+ default_value: "1"
diff --git a/.semaphore/verify-linux-packages.yml b/.semaphore/verify-linux-packages.yml
new file mode 100644
index 0000000000..cd9c579483
--- /dev/null
+++ b/.semaphore/verify-linux-packages.yml
@@ -0,0 +1,41 @@
+version: v1.0
+name: verify-linux-packages
+
+agent:
+ machine:
+ type: s1-prod-ubuntu24-04-amd64-1
+
+execution_time_limit:
+ hours: 2
+
+global_job_config:
+ prologue:
+ commands:
+ - checkout
+ env_vars:
+ - name: REPOSITORY_BASE_URL
+ value: "https://packages.confluent.io"
+
+blocks:
+ - name: "Verify Linux packages (x86_64)"
+ dependencies: []
+ task:
+ agent:
+ machine:
+ type: s1-prod-ubuntu24-04-amd64-2
+ jobs:
+ - name: "Verify packages"
+ commands:
+ - cd packaging/cp
+ - ./verify-packages.sh ${REPOSITORY_BASE_URL} linux/amd64 ${LIBRDKAFKA_VERSION}
+ - name: "Verify Linux packages (aarch64)"
+ dependencies: []
+ task:
+ agent:
+ machine:
+ type: s1-prod-ubuntu24-04-arm64-2
+ jobs:
+ - name: "Verify packages"
+ commands:
+ - cd packaging/cp
+ - ./verify-packages.sh ${REPOSITORY_BASE_URL} linux/arm64 ${LIBRDKAFKA_VERSION}
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 68142d0d3c..8e2f77e0b5 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,5 +1,619 @@
+# librdkafka v2.12.1
+
+librdkafka v2.12.1 is a maintenance release:
+
+* Restored macOS binaries compatibility with macOS 13 and 14 (#5219).
+
+
+## Fixes
+
+### General fixes
+
+* Fix to restore macOS 13 and 14 compatibility in prebuilt binaries present in `librdkafka.redist`.
+ Happening since 2.12.0 (#5219).
+
+
+
+# librdkafka v2.12.0
+
+librdkafka v2.12.0 is a feature release:
+
+## [KIP-848](https://cwiki.apache.org/confluence/display/KAFKA/KIP-848%3A+The+Next+Generation+of+the+Consumer+Rebalance+Protocol) – General Availability
+
+Starting with **librdkafka 2.12.0**, the next generation consumer group rebalance protocol defined in **[KIP-848](https://cwiki.apache.org/confluence/display/KAFKA/KIP-848%3A+The+Next+Generation+of+the+Consumer+Rebalance+Protocol)** is **production-ready**. Please refer the following [migration guide](INTRODUCTION.md#next-generation-consumer-group-protocol-kip-848) for moving from `classic` to `consumer` protocol.
+
+**Note:** The new consumer group protocol defined in [KIP-848](https://cwiki.apache.org/confluence/display/KAFKA/KIP-848%3A+The+Next+Generation+of+the+Consumer+Rebalance+Protocol) is not enabled by default. There are few contract change associated with the new protocol and might cause breaking changes. `group.protocol` configuration property dictates whether to use the new `consumer` protocol or older `classic` protocol. It defaults to `classic` if not provided.
+
+## Enhancements and Fixes
+* Support for OAUTHBEARER metadata based authentication types,
+ starting with Azure IMDS. [Introduction available](INTRODUCTION.md#oauthbearer-oidc-metadata-authentication) (#5155).
+* Fix compression types read issue in GetTelemetrySubscriptions response
+ for big-endian architectures (#5183, @paravoid).
+* Fix for KIP-1102 time based re-bootstrap condition (#5177).
+* Fix for discarding the member epoch in a consumer group heartbeat response when leaving with an inflight HB (#4672).
+* Fix for an error being raised after a commit due to an existing error in the topic partition (#4672).
+* Fix double free of headers in `rd_kafka_produceva` method (@blindspotbounty, #4628).
+* Fix to ensure `rd_kafka_query_watermark_offsets` enforces the specified timeout and does not continue beyond timeout expiry (#5201).
+* New [walkthrough](https://github.com/confluentinc/librdkafka/wiki/Using-SASL-GSSAPI-with-librdkafka-in-a-cross%E2%80%90realm-scenario-with-Windows-SSPI-and-MIT-Kerberos) in the Wiki about configuring Kafka cross-realm authentication between Windows SSPI and MIT Kerberos.
+
+
+## Fixes
+
+### General fixes
+
+* Issues: #5178.
+ Fix for KIP-1102 time based re-bootstrap condition.
+ Re-bootstrap is now triggered only after `metadata.recovery.rebootstrap.trigger.ms`
+ have passed since first metadata refresh request after last successful
+ metadata response. The calculation was since last successful metadata response
+ so it's possible it did overlap with the periodic `topic.metadata.refresh.interval.ms`
+ and cause a re-bootstrap even if not needed.
+ Happening since 2.11.0 (#5177).
+* Issues: #4878.
+ Fix to ensure `rd_kafka_query_watermark_offsets` enforces the specified timeout and does not continue beyond timeout expiry.
+ Happening since 2.3.0 (#5201).
+
+### Telemetry fixes
+
+* Issues: #5179 .
+ Fix issue in GetTelemetrySubscriptions with big-endian
+ architectures where wrong values are read as
+ accepted compression types causing the metrics to be sent uncompressed.
+ Happening since 2.5.0. Since 2.10.1 unit tests are failing when run on
+ big-endian architectures (#5183, @paravoid).
+
+### Consumer fixes
+
+ * Issues: #5199
+ Fixed an issue where topic partition errors were not cleared after a successful
+ commit. Previously, a partition could retain a stale error state even though the
+ most recent commit succeeded, causing misleading error reporting. Now, successful
+ commits correctly clear the error state for the affected partitions
+ Happening since 2.4.0 (#4672).
+
+### Producer fixes
+
+* Issues: #4627.
+ Fix double free of headers in `rd_kafka_produceva` method in cases where the partition doesn't exist.
+ Happening since 1.x (@blindspotbounty, #4628).
+
+
+# librdkafka v2.11.1
+
+librdkafka v2.11.1 is a maintenance release:
+
+* Made the conditions for enabling the features future proof (#5130).
+* Avoid returning an all brokers down error on planned disconnections (#5126).
+* An "all brokers down" error isn't returned when we haven't tried to connect
+ to all brokers since last successful connection (#5126).
+
+
+## Fixes
+
+### General fixes
+
+* Issues: #4948, #4956.
+ Made the conditions for enabling the features future proof, allowing to
+ remove RPC versions in a subsequent Apache Kafka version without disabling
+ features. The existing checks were matching a single version instead of
+ a range and were failing if the older version was removed.
+ Happening since 1.x (#5130).
+
+* Issues: #5142.
+ Avoid returning an all brokers down error on planned disconnections.
+ This is done by avoiding to count planned disconnections, such as idle
+ disconnections, broker host change and similar as events that can cause
+ the client to reach the "all brokers down" state, returning an error and
+ since 2.10.0 possibly starting a re-bootstrap sequence.
+ Happening since 1.x (#5126).
+
+* Issues: #5142.
+ An "all brokers down" error isn't returned when we haven't tried to connect
+ to all brokers since last successful connection. It happened because the down
+ state is cached and can be stale when a connection isn't needed to that
+ particular broker. Solved by resetting the cached broker down state when any
+ broker successfully connects, so that broker needs to be tried again.
+ Happening since 1.x (#5126).
+
+
+
+# librdkafka v2.11.0
+
+librdkafka v2.11.0 is a feature release:
+
+* [KIP-1102](https://cwiki.apache.org/confluence/display/KAFKA/KIP-1102%3A+Enable+clients+to+rebootstrap+based+on+timeout+or+error+code) Enable clients to rebootstrap based on timeout or error code (#4981).
+* [KIP-1139](https://cwiki.apache.org/confluence/display/KAFKA/KIP-1139%3A+Add+support+for+OAuth+jwt-bearer+grant+type) Add support for OAuth jwt-bearer grant type (#4978).
+* Fix for poll ratio calculation in case the queues are forwarded (#5017).
+* Fix data race when buffer queues are being reset instead of being
+ initialized (#4718).
+* Features BROKER_BALANCED_CONSUMER and SASL_GSSAPI don't depend on
+ JoinGroup v0 anymore, missing in AK 4.0 and CP 8.0 (#5131).
+* Improve HTTPS CA certificates configuration by probing several paths
+ when OpenSSL is statically linked and providing a way to customize their location
+ or value (#5133).
+
+
+## Fixes
+
+### General fixes
+
+* Issues: #4522.
+ A data race happened when emptying buffers of a failing broker, in its thread,
+ with the statistics callback in main thread gathering the buffer counts.
+ Solved by resetting the atomic counters instead of initializing them.
+ Happening since 1.x (#4718).
+* Issues: #4948
+ Features BROKER_BALANCED_CONSUMER and SASL_GSSAPI don't depend on
+ JoinGroup v0 anymore, missing in AK 4.0 and CP 8.0. This PR partially
+ fixes the linked issue, a complete fix for all features will follow.
+ Rest of fixes are necessary only for a subsequent Apache Kafka major
+ version (e.g. AK 5.x).
+ Happening since 1.x (#5131).
+
+### Telemetry fixes
+
+* Issues: #5109
+ Fix for poll ratio calculation in case the queues are forwarded.
+ Poll ratio is now calculated per-queue instead of per-instance and
+ it allows to avoid calculation problems linked to using the same
+ field.
+ Happens since 2.6.0 (#5017).
+
+
+
+# librdkafka v2.10.1
+
+librdkafka v2.10.1 is a maintenance release:
+
+* Fix to add locks when updating the metadata cache for the consumer
+ after no broker connection is available (@marcin-krystianc, #5066).
+* Fix to the re-bootstrap case when `bootstrap.servers` is `NULL` and
+ brokers were added manually through `rd_kafka_brokers_add` (#5067).
+* Fix an issue where the first message to any topic produced via `producev` or
+ `produceva` was delivered late (by up to 1 second) (#5032).
+* Fix for a loop of re-bootstrap sequences in case the client reaches the
+ `all brokers down` state (#5086).
+* Fix for frequent disconnections on push telemetry requests
+ with particular metric configurations (#4912).
+* Avoid copy outside boundaries when reading metric names in telemetry
+ subscription (#5105)
+* Metrics aren't duplicated when multiple prefixes match them (#5104)
+
+
+## Fixes
+
+### General fixes
+
+* Issues: #5088.
+ Fix for a loop of re-bootstrap sequences in case the client reaches the
+ `all brokers down` state. The client continues to select the
+ bootstrap brokers given they have no connection attempt and doesn't
+ re-connect to the learned ones. In case it happens a broker restart
+ can break the loop for the clients using the affected version.
+ Fixed by giving a higher chance to connect to the learned brokers
+ even if there are new ones that never tried to connect.
+ Happens since 2.10.0 (#5086).
+* Issues: #5057.
+ Fix to the re-bootstrap case when `bootstrap.servers` is `NULL` and
+ brokers were added manually through `rd_kafka_brokers_add`.
+ Avoids a segmentation fault in this case.
+ Happens since 2.10.0 (#5067).
+
+### Producer fixes
+
+* In case of `producev` or `produceva`, the producer did not enqueue a leader
+ query metadata request immediately, and rather, waited for the 1 second
+ timer to kick in. This could cause delays in the sending of the first message
+ by up to 1 second.
+ Happens since 1.x (#5032).
+
+### Consumer fixes
+
+* Issues: #5051.
+ Fix to add locks when updating the metadata cache for the consumer.
+ It can cause memory corruption or use-after-free in case
+ there's no broker connection and the consumer
+ group metadata needs to be updated.
+ Happens since 2.10.0 (#5066).
+
+### Telemetry fixes
+
+* Issues: #5106.
+ Fix for frequent disconnections on push telemetry requests
+ with particular metric configurations.
+ A `NULL` payload is sent in a push telemetry request when
+ an empty one is needed. This causes disconnections every time the
+ push is sent, only when metrics are requested and
+ some metrics are matching the producer but none the consumer
+ or the other way around.
+ Happens since 2.5.0 (#4912).
+* Issues: #5102.
+ Avoid copy outside boundaries when reading metric names in telemetry
+ subscription. It can cause that some metrics aren't matched.
+ Happens since 2.5.0 (#5105).
+* Issues: #5103.
+ Telemetry metrics aren't duplicated when multiple prefixes match them.
+ Fixed by keeping track of the metrics that already matched.
+ Happens since 2.5.0 (#5104).
+
+
+
+# librdkafka v2.10.0
+
+librdkafka v2.10.0 is a feature release:
+
+> [!WARNING] it's suggested to upgrade to 2.10.1 or later
+> because of the possibly critical bug #5088
+
+## [KIP-848](https://cwiki.apache.org/confluence/display/KAFKA/KIP-848%3A+The+Next+Generation+of+the+Consumer+Rebalance+Protocol) – Now in **Preview**
+
+- [KIP-848](https://cwiki.apache.org/confluence/display/KAFKA/KIP-848%3A+The+Next+Generation+of+the+Consumer+Rebalance+Protocol) has transitioned from *Early Access* to *Preview*.
+- Added support for **regex-based subscriptions**.
+- Implemented client-side member ID generation as per [KIP-1082](https://cwiki.apache.org/confluence/display/KAFKA/KIP-1082%3A+Require+Client-Generated+IDs+over+the+ConsumerGroupHeartbeat+RPC).
+- `rd_kafka_DescribeConsumerGroups()` now supports KIP-848-style `consumer` groups. Two new fields have been added:
+ - **Group type** – Indicates whether the group is `classic` or `consumer`.
+ - **Target assignment** – Applicable only to `consumer` protocol groups (defaults to `NULL`).
+- Group configuration is now supported in `AlterConfigs`, `IncrementalAlterConfigs`, and `DescribeConfigs`. ([#4939](https://github.com/confluentinc/librdkafka/pull/4939))
+- Added **Topic Authorization Error** support in the `ConsumerGroupHeartbeat` response.
+- Removed usage of the `partition.assignment.strategy` property for the `consumer` group protocol. An error will be raised if this is set with `group.protocol=consumer`.
+- Deprecated and disallowed the following properties for the `consumer` group protocol:
+ - `session.timeout.ms`
+ - `heartbeat.interval.ms`
+ - `group.protocol.type`
+ Attempting to set any of these will result in an error.
+- Enhanced handling for `subscribe()` and `unsubscribe()` edge cases.
+
+> [!Note]
+> The [KIP-848](https://cwiki.apache.org/confluence/display/KAFKA/KIP-848%3A+The+Next+Generation+of+the+Consumer+Rebalance+Protocol) consumer is currently in **Preview** and should not be used in production environments. Implementation is feature complete but contract could have minor changes before General Availability.
+
+
+ ## Upgrade considerations
+
+
+ Starting from this version, brokers not reported in Metadata RPC call are
+ removed along with their threads. Brokers and their threads are added back
+ when they appear in a Metadata RPC response again. When no brokers are left
+ or they're not reachable, the client will start a re-bootstrap sequence
+ by default. `metadata.recovery.strategy` controls this,
+ which defaults to `rebootstrap`.
+ Setting `metadata.recovery.strategy` to `none` avoids any re-bootstrapping and
+ leaves only the broker received in last successful metadata response.
+
+
+ ## Enhancements and Fixes
+
+ * [KIP-899](https://cwiki.apache.org/confluence/display/KAFKA/KIP-899%3A+Allow+producer+and+consumer+clients+to+rebootstrap) Allow producer and consumer clients to rebootstrap
+ * Identify brokers only by broker id (#4557, @mfleming)
+ * Remove unavailable brokers and their thread (#4557, @mfleming)
+ * Commits during a cooperative incremental rebalance aren't causing
+ an assignment lost if the generation id was bumped in between (#4908).
+ * Fix for librdkafka yielding before timeouts had been reached (#4970)
+ * Removed a 500ms latency when a consumer partition switches to a different
+ leader (#4970)
+ * The mock cluster implementation removes brokers from Metadata response
+ when they're not available, this simulates better the actual behavior of
+ a cluster that is using KRaft (#4970).
+ * Doesn't remove topics from cache on temporary Metadata errors but only
+ on metadata cache expiry (#4970).
+ * Doesn't mark the topic as unknown if it had been marked as existent earlier
+ and `topic.metadata.propagation.max.ms` hasn't passed still (@marcin-krystianc, #4970).
+ * Doesn't update partition leaders if the topic in metadata
+ response has errors (#4970).
+ * Only topic authorization errors in a metadata response are considered
+ permanent and are returned to the user (#4970).
+ * The function `rd_kafka_offsets_for_times` refreshes leader information
+ if the error requires it, allowing it to succeed on
+ subsequent manual retries (#4970).
+ * Deprecated `api.version.request`, `api.version.fallback.ms` and
+ `broker.version.fallback` configuration properties (#4970).
+ * When consumer is closed before destroying the client, the operations queue
+ isn't purged anymore as it contains operations
+ unrelated to the consumer group (#4970).
+ * When making multiple changes to the consumer subscription in a short time,
+ no unknown topic error is returned for topics that are in the new subscription but weren't in previous one (#4970).
+ * Prevent metadata cache corruption when topic id changes
+ (@kwdubuc, @marcin-krystianc, @GerKr, #4970).
+ * Fix for the case where a metadata refresh enqueued on an unreachable broker
+ prevents refreshing the controller or the coordinator until that broker
+ becomes reachable again (#4970).
+ * Remove a one second wait after a partition fetch is restarted following a
+ leader change and offset validation (#4970).
+ * Fix the Nagle algorithm (TCP_NODELAY) on broker sockets to not be enabled
+ by default (#4986).
+
+
+## Fixes
+
+### General fixes
+
+ * Issues: #4212
+ Identify brokers only by broker id, as happens in Java,
+ avoid to find the broker with same hostname and use the same thread
+ and connection.
+ Happens since 1.x (#4557, @mfleming).
+ * Issues: #4557
+ Remove brokers not reported in a metadata call, along with their threads.
+ Avoids that unavailable brokers are selected for a new connection when
+ there's no one available. We cannot tell if a broker was removed
+ temporarily or permanently so we always remove it and it'll be added back when
+ it becomes available again.
+ Happens since 1.x (#4557, @mfleming).
+ * Issues: #4970
+ librdkafka code using `cnd_timedwait` was yielding before a timeout occurred
+ without the condition being fulfilled because of spurious wake-ups.
+ Solved by verifying with a monotonic clock that the expected point in time
+ was reached and calling the function again if needed.
+ Happens since 1.x (#4970).
+ * Issues: #4970
+ Doesn't remove topics from cache on temporary Metadata errors but only
+ on metadata cache expiry. It allows the client to continue working
+ in case of temporary problems to the Kafka metadata plane.
+ Happens since 1.x (#4970).
+ * Issues: #4970
+ Doesn't mark the topic as unknown if it had been marked as existent earlier
+ and `topic.metadata.propagation.max.ms` hasn't passed still. It achieves
+ this property expected effect even if a different broker had
+ previously reported the topic as existent.
+ Happens since 1.x (@marcin-krystianc, #4970).
+ * Issues: #4907
+ Doesn't update partition leaders if the topic in metadata
+ response has errors. It's in line with what Java client does and allows
+ to avoid segmentation faults for unknown partitions.
+ Happens since 1.x (#4970).
+ * Issues: #4970
+ Only topic authorization errors in a metadata response are considered
+ permanent and are returned to the user. It's in line with what Java client
+ does and avoids returning to the user an error that wasn't meant to be
+ permanent.
+ Happens since 1.x (#4970).
+ * Issues: #4964, #4778
+ Prevent metadata cache corruption when topic id for the same topic name
+ changes. Solved by correctly removing the entry with the old topic id from metadata cache
+ to prevent subsequent use-after-free.
+ Happens since 2.4.0 (@kwdubuc, @marcin-krystianc, @GerKr, #4970).
+ * Issues: #4970
+ Fix for the case where a metadata refresh enqueued on an unreachable broker
+ prevents refreshing the controller or the coordinator until that broker
+ becomes reachable again. Given the request continues to be retried on that
+ broker, the counter for refreshing complete broker metadata doesn't reach
+ zero and prevents the client from obtaining the new controller or group or transactional coordinator.
+ It causes a series of debug messages like:
+ "Skipping metadata request: ... full request already in-transit", until
+ the broker the request is enqueued on is up again.
+ Solved by not retrying these kinds of metadata requests.
+ Happens since 1.x (#4970).
+ * The Nagle algorithm (TCP_NODELAY) is now disabled by default. It caused a
+ large increase in latency for some use cases, for example, when using an
+ SSL connection.
+ For efficient batching, the application should use `linger.ms`,
+ `batch.size` etc.
+ Happens since: 0.x (#4986).
+
+### Consumer fixes
+
+ * Issues: #4059
+ Commits during a cooperative incremental rebalance could cause an
+ assignment lost if the generation id was bumped by a second join
+ group request.
+ Solved by not rejoining the group in case an illegal generation error happens
+ during a rebalance.
+ Happening since v1.6.0 (#4908)
+ * Issues: #4970
+ When switching to a different leader a consumer could wait 500ms
+ (`fetch.error.backoff.ms`) before starting to fetch again. The fetch backoff wasn't reset when joining the new broker.
+ Solved by resetting it, given it's not needed to backoff
+ the first fetch on a different node. This way faster leader switches are
+ possible.
+ Happens since 1.x (#4970).
+ * Issues: #4970
+ The function `rd_kafka_offsets_for_times` refreshes leader information
+ if the error requires it, allowing it to succeed on
+ subsequent manual retries. Similar to the fix done in 2.3.0 in
+ `rd_kafka_query_watermark_offsets`. Additionally, the partition
+ current leader epoch is taken from metadata cache instead of
+ from passed partitions.
+ Happens since 1.x (#4970).
+ * Issues: #4970
+ When consumer is closed before destroying the client, the operations queue
+ isn't purged anymore as it contains operations
+ unrelated to the consumer group.
+ Happens since 1.x (#4970).
+ * Issues: #4970
+ When making multiple changes to the consumer subscription in a short time,
+ no unknown topic error is returned for topics that are in the new subscription
+ but weren't in previous one. This was due to the metadata request relative
+ to previous subscription.
+ Happens since 1.x (#4970).
+ * Issues: #4970
+ Remove a one second wait after a partition fetch is restarted following a
+ leader change and offset validation. This is done by resetting the fetch
+ error backoff and waking up the delegated broker if present.
+ Happens since 2.1.0 (#4970).
+
+
+
+*Note: there was no v2.9.0 librdkafka release,
+ it was a dependent clients release only*
+
+
+
+# librdkafka v2.8.0
+
+librdkafka v2.8.0 is a maintenance release:
+
+* Socket options are now all set before connection (#4893).
+* Client certificate chain is now sent when using `ssl.certificate.pem`
+ or `ssl_certificate` or `ssl.keystore.location` (#4894).
+* Avoid sending client certificates whose chain doesn't match with broker
+ trusted root certificates (#4900).
+* Fixes to allow to migrate partitions to leaders with same leader epoch,
+ or NULL leader epoch (#4901).
+* Support versions of OpenSSL without the ENGINE component (Chris Novakovic, #3535
+ and @remicollet, #4911).
+
+
+## Fixes
+
+### General fixes
+
+* Socket options are now all set before connection, as [documentation](https://man7.org/linux/man-pages/man7/tcp.7.html)
+ says it's needed for socket buffers to take effect, even if in some
+ cases they could have effect even after connection.
+ Happening since v0.9.0 (#4893).
+* Issues: #3225.
+ Client certificate chain is now sent when using `ssl.certificate.pem`
+ or `ssl_certificate` or `ssl.keystore.location`.
+ Without that, broker must explicitly add any intermediate certification
+ authority certificate to its truststore to be able to accept client
+ certificate.
+ Happens since: 1.x (#4894).
+
+### Consumer fixes
+
+* Issues: #4796.
+ Fix to allow to migrate partitions to leaders with NULL leader epoch.
+ NULL leader epoch can happen during a cluster roll with an upgrade to a
+ version supporting KIP-320.
+ Happening since v2.1.0 (#4901).
+* Issues: #4804.
+ Fix to allow to migrate partitions to leaders with same leader epoch.
+ Same leader epoch can happen when partition is
+ temporarily migrated to the internal broker (#4804), or if broker implementation
+ never bumps it, as it's not needed to validate the offsets.
+ Happening since v2.4.0 (#4901).
+
+
+*Note: there was no v2.7.0 librdkafka release*
+
+
+# librdkafka v2.6.1
+
+librdkafka v2.6.1 is a maintenance release:
+
+* Fix for a Fetch regression when connecting to Apache Kafka < 2.7 (#4871).
+* Fix for an infinite loop happening with cooperative-sticky assignor
+ under some particular conditions (#4800).
+* Fix for retrieving offset commit metadata when it contains
+ zeros and configured with `strndup` (#4876)
+* Fix for a loop of ListOffset requests, happening in a Fetch From Follower
+ scenario, if such request is made to the follower (#4616, #4754, @kphelps).
+* Fix to remove fetch queue messages that blocked the destroy of rdkafka
+ instances (#4724)
+* Upgrade Linux dependencies: OpenSSL 3.0.15, CURL 8.10.1 (#4875).
+* Upgrade Windows dependencies: MSVC runtime to 14.40.338160.0,
+ zstd 1.5.6, zlib 1.3.1, OpenSSL 3.3.2, CURL 8.10.1 (#4872).
+* SASL/SCRAM authentication fix: avoid concatenating
+ client side nonce once more, as it's already prepended in server sent nonce (#4895).
+* Allow retrying for status code 429 ('Too Many Requests') in HTTP requests for
+ OAUTHBEARER OIDC (#4902).
+
+## Fixes
+
+### General fixes
+
+* SASL/SCRAM authentication fix: avoid concatenating
+ client side nonce once more, as it's already prepended in
+ server sent nonce.
+ librdkafka was incorrectly concatenating the client side nonce again, leading to [this fix](https://github.com/apache/kafka/commit/0a004562b8475d48a9961d6dab3a6aa24021c47f) being made on AK side, released with 3.8.1, with `endsWith` instead of `equals`.
+ Happening since v0.0.99 (#4895).
+
+### Consumer fixes
+
+* Issues: #4870
+ Fix for a Fetch regression when connecting to Apache Kafka < 2.7, causing
+ fetches to fail.
+ Happening since v2.6.0 (#4871)
+* Issues: #4783.
+ A consumer configured with the `cooperative-sticky` partition assignment
+ strategy could get stuck in an infinite loop, with corresponding spike of
+ main thread CPU usage.
+ That happened with some particular orders of members and potential
+ assignable partitions.
+ Solved by removing the infinite loop cause.
+ Happening since: 1.6.0 (#4800).
+* Issues: #4649.
+ When retrieving offset metadata, if the binary value contained zeros
+ and librdkafka was configured with `strndup`, part of
+ the buffer after first zero contained uninitialized data
+ instead of rest of metadata. Solved by avoiding to use
+ `strndup` for copying metadata.
+ Happening since: 0.9.0 (#4876).
+* Issues: #4616
+ When an out of range on a follower caused an offset reset, the corresponding
+ ListOffsets request is made to the follower, causing a repeated
+ "Not leader for partition" error. Fixed by sending the request always
+ to the leader.
+ Happening since 1.5.0 (tested version) or previous ones (#4616, #4754, @kphelps).
+* Issues:
+ Fix to remove fetch queue messages that blocked the destroy of rdkafka
+ instances. Circular dependencies from a partition fetch queue message to
+ the same partition blocked the destroy of an instance, that happened
+ in case the partition was removed from the cluster while it was being
+ consumed. Solved by purging internal partition queue, after being stopped
+ and removed, to allow reference count to reach zero and trigger a destroy.
+ Happening since 2.0.2 (#4724).
+
+
+
+# librdkafka v2.6.0
+
+librdkafka v2.6.0 is a feature release:
+
+ * [KIP-460](https://cwiki.apache.org/confluence/display/KAFKA/KIP-460%3A+Admin+Leader+Election+RPC) Admin Leader Election RPC (#4845)
+ * [KIP-714] Complete consumer metrics support (#4808).
+ * [KIP-714] Produce latency average and maximum metrics support for parity with Java client (#4847).
+ * [KIP-848] ListConsumerGroups Admin API now has an optional filter to return only groups
+ of given types.
+ * Added Transactional id resource type for ACL operations (@JohnPreston, #4856).
+ * Fix for permanent fetch errors when using a newer Fetch RPC version with an older
+ inter broker protocol (#4806).
+
+
+
+## Fixes
+
+### Consumer fixes
+
+ * Issues: #4806
+ Fix for permanent fetch errors when brokers support a Fetch RPC version greater than 12
+ but cluster is configured to use an inter broker protocol that is less than 2.8.
+ In this case returned topic ids are zero valued and Fetch has to fall back
+ to version 12, using topic names.
+ Happening since v2.5.0 (#4806)
+
+
+
+# librdkafka v2.5.3
+
+librdkafka v2.5.3 is a feature release.
+
+* Fix an assert being triggered during push telemetry call when no metrics matched on the client side. (#4826)
+
+## Fixes
+
+### Telemetry fixes
+
+* Issue: #4833
+Fix a regression introduced with [KIP-714](https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability) support in which an assert is triggered during **PushTelemetry** call. This happens when no metric is matched on the client side among those requested by broker subscription.
+Happening since 2.5.0 (#4826).
+
+*Note: there were no v2.5.1 and v2.5.2 librdkafka releases*
+
+
# librdkafka v2.5.0
+> [!WARNING]
+This version has introduced a regression in which an assert is triggered during **PushTelemetry** call. This happens when no metric is matched on the client side among those requested by broker subscription.
+>
+> You won't face any problem if:
+> * Broker doesn't support [KIP-714](https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability).
+> * [KIP-714](https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability) feature is disabled on the broker side.
+> * [KIP-714](https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability) feature is disabled on the client side. This is enabled by default. Set configuration `enable.metrics.push` to `false`.
+> * If [KIP-714](https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability) is enabled on the broker side and there is no subscription configured there.
+> * If [KIP-714](https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability) is enabled on the broker side with subscriptions that match the [KIP-714](https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability) metrics defined on the client.
+>
+> Having said this, we strongly recommend using `v2.5.3` and above to not face this regression at all.
+
librdkafka v2.5.0 is a feature release.
* [KIP-951](https://cwiki.apache.org/confluence/display/KAFKA/KIP-951%3A+Leader+discovery+optimisations+for+the+client)
diff --git a/CONFIGURATION.md b/CONFIGURATION.md
index 4a44ee9797..40b7412efd 100644
--- a/CONFIGURATION.md
+++ b/CONFIGURATION.md
@@ -12,6 +12,8 @@ message.copy.max.bytes | * | 0 .. 1000000000 | 65535
receive.message.max.bytes | * | 1000 .. 2147483647 | 100000000 | medium | Maximum Kafka protocol response message size. This serves as a safety precaution to avoid memory exhaustion in case of protocol hickups. This value must be at least `fetch.max.bytes` + 512 to allow for protocol overhead; the value is adjusted automatically unless the configuration property is explicitly set.
*Type: integer*
max.in.flight.requests.per.connection | * | 1 .. 1000000 | 1000000 | low | Maximum number of in-flight requests per broker connection. This is a generic property applied to all broker communication, however it is primarily relevant to produce requests. In particular, note that other mechanisms limit the number of outstanding consumer fetch request per broker to one.
*Type: integer*
max.in.flight | * | 1 .. 1000000 | 1000000 | low | Alias for `max.in.flight.requests.per.connection`: Maximum number of in-flight requests per broker connection. This is a generic property applied to all broker communication, however it is primarily relevant to produce requests. In particular, note that other mechanisms limit the number of outstanding consumer fetch request per broker to one.
*Type: integer*
+metadata.recovery.strategy | * | none, rebootstrap | rebootstrap | low | Controls how the client recovers when none of the brokers known to it is available. If set to `none`, the client doesn't re-bootstrap. If set to `rebootstrap`, the client repeats the bootstrap process using `bootstrap.servers` and brokers added through `rd_kafka_brokers_add()`. Rebootstrapping is useful when a client communicates with brokers so infrequently that the set of brokers may change entirely before the client refreshes metadata. Metadata recovery is triggered when all last-known brokers appear unavailable simultaneously or the client cannot refresh metadata within `metadata.recovery.rebootstrap.trigger.ms` or it's requested in a metadata response.
*Type: enum value*
+metadata.recovery.rebootstrap.trigger.ms | * | 0 .. 2147483647 | 300000 | low | If a client configured to rebootstrap using `metadata.recovery.strategy=rebootstrap` is unable to obtain metadata from any of the brokers for this interval, client repeats the bootstrap process using `bootstrap.servers` configuration and brokers added through `rd_kafka_brokers_add()`.
*Type: integer*
topic.metadata.refresh.interval.ms | * | -1 .. 3600000 | 300000 | low | Period of time in milliseconds at which topic and broker metadata is refreshed in order to proactively discover any new brokers, topics, partitions or partition leader changes. Use -1 to disable the intervalled refresh (not recommended). If there are no locally referenced topics (no topic objects created, no messages produced, no subscription or no assignment) then only the broker list will be refreshed every interval but no more often than every 10s.
*Type: integer*
metadata.max.age.ms | * | 1 .. 86400000 | 900000 | low | Metadata cache max age. Defaults to topic.metadata.refresh.interval.ms * 3
*Type: integer*
topic.metadata.refresh.fast.interval.ms | * | 1 .. 60000 | 100 | low | When a topic loses its leader a new metadata request will be enqueued immediately and then with this initial interval, exponentially increasing upto `retry.backoff.max.ms`, until the topic metadata has been refreshed. If not set explicitly, it will be defaulted to `retry.backoff.ms`. This is used to recover quickly from transitioning leader brokers.
*Type: integer*
@@ -25,12 +27,12 @@ socket.blocking.max.ms | * | 1 .. 60000 | 1000
socket.send.buffer.bytes | * | 0 .. 100000000 | 0 | low | Broker socket send buffer size. System default is used if 0.
*Type: integer*
socket.receive.buffer.bytes | * | 0 .. 100000000 | 0 | low | Broker socket receive buffer size. System default is used if 0.
*Type: integer*
socket.keepalive.enable | * | true, false | false | low | Enable TCP keep-alives (SO_KEEPALIVE) on broker sockets
*Type: boolean*
-socket.nagle.disable | * | true, false | false | low | Disable the Nagle algorithm (TCP_NODELAY) on broker sockets.
*Type: boolean*
+socket.nagle.disable | * | true, false | true | low | Disable the Nagle algorithm (TCP_NODELAY) on broker sockets.
*Type: boolean*
socket.max.fails | * | 0 .. 1000000 | 1 | low | Disconnect from broker when this number of send failures (e.g., timed out requests) is reached. Disable with 0. WARNING: It is highly recommended to leave this setting at its default value of 1 to avoid the client and broker to become desynchronized in case of request timeouts. NOTE: The connection is automatically re-established.
*Type: integer*
broker.address.ttl | * | 0 .. 86400000 | 1000 | low | How long to cache the broker address resolving results (milliseconds).
*Type: integer*
broker.address.family | * | any, v4, v6 | any | low | Allowed broker IP address families: any, v4, v6
*Type: enum value*
socket.connection.setup.timeout.ms | * | 1000 .. 2147483647 | 30000 | medium | Maximum time allowed for broker connection setup (TCP connection setup as well SSL and SASL handshake). If the connection to the broker is not fully functional after this the connection will be closed and retried.
*Type: integer*
-connections.max.idle.ms | * | 0 .. 2147483647 | 0 | medium | Close broker connections after the specified time of inactivity. Disable with 0. If this property is left at its default value some heuristics are performed to determine a suitable default value, this is currently limited to identifying brokers on Azure (see librdkafka issue #3109 for more info).
*Type: integer*
+connections.max.idle.ms | * | 0 .. 2147483647 | 0 | medium | Close broker connections after the specified time of inactivity. Disable with 0. If this property is left at its default value some heuristics are performed to determine a suitable default value, this is currently limited to identifying brokers on Azure (see librdkafka issue #3109 for more info). Actual value can be lower, up to 2s lower, only if `connections.max.idle.ms` >= 4s, as jitter is added to avoid disconnecting all brokers at the same time.
*Type: integer*
reconnect.backoff.jitter.ms | * | 0 .. 3600000 | 0 | low | **DEPRECATED** No longer used. See `reconnect.backoff.ms` and `reconnect.backoff.max.ms`.
*Type: integer*
reconnect.backoff.ms | * | 0 .. 3600000 | 100 | medium | The initial time to wait before reconnecting to a broker after the connection has been closed. The time is increased exponentially until `reconnect.backoff.max.ms` is reached. -25% to +50% jitter is applied to each reconnect backoff. A value of 0 disables the backoff and reconnects immediately.
*Type: integer*
reconnect.backoff.max.ms | * | 0 .. 3600000 | 10000 | medium | The maximum time to wait before reconnecting to a broker after the connection has been closed.
*Type: integer*
@@ -54,10 +56,10 @@ resolve_cb | * | |
opaque | * | | | low | Application opaque (set with rd_kafka_conf_set_opaque())
*Type: see dedicated API*
default_topic_conf | * | | | low | Default topic configuration for automatically subscribed topics
*Type: see dedicated API*
internal.termination.signal | * | 0 .. 128 | 0 | low | Signal that librdkafka will use to quickly terminate on rd_kafka_destroy(). If this signal is not set then there will be a delay before rd_kafka_wait_destroyed() returns true as internal threads are timing out their system calls. If this signal is set however the delay will be minimal. The application should mask this signal as an internal signal handler is installed.
*Type: integer*
-api.version.request | * | true, false | true | high | Request broker's supported API versions to adjust functionality to available protocol features. If set to false, or the ApiVersionRequest fails, the fallback version `broker.version.fallback` will be used. **NOTE**: Depends on broker version >=0.10.0. If the request is not supported by (an older) broker the `broker.version.fallback` fallback is used.
*Type: boolean*
+api.version.request | * | true, false | true | high | **DEPRECATED** **Post-deprecation actions: remove this configuration property, brokers < 0.10.0 won't be supported anymore in librdkafka 3.x.** Request broker's supported API versions to adjust functionality to available protocol features. If set to false, or the ApiVersionRequest fails, the fallback version `broker.version.fallback` will be used. **NOTE**: Depends on broker version >=0.10.0. If the request is not supported by (an older) broker the `broker.version.fallback` fallback is used.
*Type: boolean*
api.version.request.timeout.ms | * | 1 .. 300000 | 10000 | low | Timeout for broker API version requests.
*Type: integer*
-api.version.fallback.ms | * | 0 .. 604800000 | 0 | medium | Dictates how long the `broker.version.fallback` fallback is used in the case the ApiVersionRequest fails. **NOTE**: The ApiVersionRequest is only issued when a new connection to the broker is made (such as after an upgrade).
*Type: integer*
-broker.version.fallback | * | | 0.10.0 | medium | Older broker versions (before 0.10.0) provide no way for a client to query for supported protocol features (ApiVersionRequest, see `api.version.request`) making it impossible for the client to know what features it may use. As a workaround a user may set this property to the expected broker version and the client will automatically adjust its feature set accordingly if the ApiVersionRequest fails (or is disabled). The fallback broker version will be used for `api.version.fallback.ms`. Valid values are: 0.9.0, 0.8.2, 0.8.1, 0.8.0. Any other value >= 0.10, such as 0.10.2.1, enables ApiVersionRequests.
*Type: string*
+api.version.fallback.ms | * | 0 .. 604800000 | 0 | medium | **DEPRECATED** **Post-deprecation actions: remove this configuration property, brokers < 0.10.0 won't be supported anymore in librdkafka 3.x.** Dictates how long the `broker.version.fallback` fallback is used in the case the ApiVersionRequest fails. **NOTE**: The ApiVersionRequest is only issued when a new connection to the broker is made (such as after an upgrade).
*Type: integer*
+broker.version.fallback | * | | 0.10.0 | medium | **DEPRECATED** **Post-deprecation actions: remove this configuration property, brokers < 0.10.0 won't be supported anymore in librdkafka 3.x.** Older broker versions (before 0.10.0) provide no way for a client to query for supported protocol features (ApiVersionRequest, see `api.version.request`) making it impossible for the client to know what features it may use. As a workaround a user may set this property to the expected broker version and the client will automatically adjust its feature set accordingly if the ApiVersionRequest fails (or is disabled). The fallback broker version will be used for `api.version.fallback.ms`. Valid values are: 0.9.0, 0.8.2, 0.8.1, 0.8.0. Any other value >= 0.10, such as 0.10.2.1, enables ApiVersionRequests.
*Type: string*
allow.auto.create.topics | * | true, false | false | low | Allow automatic topic creation on the broker when subscribing to or assigning non-existent topics. The broker must also be configured with `auto.create.topics.enable=true` for this configuration to take effect. Note: the default value (true) for the producer is different from the default value (false) for the consumer. Further, the consumer default value is different from the Java consumer (true), and this property is not supported by the Java producer. Requires broker version >= 0.11.0.0, for older broker versions only the broker configuration applies.
*Type: boolean*
security.protocol | * | plaintext, ssl, sasl_plaintext, sasl_ssl | plaintext | high | Protocol used to communicate with brokers.
*Type: enum value*
ssl.cipher.suites | * | | | low | A cipher suite is a named combination of authentication, encryption, MAC and key exchange algorithm used to negotiate the security settings for a network connection using TLS or SSL network protocol. See manual page for `ciphers(1)` and `SSL_CTX_set_cipher_list(3).
*Type: string*
@@ -71,6 +73,8 @@ ssl.certificate.location | * | |
ssl.certificate.pem | * | | | low | Client's public key string (PEM format) used for authentication.
*Type: string*
ssl_certificate | * | | | low | Client's public key as set by rd_kafka_conf_set_ssl_cert()
*Type: see dedicated API*
ssl.ca.location | * | | | low | File or directory path to CA certificate(s) for verifying the broker's key. Defaults: On Windows the system's CA certificates are automatically looked up in the Windows Root certificate store. On Mac OSX this configuration defaults to `probe`. It is recommended to install openssl using Homebrew, to provide CA certificates. On Linux install the distribution's ca-certificates package. If OpenSSL is statically linked or `ssl.ca.location` is set to `probe` a list of standard paths will be probed and the first one found will be used as the default CA certificate location path. If OpenSSL is dynamically linked the OpenSSL library's default path will be used (see `OPENSSLDIR` in `openssl version -a`).
*Type: string*
+https.ca.location | * | | | low | File or directory path to CA certificate(s) for verifying HTTPS endpoints, like `sasl.oauthbearer.token.endpoint.url` used for OAUTHBEARER/OIDC authentication. Mutually exclusive with `https.ca.pem`. Defaults: On Windows the system's CA certificates are automatically looked up in the Windows Root certificate store. On Mac OSX this configuration defaults to `probe`. It is recommended to install openssl using Homebrew, to provide CA certificates. On Linux install the distribution's ca-certificates package. If OpenSSL is statically linked or `https.ca.location` is set to `probe` a list of standard paths will be probed and the first one found will be used as the default CA certificate location path. If OpenSSL is dynamically linked the OpenSSL library's default path will be used (see `OPENSSLDIR` in `openssl version -a`).
*Type: string*
+https.ca.pem | * | | | low | CA certificate string (PEM format) for verifying HTTPS endpoints. Mutually exclusive with `https.ca.location`. Optional: see `https.ca.location`.
*Type: string*
ssl.ca.pem | * | | | low | CA certificate string (PEM format) for verifying the broker's key.
*Type: string*
ssl_ca | * | | | low | CA certificate as set by rd_kafka_conf_set_ssl_cert()
*Type: see dedicated API*
ssl.ca.certificate.stores | * | | Root | low | Comma-separated list of Windows Certificate stores to load CA certificates from. Certificates will be loaded in the same order as stores are specified. If no certificates can be loaded from any of the specified stores an error is logged and the OpenSSL library's default CA location is used instead. Store names are typically one or more of: MY, Root, Trust, CA.
*Type: string*
@@ -98,18 +102,34 @@ enable.sasl.oauthbearer.unsecure.jwt | * | true, false | false
oauthbearer_token_refresh_cb | * | | | low | SASL/OAUTHBEARER token refresh callback (set with rd_kafka_conf_set_oauthbearer_token_refresh_cb(), triggered by rd_kafka_poll(), et.al. This callback will be triggered when it is time to refresh the client's OAUTHBEARER token. Also see `rd_kafka_conf_enable_sasl_queue()`.
*Type: see dedicated API*
sasl.oauthbearer.method | * | default, oidc | default | low | Set to "default" or "oidc" to control which login method to be used. If set to "oidc", the following properties must also be be specified: `sasl.oauthbearer.client.id`, `sasl.oauthbearer.client.secret`, and `sasl.oauthbearer.token.endpoint.url`.
*Type: enum value*
sasl.oauthbearer.client.id | * | | | low | Public identifier for the application. Must be unique across all clients that the authorization server handles. Only used when `sasl.oauthbearer.method` is set to "oidc".
*Type: string*
+sasl.oauthbearer.client.credentials.client.id | * | | | low | Alias for `sasl.oauthbearer.client.id`: Public identifier for the application. Must be unique across all clients that the authorization server handles. Only used when `sasl.oauthbearer.method` is set to "oidc".
*Type: string*
+sasl.oauthbearer.client.credentials.client.secret | * | | | low | Alias for `sasl.oauthbearer.client.secret`: Client secret only known to the application and the authorization server. This should be a sufficiently random string that is not guessable. Only used when `sasl.oauthbearer.method` is set to "oidc".
*Type: string*
sasl.oauthbearer.client.secret | * | | | low | Client secret only known to the application and the authorization server. This should be a sufficiently random string that is not guessable. Only used when `sasl.oauthbearer.method` is set to "oidc".
*Type: string*
sasl.oauthbearer.scope | * | | | low | Client use this to specify the scope of the access request to the broker. Only used when `sasl.oauthbearer.method` is set to "oidc".
*Type: string*
sasl.oauthbearer.extensions | * | | | low | Allow additional information to be provided to the broker. Comma-separated list of key=value pairs. E.g., "supportFeatureX=true,organizationId=sales-emea".Only used when `sasl.oauthbearer.method` is set to "oidc".
*Type: string*
sasl.oauthbearer.token.endpoint.url | * | | | low | OAuth/OIDC issuer token endpoint HTTP(S) URI used to retrieve token. Only used when `sasl.oauthbearer.method` is set to "oidc".
*Type: string*
+sasl.oauthbearer.grant.type | * | client_credentials, urn:ietf:params:oauth:grant-type:jwt-bearer | client_credentials | low | OAuth grant type to use when communicating with the identity provider.
*Type: enum value*
+sasl.oauthbearer.assertion.algorithm | * | RS256, ES256 | RS256 | low | Algorithm the client should use to sign the assertion sent to the identity provider and in the OAuth alg header in the JWT assertion.
*Type: enum value*
+sasl.oauthbearer.assertion.private.key.file | * | | | low | Path to client's private key (PEM) used for authentication when using the JWT assertion.
*Type: string*
+sasl.oauthbearer.assertion.private.key.passphrase | * | | | low | Private key passphrase for `sasl.oauthbearer.assertion.private.key.file` or `sasl.oauthbearer.assertion.private.key.pem`.
*Type: string*
+sasl.oauthbearer.assertion.private.key.pem | * | | | low | Client's private key (PEM) used for authentication when using the JWT assertion.
*Type: string*
+sasl.oauthbearer.assertion.file | * | | | low | Path to the assertion file. Only used when `sasl.oauthbearer.method` is set to "oidc" and JWT assertion is needed.
*Type: string*
+sasl.oauthbearer.assertion.claim.aud | * | | | low | JWT audience claim. Only used when `sasl.oauthbearer.method` is set to "oidc" and JWT assertion is needed.
*Type: string*
+sasl.oauthbearer.assertion.claim.exp.seconds | * | 1 .. 2147483647 | 300 | low | Assertion expiration time in seconds. Only used when `sasl.oauthbearer.method` is set to "oidc" and JWT assertion is needed.
*Type: integer*
+sasl.oauthbearer.assertion.claim.iss | * | | | low | JWT issuer claim. Only used when `sasl.oauthbearer.method` is set to "oidc" and JWT assertion is needed.
*Type: string*
+sasl.oauthbearer.assertion.claim.jti.include | * | true, false | false | low | JWT ID claim. When set to `true`, a random UUID is generated. Only used when `sasl.oauthbearer.method` is set to "oidc" and JWT assertion is needed.
*Type: boolean*
+sasl.oauthbearer.assertion.claim.nbf.seconds | * | 0 .. 2147483647 | 60 | low | Assertion not before time in seconds. Only used when `sasl.oauthbearer.method` is set to "oidc" and JWT assertion is needed.
*Type: integer*
+sasl.oauthbearer.assertion.claim.sub | * | | | low | JWT subject claim. Only used when `sasl.oauthbearer.method` is set to "oidc" and JWT assertion is needed.
*Type: string*
+sasl.oauthbearer.assertion.jwt.template.file | * | | | low | Path to the JWT template file. Only used when `sasl.oauthbearer.method` is set to "oidc" and JWT assertion is needed.
*Type: string*
+sasl.oauthbearer.metadata.authentication.type | * | none, azure_imds | none | low | Type of metadata-based authentication to use for OAUTHBEARER/OIDC `azure_imds` authenticates using the Azure IMDS endpoint. Sets a default value for `sasl.oauthbearer.token.endpoint.url` if missing. Configuration values specific of chosen authentication type can be passed through `sasl.oauthbearer.config`.
*Type: enum value*
plugin.library.paths | * | | | low | List of plugin libraries to load (; separated). The library search path is platform dependent (see dlopen(3) for Unix and LoadLibrary() for Windows). If no filename extension is specified the platform-specific extension (such as .dll or .so) will be appended automatically.
*Type: string*
interceptors | * | | | low | Interceptors added through rd_kafka_conf_interceptor_add_..() and any configuration handled by interceptors.
*Type: see dedicated API*
group.id | C | | | high | Client group id string. All clients sharing the same group.id belong to the same group.
*Type: string*
group.instance.id | C | | | medium | Enable static group membership. Static group members are able to leave and rejoin a group within the configured `session.timeout.ms` without prompting a group rebalance. This should be used in combination with a larger `session.timeout.ms` to avoid group rebalances caused by transient unavailability (e.g. process restarts). Requires broker version >= 2.3.0.
*Type: string*
-partition.assignment.strategy | C | | range,roundrobin | medium | The name of one or more partition assignment strategies. The elected group leader will use a strategy supported by all members of the group to assign partitions to group members. If there is more than one eligible strategy, preference is determined by the order of this list (strategies earlier in the list have higher priority). Cooperative and non-cooperative (eager) strategies must not be mixed. Available strategies: range, roundrobin, cooperative-sticky.
*Type: string*
-session.timeout.ms | C | 1 .. 3600000 | 45000 | high | Client group session and failure detection timeout. The consumer sends periodic heartbeats (heartbeat.interval.ms) to indicate its liveness to the broker. If no hearts are received by the broker for a group member within the session timeout, the broker will remove the consumer from the group and trigger a rebalance. The allowed range is configured with the **broker** configuration properties `group.min.session.timeout.ms` and `group.max.session.timeout.ms`. Also see `max.poll.interval.ms`.
*Type: integer*
-heartbeat.interval.ms | C | 1 .. 3600000 | 3000 | low | Group session keepalive heartbeat interval.
*Type: integer*
-group.protocol.type | C | | consumer | low | Group protocol type for the `classic` group protocol. NOTE: Currently, the only supported group protocol type is `consumer`.
*Type: string*
+partition.assignment.strategy | C | | range,roundrobin | medium | The name of one or more partition assignment strategies. The elected group leader will use a strategy supported by all members of the group to assign partitions to group members. If there is more than one eligible strategy, preference is determined by the order of this list (strategies earlier in the list have higher priority). Cooperative and non-cooperative (eager)strategies must not be mixed. `partition.assignment.strategy` is not supported for `group.protocol=consumer`. Use `group.remote.assignor` instead. Available strategies: range, roundrobin, cooperative-sticky.
*Type: string*
+session.timeout.ms | C | 1 .. 3600000 | 45000 | high | Client group session and failure detection timeout. The consumer sends periodic heartbeats (heartbeat.interval.ms) to indicate its liveness to the broker. If no hearts are received by the broker for a group member within the session timeout, the broker will remove the consumer from the group and trigger a rebalance. The allowed range is configured with the **broker** configuration properties `group.min.session.timeout.ms` and `group.max.session.timeout.ms`. `session.timeout.ms` is not supported for `group.protocol=consumer`. It is set with the broker configuration property `group.consumer.session.timeout.ms` by default or can be configured through the AdminClient IncrementalAlterConfigs API. The allowed range is configured with the broker configuration properties `group.consumer.min.session.timeout.ms` and `group.consumer.max.session.timeout.ms`. Also see `max.poll.interval.ms`.
*Type: integer*
+heartbeat.interval.ms | C | 1 .. 3600000 | 3000 | low | Group session keepalive heartbeat interval. `heartbeat.interval.ms` is not supported for `group.protocol=consumer`. It is set with the broker configuration property `group.consumer.heartbeat.interval.ms` by default or can be configured through the AdminClient IncrementalAlterConfigs API. The allowed range is configured with the broker configuration properties `group.consumer.min.heartbeat.interval.ms` and `group.consumer.max.heartbeat.interval.ms`.
*Type: integer*
+group.protocol.type | C | | consumer | low | Group protocol type for the `classic` group protocol. NOTE: Currently, the only supported group protocol type is `consumer`. `group.protocol.type` is not supported for `group.protocol=consumer`
*Type: string*
group.protocol | C | classic, consumer | classic | high | Group protocol to use. Use `classic` for the original protocol and `consumer` for the new protocol introduced in KIP-848. Available protocols: classic or consumer. Default is `classic`, but will change to `consumer` in next releases.
*Type: enum value*
group.remote.assignor | C | | | medium | Server side assignor to use. Keep it null to make server select a suitable assignor for the group. Available assignors: uniform or range. Default is null
*Type: string*
coordinator.query.interval.ms | C | 1 .. 3600000 | 600000 | low | How often to query for the current client group coordinator. If the currently assigned coordinator is down the configured query interval will be divided by ten to more quickly recover in case of coordinator reassignment.
*Type: integer*
diff --git a/INTRODUCTION.md b/INTRODUCTION.md
index cbe9516071..aebfbbabbc 100644
--- a/INTRODUCTION.md
+++ b/INTRODUCTION.md
@@ -1,3 +1,4 @@
+
# Introduction to librdkafka - the Apache Kafka C/C++ client library
@@ -9,100 +10,123 @@ librdkafka also provides a native C++ interface.
**Table of Contents**
- [Introduction to librdkafka - the Apache Kafka C/C++ client library](#introduction-to-librdkafka---the-apache-kafka-cc-client-library)
- - [Performance](#performance)
- - [High throughput](#high-throughput)
- - [Low latency](#low-latency)
- - [Latency measurement](#latency-measurement)
- - [Compression](#compression)
- - [Message reliability](#message-reliability)
- - [Producer message delivery success](#producer-message-delivery-success)
- - [Producer message delivery failure](#producer-message-delivery-failure)
- - [Error: Timed out in transmission queue](#error-timed-out-in-transmission-queue)
- - [Error: Timed out in flight to/from broker](#error-timed-out-in-flight-tofrom-broker)
- - [Error: Temporary broker-side error](#error-temporary-broker-side-error)
- - [Error: Temporary errors due to stale metadata](#error-temporary-errors-due-to-stale-metadata)
- - [Error: Local time out](#error-local-time-out)
- - [Error: Permanent errors](#error-permanent-errors)
- - [Producer retries](#producer-retries)
- - [Reordering](#reordering)
- - [Idempotent Producer](#idempotent-producer)
- - [Guarantees](#guarantees)
- - [Ordering and message sequence numbers](#ordering-and-message-sequence-numbers)
- - [Partitioner considerations](#partitioner-considerations)
- - [Message timeout considerations](#message-timeout-considerations)
- - [Leader change](#leader-change)
- - [Error handling](#error-handling)
- - [RD_KAFKA_RESP_ERR_OUT_OF_ORDER_SEQUENCE_NUMBER](#rdkafkaresperroutofordersequencenumber)
- - [RD_KAFKA_RESP_ERR_DUPLICATE_SEQUENCE_NUMBER](#rdkafkaresperrduplicatesequencenumber)
- - [RD_KAFKA_RESP_ERR_UNKNOWN_PRODUCER_ID](#rdkafkaresperrunknownproducerid)
- - [Standard errors](#standard-errors)
- - [Message persistence status](#message-persistence-status)
- - [Transactional Producer](#transactional-producer)
- - [Error handling](#error-handling-1)
- - [Old producer fencing](#old-producer-fencing)
- - [Configuration considerations](#configuration-considerations)
- - [Exactly Once Semantics (EOS) and transactions](#exactly-once-semantics-eos-and-transactions)
- - [Usage](#usage)
- - [Documentation](#documentation)
- - [Initialization](#initialization)
- - [Configuration](#configuration)
- - [Example](#example)
- - [Termination](#termination)
- - [High-level KafkaConsumer](#high-level-kafkaconsumer)
- - [Producer](#producer)
- - [Admin API client](#admin-api-client)
- - [Speeding up termination](#speeding-up-termination)
- - [Threads and callbacks](#threads-and-callbacks)
- - [Brokers](#brokers)
- - [SSL](#ssl)
- - [OAUTHBEARER with support for OIDC](#oauthbearer-with-support-for-oidc)
- - [Sparse connections](#sparse-connections)
- - [Random broker selection](#random-broker-selection)
- - [Persistent broker connections](#persistent-broker-connections)
- - [Connection close](#connection-close)
- - [Fetch From Follower](#fetch-from-follower)
- - [Logging](#logging)
- - [Debug contexts](#debug-contexts)
- - [Feature discovery](#feature-discovery)
- - [Producer API](#producer-api)
- - [Simple Consumer API (legacy)](#simple-consumer-api-legacy)
- - [Offset management](#offset-management)
- - [Auto offset commit](#auto-offset-commit)
- - [At-least-once processing](#at-least-once-processing)
- - [Auto offset reset](#auto-offset-reset)
- - [Consumer groups](#consumer-groups)
- - [Static consumer groups](#static-consumer-groups)
- - [Next generation of the consumer group protocol](#next-generation-of-the-consumer-group-protocol-kip-848)
- - [Topics](#topics)
- - [Unknown or unauthorized topics](#unknown-or-unauthorized-topics)
- - [Topic metadata propagation for newly created topics](#topic-metadata-propagation-for-newly-created-topics)
- - [Topic auto creation](#topic-auto-creation)
- - [Metadata](#metadata)
- - [< 0.9.3](#-093)
- - [> 0.9.3](#-093)
- - [Query reasons](#query-reasons)
- - [Caching strategy](#caching-strategy)
- - [Fatal errors](#fatal-errors)
- - [Fatal producer errors](#fatal-producer-errors)
- - [Fatal consumer errors](#fatal-consumer-errors)
- - [Compatibility](#compatibility)
- - [Broker version compatibility](#broker-version-compatibility)
- - [Broker version >= 0.10.0.0 (or trunk)](#broker-version--01000-or-trunk)
- - [Broker versions 0.9.0.x](#broker-versions-090x)
- - [Broker versions 0.8.x.y](#broker-versions-08xy)
- - [Detailed description](#detailed-description)
- - [Supported KIPs](#supported-kips)
- - [Supported protocol versions](#supported-protocol-versions)
+ - [Performance](#performance)
+ - [High throughput](#high-throughput)
+ - [Low latency](#low-latency)
+ - [Latency measurement](#latency-measurement)
+ - [Compression](#compression)
+ - [Message reliability](#message-reliability)
+ - [Producer message delivery success](#producer-message-delivery-success)
+ - [Producer message delivery failure](#producer-message-delivery-failure)
+ - [Error: Timed out in transmission queue](#error-timed-out-in-transmission-queue)
+ - [Error: Timed out in flight to/from broker](#error-timed-out-in-flight-tofrom-broker)
+ - [Error: Temporary broker-side error](#error-temporary-broker-side-error)
+ - [Error: Temporary errors due to stale metadata](#error-temporary-errors-due-to-stale-metadata)
+ - [Error: Local time out](#error-local-time-out)
+ - [Error: Permanent errors](#error-permanent-errors)
+ - [Producer retries](#producer-retries)
+ - [Reordering](#reordering)
+ - [Idempotent Producer](#idempotent-producer)
+ - [Guarantees](#guarantees)
+ - [Ordering and message sequence numbers](#ordering-and-message-sequence-numbers)
+ - [Partitioner considerations](#partitioner-considerations)
+ - [Message timeout considerations](#message-timeout-considerations)
+ - [Leader change](#leader-change)
+ - [Error handling](#error-handling)
+ - RD_KAFKA_RESP_ERR_OUT_OF_ORDER_SEQUENCE_NUMBER
+ - RD_KAFKA_RESP_ERR_DUPLICATE_SEQUENCE_NUMBER
+ - RD_KAFKA_RESP_ERR_UNKNOWN_PRODUCER_ID
+ - [Standard errors](#standard-errors)
+ - [Message persistence status](#message-persistence-status)
+ - [Transactional Producer](#transactional-producer)
+ - [Error handling](#error-handling-1)
+ - [Old producer fencing](#old-producer-fencing)
+ - [Configuration considerations](#configuration-considerations)
+ - [Exactly Once Semantics (EOS) and transactions](#exactly-once-semantics-eos-and-transactions)
+ - [Usage](#usage)
+ - [Documentation](#documentation)
+ - [Initialization](#initialization)
+ - [Configuration](#configuration)
+ - [Example](#example)
+ - [Termination](#termination)
+ - [High-level KafkaConsumer](#high-level-kafkaconsumer)
+ - [Producer](#producer)
+ - [Admin API client](#admin-api-client)
+ - [Speeding up termination](#speeding-up-termination)
+ - [Threads and callbacks](#threads-and-callbacks)
+ - [Brokers](#brokers)
+ - [SSL](#ssl)
+ - [OAUTHBEARER with support for OIDC](#oauthbearer-with-support-for-oidc)
+ - [JWT bearer grant type (KIP-1139)](#jwt-bearer-grant-type-kip-1139)
+ - [Metadata based authentication](#metadata-based-authentication)
+ - [Azure IMDS](#azure-imds)
+ - [Sparse connections](#sparse-connections)
+ - [Random broker selection](#random-broker-selection)
+ - [Persistent broker connections](#persistent-broker-connections)
+ - [Connection close](#connection-close)
+ - [Fetch From Follower](#fetch-from-follower)
+ - [Logging](#logging)
+ - [Debug contexts](#debug-contexts)
+ - [Feature discovery](#feature-discovery)
+ - [Producer API](#producer-api)
+ - [Simple Consumer API (legacy)](#simple-consumer-api-legacy)
+ - [Offset management](#offset-management)
+ - [Auto offset commit](#auto-offset-commit)
+ - [At-least-once processing](#at-least-once-processing)
+ - [Auto offset reset](#auto-offset-reset)
+ - [Consumer groups](#consumer-groups)
+ - [Static consumer groups](#static-consumer-groups)
+ - [Next Generation Consumer Group Protocol (KIP-848)](#next-generation-consumer-group-protocol-kip-848)
+ - [Overview](#overview)
+ - [Available Features](#available-features)
+ - [Contract Changes](#contract-changes)
+ - [Client Configuration changes](#client-configuration-changes)
+ - [Rebalance Callback Changes](#rebalance-callback-changes)
+ - [Static Group Membership](#static-group-membership)
+ - [Session Timeout \& Fetching](#session-timeout--fetching)
+ - [Closing / Auto-Commit](#closing--auto-commit)
+ - [Error Handling Changes](#error-handling-changes)
+ - [Summary of Key Differences (Classic vs Next-Gen)](#summary-of-key-differences-classic-vs-next-gen)
+ - [Minimal Example Config](#minimal-example-config)
+ - [Classic Protocol](#classic-protocol)
+ - [Next-Gen Protocol / KIP-848](#next-gen-protocol--kip-848)
+ - [Rebalance Callback Migration](#rebalance-callback-migration)
+ - [Range Assignor (Classic)](#range-assignor-classic)
+ - [Incremental Assignor (Including Range in Consumer / KIP-848, Any Protocol)](#incremental-assignor-including-range-in-consumer--kip-848-any-protocol)
+ - [Upgrade and Downgrade](#upgrade-and-downgrade)
+ - [Migration Checklist (Next-Gen Protocol / KIP-848)](#migration-checklist-next-gen-protocol--kip-848)
+ - [Note on Batch consume APIs](#note-on-batch-consume-apis)
+ - [Topics](#topics)
+ - [Unknown or unauthorized topics](#unknown-or-unauthorized-topics)
+ - [Topic metadata propagation for newly created topics](#topic-metadata-propagation-for-newly-created-topics)
+ - [Topic auto creation](#topic-auto-creation)
+ - [Metadata](#metadata)
+ - [\< 0.9.3](#lt093)
+ - [\> 0.9.3](#gt093-1)
+ - [Query reasons](#query-reasons)
+ - [Caching strategy](#caching-strategy)
+ - [Fatal errors](#fatal-errors)
+ - [Fatal producer errors](#fatal-producer-errors)
+ - [Fatal consumer errors](#fatal-consumer-errors)
+ - [Compatibility](#compatibility)
+ - [Broker version compatibility](#broker-version-compatibility)
+ - [Broker version \>= 0.10.0.0 (or trunk)](#broker-version--01000-or-trunk)
+ - [Broker versions 0.9.0.x](#broker-versions-090x)
+ - [Broker versions 0.8.x.y](#broker-versions-08xy)
+ - [Detailed description](#detailed-description)
+ - [Supported KIPs](#supported-kips)
+ - [Supported protocol versions](#supported-protocol-versions)
- [Recommendations for language binding developers](#recommendations-for-language-binding-developers)
- - [Expose the configuration interface pass-thru](#expose-the-configuration-interface-pass-thru)
- - [Error constants](#error-constants)
- - [Reporting client software name and version to broker](#reporting-client-software-name-and-version-to-broker)
- - [Documentation reuse](#documentation-reuse)
- - [Community support](#community-support)
+ - [Expose the configuration interface pass-thru](#expose-the-configuration-interface-pass-thru)
+ - [Error constants](#error-constants)
+ - [Reporting client software name and version to broker](#reporting-client-software-name-and-version-to-broker)
+ - [Documentation reuse](#documentation-reuse)
+ - [Community support](#community-support)
+
## Performance
librdkafka is a multi-threaded library designed for use on modern hardware and
@@ -128,6 +152,7 @@ per-message load on the broker. A good general purpose setting is 5ms.
For applications seeking maximum throughput, the recommended value is >= 50ms.
+
### High throughput
The key to high throughput is message batching - waiting for a certain amount
@@ -176,6 +201,7 @@ These setting are set globally (`rd_kafka_conf_t`) but applies on a
per topic+partition basis.
+
### Low latency
When low latency messaging is required the `linger.ms` should be
@@ -188,6 +214,7 @@ increasing network, memory and CPU usage for producers, brokers and consumers.
See [How to decrease message latency](https://github.com/confluentinc/librdkafka/wiki/How-to-decrease-message-latency) for more info.
+
#### Latency measurement
End-to-end latency is preferably measured by synchronizing clocks on producers
@@ -202,42 +229,42 @@ To break down the end-to-end latencies and find where latencies are adding up
there are a number of metrics available through librdkafka statistics
on the producer:
- * `brokers[].int_latency` is the time, per message, between produce()
- and the message being written to a MessageSet and ProduceRequest.
- High `int_latency` indicates CPU core contention: check CPU load and,
- involuntary context switches (`/proc/<..>/status`).
- Consider using a machine/instance with more CPU cores.
- This metric is only relevant on the producer.
-
- * `brokers[].outbuf_latency` is the time, per protocol request
- (such as ProduceRequest), between the request being enqueued (which happens
- right after int_latency) and the time the request is written to the
- TCP socket connected to the broker.
- High `outbuf_latency` indicates CPU core contention or network congestion:
- check CPU load and socket SendQ (`netstat -anp | grep :9092`).
-
- * `brokers[].rtt` is the time, per protocol request, between the request being
- written to the TCP socket and the time the response is received from
- the broker.
- High `rtt` indicates broker load or network congestion:
- check broker metrics, local socket SendQ, network performance, etc.
-
- * `brokers[].throttle` is the time, per throttled protocol request, the
- broker throttled/delayed handling of a request due to usage quotas.
- The throttle time will also be reflected in `rtt`.
-
- * `topics[].batchsize` is the size of individual Producer MessageSet batches.
- See below.
-
- * `topics[].batchcnt` is the number of messages in individual Producer
- MessageSet batches. Due to Kafka protocol overhead a batch with few messages
- will have a higher relative processing and size overhead than a batch
- with many messages.
- Use the `linger.ms` client configuration property to set the maximum
- amount of time allowed for accumulating a single batch, the larger the
- value the larger the batches will grow, thus increasing efficiency.
- When producing messages at a high rate it is recommended to increase
- linger.ms, which will improve throughput and in some cases also latency.
+* `brokers[].int_latency` is the time, per message, between produce()
+ and the message being written to a MessageSet and ProduceRequest.
+ High `int_latency` indicates CPU core contention: check CPU load and,
+ involuntary context switches (`/proc/<..>/status`).
+ Consider using a machine/instance with more CPU cores.
+ This metric is only relevant on the producer.
+
+* `brokers[].outbuf_latency` is the time, per protocol request
+ (such as ProduceRequest), between the request being enqueued (which happens
+ right after int_latency) and the time the request is written to the
+ TCP socket connected to the broker.
+ High `outbuf_latency` indicates CPU core contention or network congestion:
+ check CPU load and socket SendQ (`netstat -anp | grep :9092`).
+
+* `brokers[].rtt` is the time, per protocol request, between the request being
+ written to the TCP socket and the time the response is received from
+ the broker.
+ High `rtt` indicates broker load or network congestion:
+ check broker metrics, local socket SendQ, network performance, etc.
+
+* `brokers[].throttle` is the time, per throttled protocol request, the
+ broker throttled/delayed handling of a request due to usage quotas.
+ The throttle time will also be reflected in `rtt`.
+
+* `topics[].batchsize` is the size of individual Producer MessageSet batches.
+ See below.
+
+* `topics[].batchcnt` is the number of messages in individual Producer
+ MessageSet batches. Due to Kafka protocol overhead a batch with few messages
+ will have a higher relative processing and size overhead than a batch
+ with many messages.
+ Use the `linger.ms` client configuration property to set the maximum
+ amount of time allowed for accumulating a single batch, the larger the
+ value the larger the batches will grow, thus increasing efficiency.
+ When producing messages at a high rate it is recommended to increase
+ linger.ms, which will improve throughput and in some cases also latency.
See [STATISTICS.md](STATISTICS.md) for the full definition of metrics.
@@ -245,6 +272,7 @@ A JSON schema for the statistics is available in
[statistics-schema.json](src/statistics-schema.json).
+
### Compression
Producer message compression is enabled through the `compression.codec`
@@ -258,6 +286,7 @@ The local batch queue size is controlled through the `batch.num.messages`,
+
## Message reliability
Message reliability is an important factor of librdkafka - an application
@@ -270,10 +299,10 @@ for message commit acknowledgements from brokers (any value but 0, see
for specifics) then librdkafka will hold on to the message until
all expected acks have been received, gracefully handling the following events:
- * Broker connection failure
- * Topic leader change
- * Produce errors signaled by the broker
- * Network problems
+* Broker connection failure
+* Topic leader change
+* Produce errors signaled by the broker
+* Network problems
We recommend `request.required.acks` to be set to `all` to make sure
produced messages are acknowledged by all in-sync replica brokers.
@@ -287,15 +316,16 @@ The delivery report callback is used by librdkafka to signal the status of
a message back to the application, it will be called once for each message
to report the status of message delivery:
- * If `error_code` is non-zero the message delivery failed and the error_code
- indicates the nature of the failure (`rd_kafka_resp_err_t` enum).
- * If `error_code` is zero the message has been successfully delivered.
+* If `error_code` is non-zero the message delivery failed and the error_code
+ indicates the nature of the failure (`rd_kafka_resp_err_t` enum).
+* If `error_code` is zero the message has been successfully delivered.
See Producer API chapter for more details on delivery report callback usage.
The delivery report callback is optional but highly recommended.
+
### Producer message delivery success
When a ProduceRequest is successfully handled by the broker and a
@@ -305,6 +335,7 @@ queue (if a delivery report callback has been set) and will be passed to
the application on the next invocation rd_kafka_poll().
+
### Producer message delivery failure
The following sub-chapters explains how different produce errors
@@ -324,6 +355,7 @@ using the `retries`, `retry.backoff.ms` and `retry.backoff.max.ms`
configuration properties.
+
#### Error: Timed out in transmission queue
Internal error ERR__TIMED_OUT_QUEUE.
@@ -339,6 +371,7 @@ since the message was never actually transmitted.
A retry by librdkafka at this point will not cause duplicate messages.
+
#### Error: Timed out in flight to/from broker
Internal error ERR__TIMED_OUT, ERR__TRANSPORT.
@@ -355,6 +388,7 @@ This is a retryable error.
A retry by librdkafka at this point may cause duplicate messages.
+
#### Error: Temporary broker-side error
Broker errors ERR_REQUEST_TIMED_OUT, ERR_NOT_ENOUGH_REPLICAS,
@@ -364,6 +398,7 @@ These errors are considered temporary and librdkafka is will retry them
if permitted by configuration.
+
#### Error: Temporary errors due to stale metadata
Broker errors ERR_LEADER_NOT_AVAILABLE, ERR_NOT_LEADER_FOR_PARTITION.
@@ -374,6 +409,7 @@ request is automatically sent to find a new leader for the partition.
A retry by librdkafka at this point will not cause duplicate messages.
+
#### Error: Local time out
Internal error ERR__MSG_TIMED_OUT.
@@ -387,6 +423,7 @@ Since the `message.timeout.ms` has passed there will be no more retries
by librdkafka.
+
#### Error: Permanent errors
Any other error is considered a permanent error and the message
@@ -397,19 +434,20 @@ The full list of permanent errors depend on the broker version and
will likely grow in the future.
Typical permanent broker errors are:
- * ERR_CORRUPT_MESSAGE
- * ERR_MSG_SIZE_TOO_LARGE - adjust client's or broker's `message.max.bytes`.
- * ERR_UNKNOWN_TOPIC_OR_PART - topic or partition does not exist,
- automatic topic creation is disabled on the
- broker or the application is specifying a
- partition that does not exist.
- * ERR_RECORD_LIST_TOO_LARGE
- * ERR_INVALID_REQUIRED_ACKS
- * ERR_TOPIC_AUTHORIZATION_FAILED
- * ERR_UNSUPPORTED_FOR_MESSAGE_FORMAT
- * ERR_CLUSTER_AUTHORIZATION_FAILED
-
-
+* ERR_CORRUPT_MESSAGE
+* ERR_MSG_SIZE_TOO_LARGE - adjust client's or broker's `message.max.bytes`.
+* ERR_UNKNOWN_TOPIC_OR_PART - topic or partition does not exist,
+ automatic topic creation is disabled on the
+ broker or the application is specifying a
+ partition that does not exist.
+* ERR_RECORD_LIST_TOO_LARGE
+* ERR_INVALID_REQUIRED_ACKS
+* ERR_TOPIC_AUTHORIZATION_FAILED
+* ERR_UNSUPPORTED_FOR_MESSAGE_FORMAT
+* ERR_CLUSTER_AUTHORIZATION_FAILED
+
+
+
### Producer retries
The ProduceRequest itself is not retried, instead the messages
@@ -421,6 +459,7 @@ A backoff time (`retry.backoff.ms`) is set on the retried messages which
effectively blocks retry attempts until the backoff time has expired.
+
### Reordering
As for all retries, if `max.in.flight` > 1 and `retries` > 0, retried messages
@@ -432,6 +471,7 @@ Using the Idempotent Producer prevents reordering even with `max.in.flight` > 1,
see [Idempotent Producer](#idempotent-producer) below for more information.
+
### Idempotent Producer
librdkafka supports the idempotent producer which provides strict ordering and
@@ -446,33 +486,35 @@ for any of the automatically adjusted properties, e.g., it is an error to
explicitly set `acks=1` when `enable.idempotence=true` is set.
+
#### Guarantees
There are three types of guarantees that the idempotent producer can satisfy:
- * Exactly-once - a message is only written to the log once.
- Does NOT cover the exactly-once consumer case.
- * Ordering - a series of messages are written to the log in the
- order they were produced.
- * Gap-less - **EXPERIMENTAL** a series of messages are written once and
- in order without risk of skipping messages. The sequence
- of messages may be cut short and fail before all
- messages are written, but may not fail individual
- messages in the series.
- This guarantee is disabled by default, but may be enabled
- by setting `enable.gapless.guarantee` if individual message
- failure is a concern.
- Messages that fail due to exceeded timeout (`message.timeout.ms`),
- are permitted by the gap-less guarantee and may cause
- gaps in the message series without raising a fatal error.
- See **Message timeout considerations** below for more info.
- **WARNING**: This is an experimental property subject to
- change or removal.
+* Exactly-once - a message is only written to the log once.
+ Does NOT cover the exactly-once consumer case.
+* Ordering - a series of messages are written to the log in the
+ order they were produced.
+* Gap-less - **EXPERIMENTAL** a series of messages are written once and
+ in order without risk of skipping messages. The sequence
+ of messages may be cut short and fail before all
+ messages are written, but may not fail individual
+ messages in the series.
+ This guarantee is disabled by default, but may be enabled
+ by setting `enable.gapless.guarantee` if individual message
+ failure is a concern.
+ Messages that fail due to exceeded timeout (`message.timeout.ms`),
+ are permitted by the gap-less guarantee and may cause
+ gaps in the message series without raising a fatal error.
+ See **Message timeout considerations** below for more info.
+ **WARNING**: This is an experimental property subject to
+ change or removal.
All three guarantees are in effect when idempotence is enabled, only
gap-less may be disabled individually.
+
#### Ordering and message sequence numbers
librdkafka maintains the original produce() ordering per-partition for all
@@ -493,28 +535,29 @@ With Idempotent Producer enabled there is no risk of reordering despite
`max.in.flight` > 1 (capped at 5).
**Note**: "MsgId" in log messages refer to the librdkafka msgid, while "seq"
- refers to the protocol message sequence, "baseseq" is the seq of
- the first message in a batch.
- MsgId starts at 1, while message seqs start at 0.
+refers to the protocol message sequence, "baseseq" is the seq of
+the first message in a batch.
+MsgId starts at 1, while message seqs start at 0.
The producer statistics also maintain two metrics for tracking the next
expected response sequence:
- * `next_ack_seq` - the next sequence to expect an acknowledgement for, which
- is the last successfully produced MessageSet's last
- sequence + 1.
- * `next_err_seq` - the next sequence to expect an error for, which is typically
- the same as `next_ack_seq` until an error occurs, in which
- case the `next_ack_seq` can't be incremented (since no
- messages were acked on error). `next_err_seq` is used to
- properly handle sub-sequent errors due to a failing
- first request.
+* `next_ack_seq` - the next sequence to expect an acknowledgement for, which
+ is the last successfully produced MessageSet's last
+ sequence + 1.
+* `next_err_seq` - the next sequence to expect an error for, which is typically
+ the same as `next_ack_seq` until an error occurs, in which
+ case the `next_ack_seq` can't be incremented (since no
+ messages were acked on error). `next_err_seq` is used to
+ properly handle sub-sequent errors due to a failing
+ first request.
**Note**: Both are exposed in partition statistics.
+
#### Partitioner considerations
Strict ordering is guaranteed on a **per partition** basis.
@@ -526,17 +569,18 @@ a topic's partition count is known, which would insert these messages
after the partition-explicit messages regardless of produce order.
+
#### Message timeout considerations
If messages time out (due to `message.timeout.ms`) while in the producer queue
there will be gaps in the series of produced messages.
E.g., Messages 1,2,3,4,5 are produced by the application.
- While messages 2,3,4 are transmitted to the broker the connection to
- the broker goes down.
- While the broker is down the message timeout expires for message 2 and 3.
- As the connection comes back up messages 4, 5 are transmitted to the
- broker, resulting in a final written message sequence of 1, 4, 5.
+While messages 2,3,4 are transmitted to the broker the connection to
+the broker goes down.
+While the broker is down the message timeout expires for message 2 and 3.
+As the connection comes back up messages 4, 5 are transmitted to the
+broker, resulting in a final written message sequence of 1, 4, 5.
The producer gracefully handles this case by draining the in-flight requests
for a given partition when one or more of its queued (not transmitted)
@@ -573,6 +617,7 @@ large `message.timeout.ms` to minimize the risk of timeouts.
**Note**: `delivery.timeout.ms` is an alias for `message.timeout.ms`.
+
#### Leader change
There are corner cases where an Idempotent Producer has outstanding
@@ -590,6 +635,7 @@ If the connection to the previous broker goes down the outstanding requests
are failed immediately.
+
#### Error handling
Background:
@@ -605,7 +651,7 @@ provide stricter and less complex error handling.
The follow sections describe librdkafka's handling of the
Idempotent Producer specific errors that may be returned by the broker.
-
+
##### RD_KAFKA_RESP_ERR_OUT_OF_ORDER_SEQUENCE_NUMBER
This error is returned by the broker when the sequence number in the
@@ -633,7 +679,7 @@ Fail the batch, reset the pid, and then continue producing
in the message series.
-
+
##### RD_KAFKA_RESP_ERR_DUPLICATE_SEQUENCE_NUMBER
Returned by broker when the request's base sequence number is
@@ -651,7 +697,7 @@ timestamp or offset set.
**Java Producer behaviour:**
Treats the message as successfully delivered.
-
+
##### RD_KAFKA_RESP_ERR_UNKNOWN_PRODUCER_ID
Returned by broker when the PID+Epoch is unknown, which may occur when
@@ -685,6 +731,7 @@ Retries the send in some cases (but KIP-360 will change this).
Not a fatal error in any case.
+
##### Standard errors
All the standard Produce errors are handled in the usual way,
@@ -702,56 +749,60 @@ the gap-less guarantee (if `enable.gapless.guarantee` is set) by failing all
queued messages.
+
##### Message persistence status
To help the application decide what to do in these error cases, a new
per-message API is introduced, `rd_kafka_message_status()`,
which returns one of the following values:
- * `RD_KAFKA_MSG_STATUS_NOT_PERSISTED` - the message has never
- been transmitted to the broker, or failed with an error indicating
- it was not written to the log.
- Application retry will risk ordering, but not duplication.
- * `RD_KAFKA_MSG_STATUS_POSSIBLY_PERSISTED` - the message was transmitted
- to the broker, but no acknowledgement was received.
- Application retry will risk ordering and duplication.
- * `RD_KAFKA_MSG_STATUS_PERSISTED` - the message was written to the log by
- the broker and fully acknowledged.
- No reason for application to retry.
+* `RD_KAFKA_MSG_STATUS_NOT_PERSISTED` - the message has never
+ been transmitted to the broker, or failed with an error indicating
+ it was not written to the log.
+ Application retry will risk ordering, but not duplication.
+* `RD_KAFKA_MSG_STATUS_POSSIBLY_PERSISTED` - the message was transmitted
+ to the broker, but no acknowledgement was received.
+ Application retry will risk ordering and duplication.
+* `RD_KAFKA_MSG_STATUS_PERSISTED` - the message was written to the log by
+ the broker and fully acknowledged.
+ No reason for application to retry.
This method should be called by the application on delivery report error.
+
### Transactional Producer
+
#### Error handling
Using the transactional producer simplifies error handling compared to the
standard or idempotent producer, a transactional application will only need
to care about these different types of errors:
- * Retriable errors - the operation failed due to temporary problems,
- such as network timeouts, the operation may be safely retried.
- Use `rd_kafka_error_is_retriable()` to distinguish this case.
- * Abortable errors - if any of the transactional APIs return a non-fatal
- error code the current transaction has failed and the application
- must call `rd_kafka_abort_transaction()`, rewind its input to the
- point before the current transaction started, and attempt a new transaction
- by calling `rd_kafka_begin_transaction()`, etc.
- Use `rd_kafka_error_txn_requires_abort()` to distinguish this case.
- * Fatal errors - the application must cease operations and destroy the
- producer instance.
- Use `rd_kafka_error_is_fatal()` to distinguish this case.
- * For all other errors returned from the transactional API: the current
- recommendation is to treat any error that has neither retriable, abortable,
- or fatal set, as a fatal error.
+* Retriable errors - the operation failed due to temporary problems,
+ such as network timeouts, the operation may be safely retried.
+ Use `rd_kafka_error_is_retriable()` to distinguish this case.
+* Abortable errors - if any of the transactional APIs return a non-fatal
+ error code the current transaction has failed and the application
+ must call `rd_kafka_abort_transaction()`, rewind its input to the
+ point before the current transaction started, and attempt a new transaction
+ by calling `rd_kafka_begin_transaction()`, etc.
+ Use `rd_kafka_error_txn_requires_abort()` to distinguish this case.
+* Fatal errors - the application must cease operations and destroy the
+ producer instance.
+ Use `rd_kafka_error_is_fatal()` to distinguish this case.
+* For all other errors returned from the transactional API: the current
+ recommendation is to treat any error that has neither retriable, abortable,
+ or fatal set, as a fatal error.
While the application should log the actual fatal or abortable errors, there
is no need for the application to handle the underlying errors specifically.
+
#### Old producer fencing
If a new transactional producer instance is started with the same
@@ -761,6 +812,7 @@ raising a fatal error with the error code set to
`RD_KAFKA_RESP_ERR__FENCED`.
+
#### Configuration considerations
To make sure messages time out (in case of connectivity problems, etc) within
@@ -773,6 +825,7 @@ automatically.
+
### Exactly Once Semantics (EOS) and transactions
librdkafka supports Exactly One Semantics (EOS) as defined in [KIP-98](https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging).
@@ -789,14 +842,17 @@ the number of input partitions.
See KIP-447 for more information.
+
## Usage
+
### Documentation
The librdkafka API is documented in the [`rdkafka.h`](src/rdkafka.h)
header file, the configuration properties are documented in
[`CONFIGURATION.md`](CONFIGURATION.md)
+
### Initialization
The application needs to instantiate a top-level object `rd_kafka_t` which is
@@ -815,13 +871,14 @@ Not using the API will cause librdkafka to use its default values which are
documented in [`CONFIGURATION.md`](CONFIGURATION.md).
**Note**: An application may create multiple `rd_kafka_t` objects and
- they share no state.
+they share no state.
**Note**: An `rd_kafka_topic_t` object may only be used with the `rd_kafka_t`
- object it was created from.
+object it was created from.
+
### Configuration
To ease integration with the official Apache Kafka software and lower
@@ -832,10 +889,11 @@ Configuration is applied prior to object creation using the
`rd_kafka_conf_set()` and `rd_kafka_topic_conf_set()` APIs.
**Note**: The `rd_kafka.._conf_t` objects are not reusable after they have been
- passed to `rd_kafka.._new()`.
- The application does not need to free any config resources after a
- `rd_kafka.._new()` call.
+passed to `rd_kafka.._new()`.
+The application does not need to free any config resources after a
+`rd_kafka.._new()` call.
+
#### Example
```c
@@ -873,6 +931,7 @@ are detected. It will also emit log warnings for deprecated and problematic
configuration properties.
+
### Termination
librdkafka is asynchronous in its nature and performs most operation in its
@@ -894,23 +953,24 @@ destroyed/deleted prior to destroying or closing the handle.
For C, make sure the following objects are destroyed prior to calling
`rd_kafka_consumer_close()` and `rd_kafka_destroy()`:
- * `rd_kafka_message_t`
- * `rd_kafka_topic_t`
- * `rd_kafka_topic_partition_t`
- * `rd_kafka_topic_partition_list_t`
- * `rd_kafka_event_t`
- * `rd_kafka_queue_t`
+* `rd_kafka_message_t`
+* `rd_kafka_topic_t`
+* `rd_kafka_topic_partition_t`
+* `rd_kafka_topic_partition_list_t`
+* `rd_kafka_event_t`
+* `rd_kafka_queue_t`
For C++ make sure the following objects are deleted prior to
calling `KafkaConsumer::close()` and delete on the Consumer, KafkaConsumer or
Producer handle:
- * `Message`
- * `Topic`
- * `TopicPartition`
- * `Event`
- * `Queue`
+* `Message`
+* `Topic`
+* `TopicPartition`
+* `Event`
+* `Queue`
+
#### High-level KafkaConsumer
Proper termination sequence for the high-level KafkaConsumer is:
@@ -927,13 +987,14 @@ Proper termination sequence for the high-level KafkaConsumer is:
**NOTE**: Any topic objects created must be destroyed prior to rd_kafka_destroy()
Effects of not doing the above, for:
- 1. Final offsets are not committed and the consumer will not actively leave
- the group, it will be kicked out of the group after the `session.timeout.ms`
- expires. It is okay to omit the `rd_kafka_consumer_close()` call in case
- the application does not want to wait for the blocking close call.
- 2. librdkafka will continue to operate on the handle. Actual memory leaks.
+1. Final offsets are not committed and the consumer will not actively leave
+ the group, it will be kicked out of the group after the `session.timeout.ms`
+ expires. It is okay to omit the `rd_kafka_consumer_close()` call in case
+ the application does not want to wait for the blocking close call.
+2. librdkafka will continue to operate on the handle. Actual memory leaks.
+
#### Producer
The proper termination sequence for Producers is:
@@ -948,10 +1009,11 @@ The proper termination sequence for Producers is:
```
Effects of not doing the above, for:
- 1. Messages in-queue or in-flight will be dropped.
- 2. librdkafka will continue to operate on the handle. Actual memory leaks.
+1. Messages in-queue or in-flight will be dropped.
+2. librdkafka will continue to operate on the handle. Actual memory leaks.
+
#### Admin API client
Unlike the Java Admin client, the Admin APIs in librdkafka are available
@@ -969,6 +1031,7 @@ topic metadata lookups to unexpectedly have the broker create topics.
+
#### Speeding up termination
To speed up the termination of librdkafka an application can set a
termination signal that will be used internally by librdkafka to quickly
@@ -982,6 +1045,7 @@ Make sure you block this signal in your application.
```
+
### Threads and callbacks
librdkafka uses multiple threads internally to fully utilize modern hardware.
@@ -992,32 +1056,32 @@ A poll-based API is used to provide signaling back to the application,
the application should call rd_kafka_poll() at regular intervals.
The poll API will call the following configured callbacks (optional):
- * `dr_msg_cb` - Message delivery report callback - signals that a message has
- been delivered or failed delivery, allowing the application to take action
- and to release any application resources used in the message.
- * `error_cb` - Error callback - signals an error. These errors are usually of
- an informational nature, i.e., failure to connect to a broker, and the
- application usually does not need to take any action.
- The type of error is passed as a rd_kafka_resp_err_t enum value,
- including both remote broker errors as well as local failures.
- An application typically does not have to perform any action when
- an error is raised through the error callback, the client will
- automatically try to recover from all errors, given that the
- client and cluster is correctly configured.
- In some specific cases a fatal error may occur which will render
- the client more or less inoperable for further use:
- if the error code in the error callback is set to
- `RD_KAFKA_RESP_ERR__FATAL` the application should retrieve the
- underlying fatal error and reason using the `rd_kafka_fatal_error()` call,
- and then begin terminating the instance.
- The Event API's EVENT_ERROR has a `rd_kafka_event_error_is_fatal()`
- function, and the C++ EventCb has a `fatal()` method, to help the
- application determine if an error is fatal or not.
- * `stats_cb` - Statistics callback - triggered if `statistics.interval.ms`
- is configured to a non-zero value, emitting metrics and internal state
- in JSON format, see [STATISTICS.md].
- * `throttle_cb` - Throttle callback - triggered whenever a broker has
- throttled (delayed) a request.
+* `dr_msg_cb` - Message delivery report callback - signals that a message has
+ been delivered or failed delivery, allowing the application to take action
+ and to release any application resources used in the message.
+* `error_cb` - Error callback - signals an error. These errors are usually of
+ an informational nature, i.e., failure to connect to a broker, and the
+ application usually does not need to take any action.
+ The type of error is passed as a rd_kafka_resp_err_t enum value,
+ including both remote broker errors as well as local failures.
+ An application typically does not have to perform any action when
+ an error is raised through the error callback, the client will
+ automatically try to recover from all errors, given that the
+ client and cluster is correctly configured.
+ In some specific cases a fatal error may occur which will render
+ the client more or less inoperable for further use:
+ if the error code in the error callback is set to
+ `RD_KAFKA_RESP_ERR__FATAL` the application should retrieve the
+ underlying fatal error and reason using the `rd_kafka_fatal_error()` call,
+ and then begin terminating the instance.
+ The Event API's EVENT_ERROR has a `rd_kafka_event_error_is_fatal()`
+ function, and the C++ EventCb has a `fatal()` method, to help the
+ application determine if an error is fatal or not.
+* `stats_cb` - Statistics callback - triggered if `statistics.interval.ms`
+ is configured to a non-zero value, emitting metrics and internal state
+ in JSON format, see [STATISTICS.md].
+* `throttle_cb` - Throttle callback - triggered whenever a broker has
+ throttled (delayed) a request.
These callbacks will also be triggered by `rd_kafka_flush()`,
`rd_kafka_consumer_poll()`, and any other functions that serve queues.
@@ -1026,20 +1090,21 @@ These callbacks will also be triggered by `rd_kafka_flush()`,
Optional callbacks not triggered by poll, these may be called spontaneously
from any thread at any time:
- * `log_cb` - Logging callback - allows the application to output log messages
- generated by librdkafka.
- * `partitioner_cb` - Partitioner callback - application provided message partitioner.
- The partitioner may be called in any thread at any time, it may be
- called multiple times for the same key.
- Partitioner function contraints:
- - MUST NOT call any rd_kafka_*() functions
- - MUST NOT block or execute for prolonged periods of time.
- - MUST return a value between 0 and partition_cnt-1, or the
- special RD_KAFKA_PARTITION_UA value if partitioning
- could not be performed.
+* `log_cb` - Logging callback - allows the application to output log messages
+ generated by librdkafka.
+* `partitioner_cb` - Partitioner callback - application provided message partitioner.
+ The partitioner may be called in any thread at any time, it may be
+ called multiple times for the same key.
+ Partitioner function contraints:
+ - MUST NOT call any rd_kafka_*() functions
+ - MUST NOT block or execute for prolonged periods of time.
+ - MUST return a value between 0 and partition_cnt-1, or the
+ special RD_KAFKA_PARTITION_UA value if partitioning
+ could not be performed.
+
### Brokers
On initialization, librdkafka only needs a partial list of
@@ -1058,6 +1123,7 @@ A DNS record containing all broker address can thus be used to provide a
reliable bootstrap broker.
+
#### SSL
If the client is to connect to a broker's SSL endpoints/listeners the client
@@ -1069,15 +1135,15 @@ the connection is closed (and retried). This is to protect the client
from connecting to rogue brokers.
The CA root certificate defaults are system specific:
- * On Linux, Mac OSX, and other Unix-like system the OpenSSL default
- CA path will be used, also called the OPENSSLDIR, which is typically
- `/etc/ssl/certs` (on Linux, typcially in the `ca-certificates` package) and
- `/usr/local/etc/openssl` on Mac OSX (Homebrew).
- * On Windows the Root certificate store is used, unless
- `ssl.ca.certificate.stores` is configured in which case certificates are
- read from the specified stores.
- * If OpenSSL is linked statically, librdkafka will set the default CA
- location to the first of a series of probed paths (see below).
+* On Linux, Mac OSX, and other Unix-like system the OpenSSL default
+ CA path will be used, also called the OPENSSLDIR, which is typically
+ `/etc/ssl/certs` (on Linux, typcially in the `ca-certificates` package) and
+ `/usr/local/etc/openssl` on Mac OSX (Homebrew).
+* On Windows the Root certificate store is used, unless
+ `ssl.ca.certificate.stores` is configured in which case certificates are
+ read from the specified stores.
+* If OpenSSL is linked statically, librdkafka will set the default CA
+ location to the first of a series of probed paths (see below).
If the system-provided default CA root certificates are not sufficient to
verify the broker's certificate, such as when a self-signed certificate
@@ -1118,15 +1184,16 @@ of certificate stores can be read by setting the `ssl.ca.certificate.stores`
configuration property to a comma-separated list of certificate store names.
The predefined system store names are:
- * `MY` - User certificates
- * `Root` - System CA certificates (default)
- * `CA` - Intermediate CA certificates
- * `Trust` - Trusted publishers
+* `MY` - User certificates
+* `Root` - System CA certificates (default)
+* `CA` - Intermediate CA certificates
+* `Trust` - Trusted publishers
For example, to read both intermediate and root CAs, set
`ssl.ca.certificate.stores=CA,Root`.
+
#### OAUTHBEARER with support for OIDC
OAUTHBEARER with OIDC provides a method for the client to authenticate to the
@@ -1135,24 +1202,98 @@ and passing the retrieved token to brokers during connection setup.
To use this authentication method the client needs to be configured as follows:
- * `security.protocol` - set to `SASL_SSL` or `SASL_PLAINTEXT`.
- * `sasl.mechanism` - set to `OAUTHBEARER`.
- * `sasl.oauthbearer.method` - set to `OIDC`.
- * `sasl.oauthbearer.token.endpoint.url` - OAUTH issuer token
- endpoint HTTP(S) URI used to retrieve the token.
- * `sasl.oauthbearer.client.id` - public identifier for the application.
- It must be unique across all clients that the authorization server handles.
- * `sasl.oauthbearer.client.secret` - secret known only to the
- application and the authorization server.
- This should be a sufficiently random string that is not guessable.
- * `sasl.oauthbearer.scope` - clients use this to specify the scope of the
- access request to the broker.
- * `sasl.oauthbearer.extensions` - (optional) additional information to be
- provided to the broker. A comma-separated list of key=value pairs.
- For example:
- `supportFeatureX=true,organizationId=sales-emea`
-
-
+* `security.protocol` - set to `SASL_SSL` or `SASL_PLAINTEXT`.
+* `sasl.mechanism` - set to `OAUTHBEARER`.
+* `sasl.oauthbearer.method` - set to `OIDC`.
+* `sasl.oauthbearer.token.endpoint.url` - OAUTH issuer token
+ endpoint HTTP(S) URI used to retrieve the token.
+* `sasl.oauthbearer.client.id` - public identifier for the application.
+ It must be unique across all clients that the authorization server handles.
+* `sasl.oauthbearer.client.secret` - secret known only to the
+ application and the authorization server.
+ This should be a sufficiently random string that is not guessable.
+* `sasl.oauthbearer.scope` - clients use this to specify the scope of the
+ access request to the broker.
+* `sasl.oauthbearer.extensions` - (optional) additional information to be
+ provided to the broker. A comma-separated list of key=value pairs.
+ For example:
+ `supportFeatureX=true,organizationId=sales-emea`
+* `https.ca.location` - (optional) to customize the CA certificates
+ location.
+
+* `https.ca.pem` - (optional) to provide the CA certificates as a PEM string.
+
+
+##### JWT bearer grant type (KIP-1139)
+
+This KIP adds support for the `client_credentials, urn:ietf:params:oauth:grant-type:jwt-bearer`
+grant type, with a series of properties to be used for creating a JWT assertion
+sent to the token endpoint. The authenticated principal corresponds to the
+`sub` claim returned by token endpoint, `sasl.oauthbearer.client.id` and
+`sasl.oauthbearer.client.secret` aren't used. Required JWT claims must be set
+either through the template or with the `claim` properties.
+
+* `sasl.oauthbearer.grant.type` - changes the default grant type, set it to
+ `urn:ietf:params:oauth:grant-type:jwt-bearer`.
+* `sasl.oauthbearer.assertion.algorithm` - JWT algorithm defaults to `RS256`.
+* `sasl.oauthbearer.assertion.private.key.file` - a private key file for signing
+ the token.
+* `sasl.oauthbearer.assertion.private.key.passphrase` - (optional) passphrase for the key if encrypted.
+* `sasl.oauthbearer.assertion.private.key.pem` - alternatively to the key file
+ it's possible to pass the private key as a string.
+* `sasl.oauthbearer.assertion.file` - (optional) assertion file: with this property all other
+ assertion related fields are ignored and the assertion is read from this file
+ that should be periodically updated.
+* `sasl.oauthbearer.assertion.jwt.template.file` - (optional) template file: a template containing
+ a default `header` and `payload` that can be overwritten by the `claim` properties.
+* `sasl.oauthbearer.assertion.claim.aud`,
+ `sasl.oauthbearer.assertion.claim.exp.seconds`,
+ `sasl.oauthbearer.assertion.claim.iss`,
+ `sasl.oauthbearer.assertion.claim.jti.include`,
+ `sasl.oauthbearer.assertion.claim.sub` - (optional) the `claim` properties:
+ it's possible to dynamically customize the JWT claims with these or to
+ skip the template file and use only these properties.
+
+
+##### Metadata based authentication
+
+Some cloud providers added the ability to authenticate clients based on
+OAUTHBEARER/OIDC tokens returned from endpoints that can only be called from
+a given instance. Such endpoints are served on a specific IP address (169.254.169.254)
+that is a link-local metadata endpoint.
+
+
+While there is no standard for that still, librdkafka has support for
+some metadata based OAUTHBEARER authentication types.
+
+
+Currently these authentication types are supported:
+
+
+###### Azure IMDS
+
+To use this method you set:
+
+* `sasl.oauthbearer.metadata.authentication.type=azure_imds` this makes it so
+ that ` sasl.oauthbearer.client.id` and `sasl.oauthbearer.client.secret`
+ aren't required.
+* `sasl.oauthbearer.config` is a general purpose configuration property
+ In this case it accepts comma-separated `key=value` pairs.
+ The `query` key is required in case `sasl.oauthbearer.token.endpoint.url` isn't
+ specified and its value is the GET query string to append
+ to the token endpoint URL. Such query string contains params required by
+ Azure IMDS such as `client_id` (the UAMI), `resource` for determining the
+ target audience and `api-version` for the API version to be used by the endpoint
+* `sasl.oauthbearer.token.endpoint.url` (optional) is set automatically.
+ when choosing `sasl.oauthbearer.metadata.authentication.type=azure_imds` but can
+ be customized.
+
+
+_Example:_ `sasl.oauthbearer.metadata.authentication.type=azure_imds` and
+`sasl.oauthbearer.config=params=api-version=2025-04-07&resource=api://&client_id=`
+
+
+
#### Sparse connections
The client will only connect to brokers it needs to communicate with, and
@@ -1160,12 +1301,13 @@ only when necessary.
Examples of needed broker connections are:
- * leaders for partitions being consumed from
- * leaders for partitions being produced to
- * consumer group coordinator broker
- * cluster controller for Admin API operations
+* leaders for partitions being consumed from
+* leaders for partitions being produced to
+* consumer group coordinator broker
+* cluster controller for Admin API operations
+
##### Random broker selection
When there is no broker connection and a connection to any broker
@@ -1179,11 +1321,11 @@ If there is already an available broker connection to any broker it is used,
rather than connecting to a new one.
The random broker selection and connection scheduling is triggered when:
- * bootstrap servers are configured (`rd_kafka_new()`)
- * brokers are manually added (`rd_kafka_brokers_add()`).
- * a consumer group coordinator needs to be found.
- * acquiring a ProducerID for the Idempotent Producer.
- * cluster or topic metadata is being refreshed.
+* bootstrap servers are configured (`rd_kafka_new()`)
+* brokers are manually added (`rd_kafka_brokers_add()`).
+* a consumer group coordinator needs to be found.
+* acquiring a ProducerID for the Idempotent Producer.
+* cluster or topic metadata is being refreshed.
A single connection attempt will be performed, and the broker will
return to an idle INIT state on failure to connect.
@@ -1192,21 +1334,23 @@ The random broker selection is rate-limited to:
10 < `reconnect.backoff.ms`/2 < 1000 milliseconds.
**Note**: The broker connection will be maintained until it is closed
- by the broker (idle connection reaper).
+by the broker (idle connection reaper).
+
##### Persistent broker connections
While the random broker selection is useful for one-off queries, there
is need for the client to maintain persistent connections to certain brokers:
- * Consumer: the group coordinator.
- * Consumer: partition leader for topics being fetched from.
- * Producer: partition leader for topics being produced to.
+* Consumer: the group coordinator.
+* Consumer: partition leader for topics being fetched from.
+* Producer: partition leader for topics being produced to.
These dependencies are discovered and maintained automatically, marking
matching brokers as persistent, which will make the client maintain connections
to these brokers at all times, reconnecting as necessary.
+
#### Connection close
A broker connection may be closed by the broker, intermediary network gear,
@@ -1234,6 +1378,7 @@ the logging level will be LOG_WARNING (4), else LOG_INFO (6).
but it is recommended to instead rely on the above heuristics.
+
#### Fetch From Follower
librdkafka supports consuming messages from follower replicas
@@ -1244,51 +1389,54 @@ consumers to replicas is determined by the configured `replica.selector.class`
on the broker.
+
### Logging
+
#### Debug contexts
Extensive debugging of librdkafka can be enabled by setting the
`debug` configuration property to a CSV string of debug contexts:
-Debug context | Type | Description
---------------|----------|----------------------
-generic | * | General client instance level debugging. Includes initialization and termination debugging.
-broker | * | Broker and connection state debugging.
-topic | * | Topic and partition state debugging. Includes leader changes.
-metadata | * | Cluster and topic metadata retrieval debugging.
-feature | * | Kafka protocol feature support as negotiated with the broker.
-queue | producer | Message queue debugging.
-msg | * | Message debugging. Includes information about batching, compression, sizes, etc.
-protocol | * | Kafka protocol request/response debugging. Includes latency (rtt) printouts.
-cgrp | consumer | Low-level consumer group state debugging.
-security | * | Security and authentication debugging.
-fetch | consumer | Consumer message fetch debugging. Includes decision when and why messages are fetched.
-interceptor | * | Interceptor interface debugging.
-plugin | * | Plugin loading debugging.
-consumer | consumer | High-level consumer debugging.
-admin | admin | Admin API debugging.
-eos | producer | Idempotent Producer debugging.
-mock | * | Mock cluster functionality debugging.
-assignor | consumer | Detailed consumer group partition assignor debugging.
-conf | * | Display set configuration properties on startup.
-all | * | All of the above.
+| Debug context | Type | Description |
+| ------------- | -------- | ------------------------------------------------------------------------------------------- |
+| generic | * | General client instance level debugging. Includes initialization and termination debugging. |
+| broker | * | Broker and connection state debugging. |
+| topic | * | Topic and partition state debugging. Includes leader changes. |
+| metadata | * | Cluster and topic metadata retrieval debugging. |
+| feature | * | Kafka protocol feature support as negotiated with the broker. |
+| queue | producer | Message queue debugging. |
+| msg | * | Message debugging. Includes information about batching, compression, sizes, etc. |
+| protocol | * | Kafka protocol request/response debugging. Includes latency (rtt) printouts. |
+| cgrp | consumer | Low-level consumer group state debugging. |
+| security | * | Security and authentication debugging. |
+| fetch | consumer | Consumer message fetch debugging. Includes decision when and why messages are fetched. |
+| interceptor | * | Interceptor interface debugging. |
+| plugin | * | Plugin loading debugging. |
+| consumer | consumer | High-level consumer debugging. |
+| admin | admin | Admin API debugging. |
+| eos | producer | Idempotent Producer debugging. |
+| mock | * | Mock cluster functionality debugging. |
+| assignor | consumer | Detailed consumer group partition assignor debugging. |
+| conf | * | Display set configuration properties on startup. |
+| all | * | All of the above. |
Suggested debugging settings for troubleshooting:
-Problem space | Type | Debug setting
------------------------|----------|-------------------
-Producer not delivering messages to broker | producer | `broker,topic,msg`
-Consumer not fetching messages | consumer | Start with `consumer`, or use `cgrp,fetch` for detailed information.
-Consumer starts reading at unexpected offset | consumer | `consumer` or `cgrp,fetch`
-Authentication or connectivity issues | * | `broker,auth`
-Protocol handling or latency | * | `broker,protocol`
-Topic leader and state | * | `topic,metadata`
+| Problem space | Type | Debug setting |
+| -------------------------------------------- | -------- | -------------------------------------------------------------------- |
+| Producer not delivering messages to broker | producer | `broker,topic,msg` |
+| Consumer not fetching messages | consumer | Start with `consumer`, or use `cgrp,fetch` for detailed information. |
+| Consumer starts reading at unexpected offset | consumer | `consumer` or `cgrp,fetch` |
+| Authentication or connectivity issues | * | `broker,auth` |
+| Protocol handling or latency | * | `broker,protocol` |
+| Topic leader and state | * | `topic,metadata` |
+
### Feature discovery
Apache Kafka broker version 0.10.0 added support for the ApiVersionRequest API
@@ -1305,6 +1453,7 @@ and is controlled by the `broker.version.fallback` configuration property.
+
### Producer API
After setting up the `rd_kafka_t` object with type `RD_KAFKA_PRODUCER` and one
@@ -1313,36 +1462,36 @@ to be produced and sent to brokers.
The `rd_kafka_produce()` function takes the following arguments:
- * `rkt` - the topic to produce to, previously created with
- `rd_kafka_topic_new()`
- * `partition` - partition to produce to. If this is set to
- `RD_KAFKA_PARTITION_UA` (UnAssigned) then the configured partitioner
- function will be used to select a target partition.
- * `msgflags` - 0, or one of:
- * `RD_KAFKA_MSG_F_COPY` - librdkafka will immediately make a copy of
- the payload. Use this when the payload is in non-persistent
- memory, such as the stack.
- * `RD_KAFKA_MSG_F_FREE` - let librdkafka free the payload using
- `free(3)` when it is done with it.
-
- These two flags are mutually exclusive and neither need to be set in
- which case the payload is neither copied nor freed by librdkafka.
-
- If `RD_KAFKA_MSG_F_COPY` flag is not set no data copying will be
- performed and librdkafka will hold on the payload pointer until
- the message has been delivered or fails.
- The delivery report callback will be called when librdkafka is done
- with the message to let the application regain ownership of the
- payload memory.
- The application must not free the payload in the delivery report
- callback if `RD_KAFKA_MSG_F_FREE is set`.
- * `payload`,`len` - the message payload
- * `key`,`keylen` - an optional message key which can be used for partitioning.
- It will be passed to the topic partitioner callback, if any, and
- will be attached to the message when sending to the broker.
- * `msg_opaque` - an optional application-provided per-message opaque pointer
- that will be provided in the message delivery callback to let
- the application reference a specific message.
+* `rkt` - the topic to produce to, previously created with
+ `rd_kafka_topic_new()`
+* `partition` - partition to produce to. If this is set to
+ `RD_KAFKA_PARTITION_UA` (UnAssigned) then the configured partitioner
+ function will be used to select a target partition.
+* `msgflags` - 0, or one of:
+ * `RD_KAFKA_MSG_F_COPY` - librdkafka will immediately make a copy of
+ the payload. Use this when the payload is in non-persistent
+ memory, such as the stack.
+ * `RD_KAFKA_MSG_F_FREE` - let librdkafka free the payload using
+ `free(3)` when it is done with it.
+
+ These two flags are mutually exclusive and neither need to be set in
+ which case the payload is neither copied nor freed by librdkafka.
+
+ If `RD_KAFKA_MSG_F_COPY` flag is not set no data copying will be
+ performed and librdkafka will hold on the payload pointer until
+ the message has been delivered or fails.
+ The delivery report callback will be called when librdkafka is done
+ with the message to let the application regain ownership of the
+ payload memory.
+ The application must not free the payload in the delivery report
+ callback if `RD_KAFKA_MSG_F_FREE is set`.
+* `payload`,`len` - the message payload
+* `key`,`keylen` - an optional message key which can be used for partitioning.
+ It will be passed to the topic partitioner callback, if any, and
+ will be attached to the message when sending to the broker.
+* `msg_opaque` - an optional application-provided per-message opaque pointer
+ that will be provided in the message delivery callback to let
+ the application reference a specific message.
`rd_kafka_produce()` is a non-blocking API, it will enqueue the message
@@ -1362,6 +1511,7 @@ message fields, such as timestamp and headers.
**Note**: See `examples/rdkafka_performance.c` for a producer implementation.
+
### Simple Consumer API (legacy)
NOTE: For the high-level KafkaConsumer interface see rd_kafka_subscribe (rdkafka.h) or KafkaConsumer (rdkafkacpp.h)
@@ -1373,16 +1523,16 @@ for a given partition by calling `rd_kafka_consume_start()`.
`rd_kafka_consume_start()` arguments:
- * `rkt` - the topic to start consuming from, previously created with
- `rd_kafka_topic_new()`.
- * `partition` - partition to consume from.
- * `offset` - message offset to start consuming from. This may either be an
- absolute message offset or one of the three special offsets:
- `RD_KAFKA_OFFSET_BEGINNING` to start consuming from the beginning
- of the partition's queue (oldest message), or
- `RD_KAFKA_OFFSET_END` to start consuming at the next message to be
- produced to the partition, or
- `RD_KAFKA_OFFSET_STORED` to use the offset store.
+* `rkt` - the topic to start consuming from, previously created with
+ `rd_kafka_topic_new()`.
+* `partition` - partition to consume from.
+* `offset` - message offset to start consuming from. This may either be an
+ absolute message offset or one of the three special offsets:
+ `RD_KAFKA_OFFSET_BEGINNING` to start consuming from the beginning
+ of the partition's queue (oldest message), or
+ `RD_KAFKA_OFFSET_END` to start consuming at the next message to be
+ produced to the partition, or
+ `RD_KAFKA_OFFSET_STORED` to use the offset store.
After a topic+partition consumer has been started librdkafka will attempt
to keep `queued.min.messages` messages in the local queue by repeatedly
@@ -1393,10 +1543,10 @@ request.
This local message queue is then served to the application through three
different consume APIs:
- * `rd_kafka_consume()` - consumes a single message
- * `rd_kafka_consume_batch()` - consumes one or more messages
- * `rd_kafka_consume_callback()` - consumes all messages in the local
- queue and calls a callback function for each one.
+* `rd_kafka_consume()` - consumes a single message
+* `rd_kafka_consume_batch()` - consumes one or more messages
+* `rd_kafka_consume_callback()` - consumes all messages in the local
+ queue and calls a callback function for each one.
These three APIs are listed above the ascending order of performance,
`rd_kafka_consume()` being the slowest and `rd_kafka_consume_callback()` being
@@ -1408,15 +1558,15 @@ is represented by the `rd_kafka_message_t` type.
`rd_kafka_message_t` members:
- * `err` - Error signaling back to the application. If this field is non-zero
- the `payload` field should be considered an error message and
- `err` is an error code (`rd_kafka_resp_err_t`).
- If `err` is zero then the message is a proper fetched message
- and `payload` et.al contains message payload data.
- * `rkt`,`partition` - Topic and partition for this message or error.
- * `payload`,`len` - Message payload data or error message (err!=0).
- * `key`,`key_len` - Optional message key as specified by the producer
- * `offset` - Message offset
+* `err` - Error signaling back to the application. If this field is non-zero
+ the `payload` field should be considered an error message and
+ `err` is an error code (`rd_kafka_resp_err_t`).
+ If `err` is zero then the message is a proper fetched message
+ and `payload` et.al contains message payload data.
+* `rkt`,`partition` - Topic and partition for this message or error.
+* `payload`,`len` - Message payload data or error message (err!=0).
+* `key`,`key_len` - Optional message key as specified by the producer
+* `offset` - Message offset
Both the `payload` and `key` memory, as well as the message as a whole, is
owned by librdkafka and must not be used after an `rd_kafka_message_destroy()`
@@ -1434,6 +1584,7 @@ purge any messages currently in the local queue.
**Note**: See `examples/rdkafka_performance.c` for a consumer implementation.
+
#### Offset management
Broker based offset management is available for broker version >= 0.9.0
@@ -1444,16 +1595,17 @@ Offset management is also available through a deprecated local offset file,
where the offset is periodically written to a local file for each
topic+partition according to the following topic configuration properties:
- * `enable.auto.commit`
- * `auto.commit.interval.ms`
- * `offset.store.path`
- * `offset.store.sync.interval.ms`
+* `enable.auto.commit`
+* `auto.commit.interval.ms`
+* `offset.store.path`
+* `offset.store.sync.interval.ms`
The legacy `auto.commit.enable` topic configuration property is only to be used
with the legacy low-level consumer.
Use `enable.auto.commit` with the modern KafkaConsumer.
+
##### Auto offset commit
The consumer will automatically commit offsets every `auto.commit.interval.ms`
@@ -1464,6 +1616,7 @@ this offset store is updated by `consumer_poll()` (et.al) to
store the offset of the last message passed to the application
(per topic+partition).
+
##### At-least-once processing
Since auto commits are performed in a background thread this may result in
the offset for the latest message being committed before the application has
@@ -1483,10 +1636,11 @@ The latest stored offset will be automatically committed every
`auto.commit.interval.ms`.
**Note**: Only greater offsets are committed, e.g., if the latest committed
- offset was 10 and the application performs an offsets_store()
- with offset 9, that offset will not be committed.
+offset was 10 and the application performs an offsets_store()
+with offset 9, that offset will not be committed.
+
##### Auto offset reset
The consumer will by default try to acquire the last committed offsets for
@@ -1495,14 +1649,15 @@ If there is no committed offset available, or the consumer is unable to
fetch the committed offsets, the policy of `auto.offset.reset` will kick in.
This configuration property may be set to one the following values:
- * `earliest` - start consuming the earliest message of the partition.
- * `latest` - start consuming the next message to be produced to the partition.
- * `error` - don't start consuming but isntead raise a consumer error
- with error-code `RD_KAFKA_RESP_ERR__AUTO_OFFSET_RESET` for
- the topic+partition. This allows the application to decide what
- to do in case there is no committed start offset.
+* `earliest` - start consuming the earliest message of the partition.
+* `latest` - start consuming the next message to be produced to the partition.
+* `error` - don't start consuming but isntead raise a consumer error
+ with error-code `RD_KAFKA_RESP_ERR__AUTO_OFFSET_RESET` for
+ the topic+partition. This allows the application to decide what
+ to do in case there is no committed start offset.
+
### Consumer groups
Broker based consumer groups (requires Apache Kafka broker >=0.9) are supported,
@@ -1515,6 +1670,7 @@ group coordinator, and partition leader(s).

+
#### Static consumer groups
By default Kafka consumers are rebalanced each time a new consumer joins
@@ -1540,99 +1696,218 @@ the original fatal error code and reason.
To read more about static group membership, see [KIP-345](https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances).
+
+### Next Generation Consumer Group Protocol ([KIP-848](https://cwiki.apache.org/confluence/display/KAFKA/KIP-848%3A+The+Next+Generation+of+the+Consumer+Rebalance+Protocol))
-### Next generation of the consumer group protocol: [KIP 848](https://cwiki.apache.org/confluence/display/KAFKA/KIP-848%3A+The+Next+Generation+of+the+Consumer+Rebalance+Protocol)
+Starting with **librdkafka v2.12.0** (GA release), the next generation consumer group rebalance protocol defined in **[KIP-848](https://cwiki.apache.org/confluence/display/KAFKA/KIP-848%3A+The+Next+Generation+of+the+Consumer+Rebalance+Protocol)** is **production-ready**.
-Starting from librdkafka 2.4.0 the next generation consumer group rebalance protocol
-defined in KIP 848 is introduced.
+**Note:** The new consumer group protocol defined in [KIP-848](https://cwiki.apache.org/confluence/display/KAFKA/KIP-848%3A+The+Next+Generation+of+the+Consumer+Rebalance+Protocol) is not enabled by default. There are few contract change associated with the new protocol and might cause breaking changes. `group.protocol` configuration property dictates whether to use the new `consumer` protocol or older `classic` protocol. It defaults to `classic` if not provided.
-**Warning**
-It's still in **Early Access** which means it's _not production-ready_,
-given it's still under validation and lacking some needed features.
-Features and their contract might change in future.
+
+#### Overview
+- **What changed:**
+ The **Group Leader role** (consumer member) is removed. Assignments are calculated by the **Group Coordinator (broker)** and distributed via **heartbeats**.
-With this protocol the role of the Group Leader (a member) is removed and
-the assignment is calculated by the Group Coordinator (a broker) and sent
-to each member through heartbeats.
+- **Requirements:**
+ - Broker version: **v4.0.0+**
+ - librdkafka version: **v2.12.0+**: GA (production-ready)
-To test it, a Kafka cluster must be set up, in KRaft mode, and the new group
-protocol enabled with the `group.coordinator.rebalance.protocols` property.
-Broker version must be Apache Kafka 3.7.0 or newer. See Apache Kafka
-[Release Notes](https://cwiki.apache.org/confluence/display/KAFKA/The+Next+Generation+of+the+Consumer+Rebalance+Protocol+%28KIP-848%29+-+Early+Access+Release+Notes).
+- **Enablement (client-side):**
+ - `group.protocol=consumer`
+ - `group.remote.assignor=` (optional; broker-controlled if `NULL`; default broker assignor is **`uniform`**)
-Client side, it can be enabled by setting the new property `group.protocol=consumer`.
-A second property named `group.remote.assignor` is added to choose desired
-remote assignor.
+
+#### Available Features
-**Available features**
+All [KIP-848](https://cwiki.apache.org/confluence/display/KAFKA/KIP-848%3A+The+Next+Generation+of+the+Consumer+Rebalance+Protocol) features are supported including:
-- Subscription to one or more topics
-- Rebalance callbacks (see contract changes)
+- Subscription to one or more topics, including **regular expression (regex) subscriptions**
+- Rebalance callbacks (**incremental only**)
- Static group membership
-- Configure remote assignor
-- Max poll interval is enforced
-- Offline upgrade from an empty consumer group with committed offsets
-
-**Future features**
-
-- Regular expression support when subscribing
-- AdminClient changes as described in the KIP
-
-**Contract changes**
-
-Along with the new feature there are some needed contract changes,
-so the protocol will be enabled by default only with a librdkafka major release.
-
- - Deprecated client configurations with the new protocol:
- - `partition.assignment.strategy` replaced by `group.remote.assignor`
- - `session.timeout.ms` replaced by broker configuration `group.consumer.session.timeout.ms`
- - `heartbeat.interval.ms`, replaced by broker configuration `group.consumer.heartbeat.interval.ms`
- - `group.protocol.type` which is not used in the new protocol
-
- - Protocol rebalance is fully incremental, so the only allowed functions to
- use in a rebalance callback will be `rd_kafka_incremental_assign` and
- `rd_kafka_incremental_unassign`. Currently you can still use existing code
- and the expected function to call is determined based on the chosen
- `partition.assignment.strategy` but this will be removed in next
- release.
-
- When setting the `group.remote.assignor` property, it's already
- required to use the incremental assign and unassign functions.
- All assignors are sticky with new protocol, including the _range_ one, that wasn't.
-
- - With a static group membership, if two members are using the same
- `group.instance.id`, the one that joins the consumer group later will be
- fenced, with the fatal `UNRELEASED_INSTANCE_ID` error. Before, it was the existing
- member to be fenced. This was changed to avoid two members contending the
- same id. It also means that any instance that crashes won't be automatically
- replaced by a new instance until session times out and it's especially required
- to check that consumers are being closed properly on shutdown. Ensuring that
- no two instances with same `group.instance.id` are running at any time
- is also important.
-
- - Session timeout is remote only and, if the Coordinator isn't reachable
- by a member, this will continue to fetch messages, even if it won't be able to
- commit them. Otherwise, the member will be fenced as soon as it receives an
- heartbeat response from the Coordinator.
- With `classic` protocol, instead, member stops fetching when session timeout
- expires on the client.
-
- For the same reason, when closing or unsubscribing with auto-commit set,
- the member will try to commit until a specific timeout has passed.
- Currently the timeout is the same as the `classic` protocol and it corresponds
- to the `session.timeout.ms`, but it will change before the feature
- reaches a stable state.
-
- - An `UNKNOWN_TOPIC_OR_PART` error isn't received anymore when a consumer is
- subscribing to a topic that doesn't exist in local cache, as the consumer
- is still subscribing to the topic and it could be created just after that.
-
- - A consumer won't do a preliminary Metadata call that returns a
- `TOPIC_AUTHORIZATION_FAILED`, as it's happening with group protocol `classic`.
- Topic partitions will still be assigned to the member
- by the Coordinator only if it's authorized to consume from the topic.
+- Configurable remote assignor
+- Enforced max poll interval
+- Upgrade from `classic` protocol or downgrade from `consumer` protocol
+- AdminClient changes as per KIP
+
+
+#### Contract Changes
+
+
+##### Client Configuration changes
+
+| Classic Protocol (Deprecated Configs in KIP-848) | KIP-848 / Next-Gen Replacement |
+| ------------------------------------------------ | ----------------------------------------------------- |
+| `partition.assignment.strategy` | `group.remote.assignor` |
+| `session.timeout.ms` | Broker config: `group.consumer.session.timeout.ms` |
+| `heartbeat.interval.ms` | Broker config: `group.consumer.heartbeat.interval.ms` |
+| `group.protocol.type` | Not used in the new protocol |
+
+**Note:** The properties listed under “Classic Protocol (Deprecated Configs in KIP-848)” are **no longer used** when using the KIP-848 consumer protocol.
+
+
+##### Rebalance Callback Changes
+
+- Protocol is **fully incremental**.
+- **Inside the rebalance callback**, you **must use**:
+ - `rd_kafka_incremental_assign(rk, partitions)` to assign partitions
+ - `rd_kafka_incremental_unassign(rk, partitions)` to revoke partitions
+- **Do not** use `rd_kafka_assign()` or other assignment APIs in KIP-848.
+- **Important:** The `partitions` parameter passed to `rd_kafka_incremental_assign` or `rd_kafka_incremental_unassign` contains only an **incremental list of partitions**—those being added or revoked—rather than the full partition list returned by `rd_kafka_assign(rk, partitions)` in the **range assignor of the classic protocol**, which was the default.
+- All assignors are **sticky**, including `range` (which wasn’t sticky before).
+
+
+##### Static Group Membership
+
+- Duplicate `group.instance.id` handling:
+ - **Newly joining member** is fenced with **UNRELEASED_INSTANCE_ID (fatal)**.
+ - (Classic protocol fenced the **existing** member instead.)
+- Implications:
+ - Ensure only **one active instance per `group.instance.id`**.
+ - Consumers must shut down cleanly to avoid blocking replacements until session timeout expires.
+
+
+##### Session Timeout & Fetching
+
+- **Session timeout is broker-controlled**:
+ - If the Coordinator is unreachable, a consumer **continues fetching messages** but cannot commit offsets.
+ - Consumer is fenced once a heartbeat response is received from the Coordinator.
+- In the classic protocol, the client stopped fetching when session timeout expired.
+
+
+##### Closing / Auto-Commit
+
+- On `close()` or unsubscribe with auto-commit enabled:
+ - Member retries committing offsets until a timeout expires.
+ - Currently uses the **default remote session timeout**.
+ - Future **KIP-1092** will allow custom commit timeouts.
+
+
+##### Error Handling Changes
+
+- `UNKNOWN_TOPIC_OR_PART` (**subscription case**):
+ - No longer returned if a topic is missing in the **local cache** when subscribing; the subscription proceeds.
+- `TOPIC_AUTHORIZATION_FAILED`:
+ - Reported once per heartbeat or subscription change, even if only one topic is unauthorized.
+
+
+##### Summary of Key Differences (Classic vs Next-Gen)
+
+- **Assignment:** Classic protocol calculated by **Group Leader (consumer)**; KIP-848 calculated by **Group Coordinator (broker)**
+- **Assignors:** Classic range assignor was **not sticky**; KIP-848 assignors are **sticky**, including range
+- **Deprecated configs:** Classic client configs are replaced by `group.remote.assignor` and broker-controlled session/heartbeat configs
+- **Static membership fencing:** KIP-848 fences **new member** on duplicate `group.instance.id`
+- **Session timeout:** Classic enforced on client; KIP-848 enforced on broker
+- **Auto-commit on close:** Classic stops at client session timeout; KIP-848 retries until remote timeout
+- **Unknown topics:** KIP-848 does not return error on subscription if topic missing
+- **Upgrade/Downgrade:** KIP-848 supports upgrade/downgrade from/to `classic` and `consumer` protocols
+
+
+#### Minimal Example Config
+
+
+##### Classic Protocol
+```properties
+# Optional; default is 'classic'
+group.protocol=classic
+
+partition.assignment.strategy=
+session.timeout.ms=45000
+heartbeat.interval.ms=15000
+```
+
+
+##### Next-Gen Protocol / KIP-848
+```properties
+group.protocol=consumer
+
+# Optional: select a remote assignor
+# Valid options currently: 'uniform' or 'range'
+# group.remote.assignor=
+# If unset(NULL), broker chooses the assignor (default: 'uniform')
+
+# Session & heartbeat now controlled by broker:
+# group.consumer.session.timeout.ms
+# group.consumer.heartbeat.interval.ms
+```
+
+
+#### Rebalance Callback Migration
+
+
+##### Range Assignor (Classic)
+```c
+/* Rebalance callback for range assignor (classic) */
+static void rebalance_cb (rd_kafka_t *rk,
+ rd_kafka_resp_err_t err,
+ rd_kafka_topic_partition_list_t *partitions,
+ void *opaque) {
+ switch (err) {
+ case RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS:
+ rd_kafka_assign(rk, partitions); /* full partition list */
+ break;
+
+ case RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS:
+ rd_kafka_assign(rk, NULL); /* revoke all partitions */
+ break;
+
+ default:
+ fprintf(stderr, "Rebalance error: %s\n", rd_kafka_err2str(err));
+ break;
+ }
+}
+```
+
+
+##### Incremental Assignor (Including Range in Consumer / KIP-848, Any Protocol)
+
+```c
+/* Rebalance callback for incremental assignor */
+static void rebalance_cb (rd_kafka_t *rk,
+ rd_kafka_resp_err_t err,
+ rd_kafka_topic_partition_list_t *partitions,
+ void *opaque) {
+ switch (err) {
+ case RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS:
+ rd_kafka_incremental_assign(rk, partitions); /* incremental partitions only */
+ break;
+
+ case RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS:
+ rd_kafka_incremental_unassign(rk, partitions);
+ break;
+
+ default:
+ fprintf(stderr, "Rebalance error: %s\n", rd_kafka_err2str(err));
+ break;
+ }
+}
+```
+**Note:**
+- The `partitions` list contains **only partitions being added or revoked**, not the full partition list as in the classic `rd_kafka_assign()`.
+
+#### Upgrade and Downgrade
+- A group made up entirely of `classic` consumers runs under the classic protocol.
+- The group is **upgraded to the consumer protocol** as soon as at least one `consumer` protocol member joins.
+- The group is **downgraded back to the classic protocol** if the last `consumer` protocol member leaves while `classic` members remain.
+- Both **rolling upgrade** (classic → consumer) and **rolling downgrade** (consumer → classic) are supported.
+
+
+
+#### Migration Checklist (Next-Gen Protocol / [KIP-848](https://cwiki.apache.org/confluence/display/KAFKA/KIP-848%3A+The+Next+Generation+of+the+Consumer+Rebalance+Protocol))
+
+1. Upgrade to **librdkafka ≥ v2.12.0** (GA release)
+2. Run against **Kafka brokers ≥ v4.0.0**
+3. Set `group.protocol=consumer`
+4. Optionally set `group.remote.assignor`; leave `NULL` for broker-controlled (default: `uniform`), valid options: `uniform` or `range`
+5. Replace deprecated configs with new ones
+6. Update rebalance callbacks to **incremental APIs only**
+7. Review static membership handling (`group.instance.id`)
+8. Ensure proper shutdown to avoid fencing issues
+9. Adjust error handling for unknown topics and authorization failures
+
+
+
### Note on Batch consume APIs
Using multiple instances of `rd_kafka_consume_batch()` and/or `rd_kafka_consume_batch_queue()`
@@ -1653,8 +1928,10 @@ set `rebalance_cb` configuration property (refer [examples/rdkafka_complex_consu
for the help with the usage) for the consumer.
+
### Topics
+
#### Unknown or unauthorized topics
If a consumer application subscribes to non-existent or unauthorized topics
@@ -1679,6 +1956,7 @@ occurs, for instance adjusting its subscription or assignment to exclude the
unauthorized topic.
+
#### Topic metadata propagation for newly created topics
Due to the asynchronous nature of topic creation in Apache Kafka it may
@@ -1702,12 +1980,15 @@ Should the topic propagation time expire without the topic being seen the
produced messages will fail with `RD_KAFKA_RESP_ERR__UNKNOWN_TOPIC`.
**Note**: The propagation time will not take affect if a topic is known to
- the client and then deleted, in this case the topic will immediately
- be marked as non-existent and remain non-existent until a topic
- metadata refresh sees the topic again (after the topic has been
- re-created).
+the client and then deleted, in this case the topic will immediately
+be marked as non-existent and remain non-existent until a topic
+metadata refresh sees the topic again (after the topic has been
+re-created).
+
+**Note**: `RD_KAFKA_RESP_ERR__UNKNOWN_TOPIC*` during a `subscribe()` call occurs **only with the classic protocol**. With the next-gen `consumer` protocol (KIP-848), subscription proceeds even if the topic is not yet in the local cache (e.g., it may be created later).
+
#### Topic auto creation
Topic auto creation is supported by librdkafka, if a non-existent topic is
@@ -1733,41 +2014,46 @@ be configured on the broker.
+
### Metadata
+
#### < 0.9.3
Previous to the 0.9.3 release librdkafka's metadata handling
was chatty and excessive, which usually isn't a problem in small
to medium-sized clusters, but in large clusters with a large amount
of librdkafka clients the metadata requests could hog broker CPU and bandwidth.
+
#### > 0.9.3
The remaining Metadata sections describe the current behaviour.
**Note:** "Known topics" in the following section means topics for
- locally created `rd_kafka_topic_t` objects.
+locally created `rd_kafka_topic_t` objects.
+
#### Query reasons
There are four reasons to query metadata:
- * brokers - update/populate cluster broker list, so the client can
- find and connect to any new brokers added.
+* brokers - update/populate cluster broker list, so the client can
+ find and connect to any new brokers added.
- * specific topic - find leader or partition count for specific topic
+* specific topic - find leader or partition count for specific topic
- * known topics - same, but for all locally known topics.
+* known topics - same, but for all locally known topics.
- * all topics - get topic names for consumer group wildcard subscription
- matching
+* all topics - get topic names for consumer group wildcard subscription
+ matching
The above list is sorted so that the sub-sequent entries contain the
information above, e.g., 'known topics' contains enough information to
also satisfy 'specific topic' and 'brokers'.
+
#### Caching strategy
The prevalent cache timeout is `metadata.max.age.ms`, any cached entry
@@ -1775,35 +2061,37 @@ will remain authoritative for this long or until a relevant broker error
is returned.
- * brokers - eternally cached, the broker list is additative.
+* brokers - eternally cached, the broker list is additative.
- * topics - cached for `metadata.max.age.ms`
+* topics - cached for `metadata.max.age.ms`
+
### Fatal errors
If an unrecoverable error occurs, a fatal error is triggered in one
or more of the follow ways depending on what APIs the application is utilizing:
- * C: the `error_cb` is triggered with error code `RD_KAFKA_RESP_ERR__FATAL`,
- the application should call `rd_kafka_fatal_error()` to retrieve the
- underlying fatal error code and error string.
- * C: an `RD_KAFKA_EVENT_ERROR` event is triggered and
- `rd_kafka_event_error_is_fatal()` returns true: the fatal error code
- and string are available through `rd_kafka_event_error()`, and `.._string()`.
- * C and C++: any API call may return `RD_KAFKA_RESP_ERR__FATAL`, use
- `rd_kafka_fatal_error()` to retrieve the underlying fatal error code
- and error string.
- * C++: an `EVENT_ERROR` event is triggered and `event.fatal()` returns true:
- the fatal error code and string are available through `event.err()` and
- `event.str()`.
+* C: the `error_cb` is triggered with error code `RD_KAFKA_RESP_ERR__FATAL`,
+ the application should call `rd_kafka_fatal_error()` to retrieve the
+ underlying fatal error code and error string.
+* C: an `RD_KAFKA_EVENT_ERROR` event is triggered and
+ `rd_kafka_event_error_is_fatal()` returns true: the fatal error code
+ and string are available through `rd_kafka_event_error()`, and `.._string()`.
+* C and C++: any API call may return `RD_KAFKA_RESP_ERR__FATAL`, use
+ `rd_kafka_fatal_error()` to retrieve the underlying fatal error code
+ and error string.
+* C++: an `EVENT_ERROR` event is triggered and `event.fatal()` returns true:
+ the fatal error code and string are available through `event.err()` and
+ `event.str()`.
An application may call `rd_kafka_fatal_error()` at any time to check if
a fatal error has been raised.
+
#### Fatal producer errors
The idempotent producer guarantees of ordering and no duplicates also
@@ -1813,9 +2101,9 @@ can't be satisfied.
If a fatal error has been raised, sub-sequent use of the following API calls
will fail:
- * `rd_kafka_produce()`
- * `rd_kafka_producev()`
- * `rd_kafka_produce_batch()`
+* `rd_kafka_produce()`
+* `rd_kafka_producev()`
+* `rd_kafka_produce_batch()`
The underlying fatal error code will be returned, depending on the error
reporting scheme for each of those APIs.
@@ -1831,6 +2119,7 @@ The purged messages in queue will fail with error code set to
`RD_KAFKA_RESP_ERR__PURGE_QUEUE`.
+
#### Fatal consumer errors
A consumer configured for static group membership (`group.instance.id`) may
@@ -1838,13 +2127,13 @@ raise a fatal error if a new consumer instance is started with the same
instance id, causing the existing consumer to be fenced by the new consumer.
This fatal error is propagated on the fenced existing consumer in multiple ways:
- * `error_cb` (if configured) is triggered.
- * `rd_kafka_consumer_poll()` (et.al) will return a message object
- with the `err` field set to `RD_KAFKA_ERR__FATAL`.
- * any sub-sequent calls to state-changing consumer calls will
- return `RD_KAFKA_ERR___FATAL`.
- This includes `rd_kafka_subscribe()`, `rd_kafka_assign()`,
- `rd_kafka_consumer_close()`, `rd_kafka_commit*()`, etc.
+* `error_cb` (if configured) is triggered.
+* `rd_kafka_consumer_poll()` (et.al) will return a message object
+ with the `err` field set to `RD_KAFKA_ERR__FATAL`.
+* any sub-sequent calls to state-changing consumer calls will
+ return `RD_KAFKA_ERR___FATAL`.
+ This includes `rd_kafka_subscribe()`, `rd_kafka_assign()`,
+ `rd_kafka_consumer_close()`, `rd_kafka_commit*()`, etc.
The consumer will automatically stop consuming when a fatal error has occurred
and no further subscription, assignment, consumption or offset committing
@@ -1853,8 +2142,10 @@ consumer instance and terminate the application since it has been replaced
by a newer instance.
+
## Compatibility
+
### Broker version compatibility
librdkafka supports all released Apache Kafka broker versions since 0.8.0.0.0,
@@ -1862,13 +2153,14 @@ but not all features may be available on all broker versions since some
features rely on newer broker functionality.
**Current defaults:**
- * `api.version.request=true`
- * `broker.version.fallback=0.10.0`
- * `api.version.fallback.ms=0` (never revert to `broker.version.fallback`)
+* `api.version.request=true`
+* `broker.version.fallback=0.10.0`
+* `api.version.fallback.ms=0` (never revert to `broker.version.fallback`)
Depending on what broker version you are using, please configure your
librdkafka based client as follows:
+
#### Broker version >= 0.10.0.0 (or trunk)
For librdkafka >= v1.0.0 there is no need to set any api.version-related
@@ -1882,6 +2174,7 @@ api.version.fallback.ms=0
```
+
#### Broker versions 0.9.0.x
```
@@ -1889,6 +2182,7 @@ api.version.request=false
broker.version.fallback=0.9.0.x (the exact 0.9.0.. version you are using)
```
+
#### Broker versions 0.8.x.y
```
@@ -1896,6 +2190,7 @@ api.version.request=false
broker.version.fallback=0.8.x.y (your exact 0.8... broker version)
```
+
#### Detailed description
Apache Kafka version 0.10.0.0 added support for
@@ -1906,27 +2201,27 @@ But for older broker versions there is no way for the client to reliably know
what protocol features the broker supports.
To alleviate this situation librdkafka has three configuration properties:
- * `api.version.request=true|false` - enables the API version request,
- this requires a >= 0.10.0.0 broker and will cause a disconnect on
- brokers 0.8.x - this disconnect is recognized by librdkafka and on the next
- connection attempt (which is immediate) it will disable the API version
- request and use `broker.version.fallback` as a basis of available features.
- **NOTE**: Due to a bug in broker version 0.9.0.0 & 0.9.0.1 the broker will
- not close the connection when receiving the API version request, instead
- the request will time out in librdkafka after 10 seconds and it will fall
- back to `broker.version.fallback` on the next immediate connection attempt.
- * `broker.version.fallback=X.Y.Z.N` - if the API version request fails
- (if `api.version.request=true`) or API version requests are disabled
- (`api.version.request=false`) then this tells librdkafka what version the
- broker is running and adapts its feature set accordingly.
- * `api.version.fallback.ms=MS` - In the case where `api.version.request=true`
- and the API version request fails, this property dictates for how long
- librdkafka will use `broker.version.fallback` instead of
- `api.version.request=true`. After `MS` has passed the API version request
- will be sent on any new connections made for the broker in question.
- This allows upgrading the Kafka broker to a new version with extended
- feature set without needing to restart or reconfigure the client
- (given that `api.version.request=true`).
+* `api.version.request=true|false` - enables the API version request,
+ this requires a >= 0.10.0.0 broker and will cause a disconnect on
+ brokers 0.8.x - this disconnect is recognized by librdkafka and on the next
+ connection attempt (which is immediate) it will disable the API version
+ request and use `broker.version.fallback` as a basis of available features.
+ **NOTE**: Due to a bug in broker version 0.9.0.0 & 0.9.0.1 the broker will
+ not close the connection when receiving the API version request, instead
+ the request will time out in librdkafka after 10 seconds and it will fall
+ back to `broker.version.fallback` on the next immediate connection attempt.
+* `broker.version.fallback=X.Y.Z.N` - if the API version request fails
+ (if `api.version.request=true`) or API version requests are disabled
+ (`api.version.request=false`) then this tells librdkafka what version the
+ broker is running and adapts its feature set accordingly.
+* `api.version.fallback.ms=MS` - In the case where `api.version.request=true`
+ and the API version request fails, this property dictates for how long
+ librdkafka will use `broker.version.fallback` instead of
+ `api.version.request=true`. After `MS` has passed the API version request
+ will be sent on any new connections made for the broker in question.
+ This allows upgrading the Kafka broker to a new version with extended
+ feature set without needing to restart or reconfigure the client
+ (given that `api.version.request=true`).
*Note: These properties applies per broker.*
@@ -1936,13 +2231,14 @@ broker version 0.9.0.0 & 0.9.0.1, but was changed to `true` in
librdkafka v0.11.0.
+
### Supported KIPs
The [Apache Kafka Implementation Proposals (KIPs)](https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals) supported by librdkafka.
| KIP | Kafka release | Status |
-|--------------------------------------------------------------------------|-----------------------------|-----------------------------------------------------------------------------------------------|
+| ------------------------------------------------------------------------ | --------------------------- | --------------------------------------------------------------------------------------------- |
| KIP-1 - Stop accepting request.required.acks > 1 | 0.9.0.0 | Not enforced on client (due to backwards compat with brokers <0.8.3) |
| KIP-4 - Metadata protocol changes | 0.9.0.0, 0.10.0.0, 0.10.1.0 | Supported |
| KIP-8 - Producer flush() | 0.9.0.0 | Supported |
@@ -2024,7 +2320,7 @@ The [Apache Kafka Implementation Proposals (KIPs)](https://cwiki.apache.org/conf
| KIP-436 - Start time in stats | 2.3.0 | Supported |
| KIP-447 - Producer scalability for EOS | 2.5.0 | Supported |
| KIP-455 - AdminAPI: Replica assignment | 2.4.0 (WIP) | Not supported |
-| KIP-460 - AdminAPI: electPreferredLeader | 2.4.0 | Not supported |
+| KIP-460 - AdminAPI: electLeaders | 2.6.0 | Supported |
| KIP-464 - AdminAPI: defaults for createTopics | 2.4.0 | Supported |
| KIP-467 - Per-message (sort of) error codes in ProduceResponse | 2.4.0 | Supported |
| KIP-480 - Sticky partitioner | 2.4.0 | Supported |
@@ -2055,66 +2351,75 @@ The [Apache Kafka Implementation Proposals (KIPs)](https://cwiki.apache.org/conf
| KIP-735 - Increase default consumer session timeout | 3.0.0 | Supported |
| KIP-768 - SASL/OAUTHBEARER OIDC support | 3.0 | Supported |
| KIP-881 - Rack-aware Partition Assignment for Kafka Consumers | 3.5.0 | Supported |
-| KIP-848 - The Next Generation of the Consumer Rebalance Protocol | 3.7.0 (EA) | Early Access |
+| KIP-848 - The Next Generation of the Consumer Rebalance Protocol | 4.0.0 | Supported |
+| KIP-899 - Allow producer and consumer clients to rebootstrap | 3.8.0 | Supported |
| KIP-951 - Leader discovery optimisations for the client | 3.7.0 | Supported |
+| KIP-1082 - Require Client-Generated IDs over the ConsumerGroupHeartbeat | 4.0.0 | Supported |
+| KIP-1102 - Enable clients to rebootstrap based on timeout or error code | 4.0.0 | Supported |
+| KIP-1139 - Add support for OAuth jwt-bearer grant type | 4.1.0 (WIP) | Supported |
+
### Supported protocol versions
-"Kafka max" is the maximum ApiVersion supported in Apache Kafka 3.7.0, while
+"Kafka max" is the maximum ApiVersion supported in Apache Kafka 4.0.0, while
"librdkafka max" is the maximum ApiVersion supported in the latest
release of librdkafka.
-| ApiKey | Request name | Kafka max | librdkafka max |
-| ------- | ----------------------------- | ---------- | -------------- |
-| 0 | Produce | 10 | 10 |
-| 1 | Fetch | 16 | 16 |
-| 2 | ListOffsets | 8 | 7 |
-| 3 | Metadata | 12 | 12 |
-| 8 | OffsetCommit | 9 | 9 |
-| 9 | OffsetFetch | 9 | 9 |
-| 10 | FindCoordinator | 4 | 2 |
-| 11 | JoinGroup | 9 | 5 |
-| 12 | Heartbeat | 4 | 3 |
-| 13 | LeaveGroup | 5 | 1 |
-| 14 | SyncGroup | 5 | 3 |
-| 15 | DescribeGroups | 5 | 4 |
-| 16 | ListGroups | 4 | 4 |
-| 17 | SaslHandshake | 1 | 1 |
-| 18 | ApiVersions | 3 | 3 |
-| 19 | CreateTopics | 7 | 4 |
-| 20 | DeleteTopics | 6 | 1 |
-| 21 | DeleteRecords | 2 | 1 |
-| 22 | InitProducerId | 4 | 4 |
-| 23 | OffsetForLeaderEpoch | 4 | 2 |
-| 24 | AddPartitionsToTxn | 4 | 0 |
-| 25 | AddOffsetsToTxn | 3 | 0 |
-| 26 | EndTxn | 3 | 1 |
-| 28 | TxnOffsetCommit | 3 | 3 |
-| 29 | DescribeAcls | 3 | 1 |
-| 30 | CreateAcls | 3 | 1 |
-| 31 | DeleteAcls | 3 | 1 |
-| 32 | DescribeConfigs | 4 | 1 |
-| 33 | AlterConfigs | 2 | 2 |
-| 36 | SaslAuthenticate | 2 | 1 |
-| 37 | CreatePartitions | 3 | 0 |
-| 42 | DeleteGroups | 2 | 1 |
-| 44 | IncrementalAlterConfigs | 1 | 1 |
-| 47 | OffsetDelete | 0 | 0 |
-| 50 | DescribeUserScramCredentials | 0 | 0 |
-| 51 | AlterUserScramCredentials | 0 | 0 |
-| 68 | ConsumerGroupHeartbeat | 0 | 0 |
-| 71 | GetTelemetrySubscriptions | 0 | 0 |
-| 72 | PushTelemetry | 0 | 0 |
-
+| ApiKey | Request name | Kafka max | librdkafka max |
+| ------ | ---------------------------- | --------- | -------------- |
+| 0 | Produce | 12 | 10 |
+| 1 | Fetch | 17 | 16 |
+| 2 | ListOffsets | 10 | 7 |
+| 3 | Metadata | 13 | 13 |
+| 8 | OffsetCommit | 9 | 9 |
+| 9 | OffsetFetch | 9 | 9 |
+| 10 | FindCoordinator | 6 | 2 |
+| 11 | JoinGroup | 9 | 5 |
+| 12 | Heartbeat | 4 | 3 |
+| 13 | LeaveGroup | 5 | 1 |
+| 14 | SyncGroup | 5 | 3 |
+| 15 | DescribeGroups | 6 | 4 |
+| 16 | ListGroups | 5 | 4 |
+| 17 | SaslHandshake | 1 | 1 |
+| 18 | ApiVersions | 4 | 3 |
+| 19 | CreateTopics | 7 | 4 |
+| 20 | DeleteTopics | 6 | 1 |
+| 21 | DeleteRecords | 2 | 1 |
+| 22 | InitProducerId | 5 | 4 |
+| 23 | OffsetForLeaderEpoch | 4 | 2 |
+| 24 | AddPartitionsToTxn | 5 | 0 |
+| 25 | AddOffsetsToTxn | 4 | 0 |
+| 26 | EndTxn | 5 | 1 |
+| 28 | TxnOffsetCommit | 5 | 3 |
+| 29 | DescribeAcls | 3 | 1 |
+| 30 | CreateAcls | 3 | 1 |
+| 31 | DeleteAcls | 3 | 1 |
+| 32 | DescribeConfigs | 4 | 1 |
+| 33 | AlterConfigs | 2 | 2 |
+| 36 | SaslAuthenticate | 2 | 1 |
+| 37 | CreatePartitions | 3 | 0 |
+| 42 | DeleteGroups | 2 | 1 |
+| 43 | ElectLeaders | 2 | 2 |
+| 44 | IncrementalAlterConfigs | 1 | 1 |
+| 47 | OffsetDelete | 0 | 0 |
+| 50 | DescribeUserScramCredentials | 0 | 0 |
+| 51 | AlterUserScramCredentials | 0 | 0 |
+| 68 | ConsumerGroupHeartbeat | 1 | 1 |
+| 69 | ConsumerGroupDescribe | 1 | 0 |
+| 71 | GetTelemetrySubscriptions | 0 | 0 |
+| 72 | PushTelemetry | 0 | 0 |
+
+
# Recommendations for language binding developers
These recommendations are targeted for developers that wrap librdkafka
with their high-level languages, such as confluent-kafka-go or node-rdkafka.
+
## Expose the configuration interface pass-thru
librdkafka's string-based key=value configuration property interface controls
@@ -2129,6 +2434,7 @@ in a pass-through fashion without any pre-checking done by your binding code it
means that a simple upgrade of the underlying librdkafka library (but not your
bindings) will provide new features to the user.
+
## Error constants
The error constants, both the official (value >= 0) errors as well as the
@@ -2137,6 +2443,7 @@ To avoid hard-coding them to expose to your users, librdkafka provides an API
to extract the full list programmatically during runtime or for
code generation, see `rd_kafka_get_err_descs()`.
+
## Reporting client software name and version to broker
[KIP-511](https://cwiki.apache.org/confluence/display/KAFKA/KIP-511%3A+Collect+and+Expose+Client%27s+Name+and+Version+in+the+Brokers) introduces a means for a
@@ -2150,23 +2457,25 @@ librdkafka will send its name (`librdkafka`) and version (e.g., `v1.3.0`)
upon connect to a supporting broker.
To help distinguish high-level client bindings on top of librdkafka, a client
binding should configure the following two properties:
- * `client.software.name` - set to the binding name, e.g,
- `confluent-kafka-go` or `node-rdkafka`.
- * `client.software.version` - the version of the binding and the version
- of librdkafka, e.g., `v1.3.0-librdkafka-v1.3.0` or
- `1.2.0-librdkafka-v1.3.0`.
- It is **highly recommended** to include the librdkafka version in this
- version string.
+* `client.software.name` - set to the binding name, e.g,
+ `confluent-kafka-go` or `node-rdkafka`.
+* `client.software.version` - the version of the binding and the version
+ of librdkafka, e.g., `v1.3.0-librdkafka-v1.3.0` or
+ `1.2.0-librdkafka-v1.3.0`.
+ It is **highly recommended** to include the librdkafka version in this
+ version string.
These configuration properties are hidden (from CONFIGURATION.md et.al.) as
they should typically not be modified by the user.
+
## Documentation reuse
You are free to reuse the librdkafka API and CONFIGURATION documentation in
your project, but please do return any documentation improvements back to
librdkafka (file a github pull request).
+
## Community support
Community support is offered through GitHub Issues and Discussions.
diff --git a/Makefile b/Makefile
index 3188b84a2e..0d33e9e343 100755
--- a/Makefile
+++ b/Makefile
@@ -9,6 +9,7 @@ DOC_FILES+= LICENSE LICENSES.txt INTRODUCTION.md README.md \
PKGNAME?= librdkafka
VERSION?= $(shell python3 packaging/get_version.py src/rdkafka.h)
+SHELL = /bin/sh
# Jenkins CI integration
BUILD_NUMBER ?= 1
@@ -79,7 +80,7 @@ rpm: distclean
$(MAKE) -C packaging/rpm
LICENSES.txt: .PHONY
- @(for i in LICENSE LICENSE.*[^~] ; do (echo "$$i" ; echo "--------------------------------------------------------------" ; cat $$i ; echo "" ; echo "") ; done) > $@.tmp
+ @(for i in LICENSE LICENSE.*[!~] ; do (echo "$$i" ; echo "--------------------------------------------------------------" ; cat $$i ; echo "" ; echo "") ; done) > $@.tmp
@cmp $@ $@.tmp || mv -f $@.tmp $@ ; rm -f $@.tmp
diff --git a/examples/.gitignore b/examples/.gitignore
index 9b2c65a2f7..a2cd3d609a 100644
--- a/examples/.gitignore
+++ b/examples/.gitignore
@@ -21,4 +21,5 @@ list_consumer_group_offsets
alter_consumer_group_offsets
incremental_alter_configs
user_scram
-list_offsets
\ No newline at end of file
+list_offsets
+elect_leaders
\ No newline at end of file
diff --git a/examples/CMakeLists.txt b/examples/CMakeLists.txt
index 8c0079abee..91851d2cbf 100644
--- a/examples/CMakeLists.txt
+++ b/examples/CMakeLists.txt
@@ -62,6 +62,9 @@ target_link_libraries(describe_cluster PUBLIC rdkafka)
add_executable(list_offsets list_offsets.c ${win32_sources})
target_link_libraries(list_offsets PUBLIC rdkafka)
+add_executable(elect_leaders elect_leaders.c ${win32_sources})
+target_link_libraries(elect_leaders PUBLIC rdkafka)
+
# The targets below has Unix include dirs and do not compile on Windows.
if(NOT WIN32)
add_executable(rdkafka_example rdkafka_example.c)
diff --git a/examples/Makefile b/examples/Makefile
index f76702d02c..f8b6513540 100644
--- a/examples/Makefile
+++ b/examples/Makefile
@@ -13,6 +13,7 @@ EXAMPLES ?= rdkafka_example rdkafka_performance rdkafka_example_cpp \
incremental_alter_configs \
user_scram \
list_offsets \
+ elect_leaders \
misc
all: $(EXAMPLES)
@@ -153,6 +154,10 @@ list_offsets: ../src/librdkafka.a list_offsets.c
$(CC) $(CPPFLAGS) $(CFLAGS) $@.c -o $@ $(LDFLAGS) \
../src/librdkafka.a $(LIBS)
+elect_leaders: ../src/librdkafka.a elect_leaders.c
+ $(CC) $(CPPFLAGS) $(CFLAGS) $@.c -o $@ $(LDFLAGS) \
+ ../src/librdkafka.a $(LIBS)
+
misc: ../src/librdkafka.a misc.c
$(CC) $(CPPFLAGS) $(CFLAGS) $@.c -o $@ $(LDFLAGS) \
../src/librdkafka.a $(LIBS)
diff --git a/examples/consumer.c b/examples/consumer.c
index dad3efc43b..f621a9db35 100644
--- a/examples/consumer.c
+++ b/examples/consumer.c
@@ -41,7 +41,7 @@
/* Typical include path would be , but this program
* is builtin from within the librdkafka source tree and thus differs. */
-//#include
+// #include
#include "rdkafka.h"
@@ -77,8 +77,9 @@ int main(int argc, char **argv) {
char errstr[512]; /* librdkafka API error reporting buffer */
const char *brokers; /* Argument: broker list */
const char *groupid; /* Argument: Consumer group id */
- char **topics; /* Argument: list of topics to subscribe to */
- int topic_cnt; /* Number of topics to subscribe to */
+ const char *group_protocol;
+ char **topics; /* Argument: list of topics to subscribe to */
+ int topic_cnt; /* Number of topics to subscribe to */
rd_kafka_topic_partition_list_t *subscription; /* Subscribed topics */
int i;
@@ -88,15 +89,17 @@ int main(int argc, char **argv) {
if (argc < 4) {
fprintf(stderr,
"%% Usage: "
- "%s ..\n",
+ "%s "
+ "..\n",
argv[0]);
return 1;
}
- brokers = argv[1];
- groupid = argv[2];
- topics = &argv[3];
- topic_cnt = argc - 3;
+ brokers = argv[1];
+ groupid = argv[2];
+ group_protocol = argv[3];
+ topics = &argv[4];
+ topic_cnt = argc - 4;
/*
@@ -127,6 +130,13 @@ int main(int argc, char **argv) {
return 1;
}
+ if (rd_kafka_conf_set(conf, "group.protocol", group_protocol, errstr,
+ sizeof(errstr)) != RD_KAFKA_CONF_OK) {
+ fprintf(stderr, "%s\n", errstr);
+ rd_kafka_conf_destroy(conf);
+ return 1;
+ }
+
/* If there is no previously committed offset for a partition
* the auto.offset.reset strategy will be used to decide where
* in the partition to start fetching messages.
diff --git a/examples/describe_consumer_groups.c b/examples/describe_consumer_groups.c
index daacc1d021..e667eac7f1 100644
--- a/examples/describe_consumer_groups.c
+++ b/examples/describe_consumer_groups.c
@@ -164,15 +164,31 @@ print_group_member_info(const rd_kafka_MemberDescription_t *member) {
rd_kafka_MemberDescription_host(member));
const rd_kafka_MemberAssignment_t *assignment =
rd_kafka_MemberDescription_assignment(member);
- const rd_kafka_topic_partition_list_t *topic_partitions =
+ const rd_kafka_topic_partition_list_t *assigned_topic_partitions =
rd_kafka_MemberAssignment_partitions(assignment);
- if (!topic_partitions) {
+ const rd_kafka_MemberAssignment_t *target_assignment =
+ rd_kafka_MemberDescription_target_assignment(member);
+ const rd_kafka_topic_partition_list_t *target_topic_partitions =
+ target_assignment
+ ? rd_kafka_MemberAssignment_partitions(target_assignment)
+ : NULL;
+ if (!assigned_topic_partitions) {
printf(" No assignment\n");
- } else if (topic_partitions->cnt == 0) {
+ } else if (assigned_topic_partitions->cnt == 0) {
printf(" Empty assignment\n");
} else {
printf(" Assignment:\n");
- print_partition_list(stdout, topic_partitions, 0, " ");
+ print_partition_list(stdout, assigned_topic_partitions, 0,
+ " ");
+ }
+ if (!target_topic_partitions) {
+ printf(" No target assignment\n");
+ } else if (target_topic_partitions->cnt == 0) {
+ printf(" Empty target assignment\n");
+ } else {
+ printf(" Target assignment:\n");
+ print_partition_list(stdout, target_topic_partitions, 0,
+ " ");
}
}
@@ -194,6 +210,8 @@ static void print_group_info(const rd_kafka_ConsumerGroupDescription_t *group) {
rd_kafka_ConsumerGroupDescription_partition_assignor(group);
rd_kafka_consumer_group_state_t state =
rd_kafka_ConsumerGroupDescription_state(group);
+ rd_kafka_consumer_group_type_t type =
+ rd_kafka_ConsumerGroupDescription_type(group);
authorized_operations =
rd_kafka_ConsumerGroupDescription_authorized_operations(
group, &authorized_operations_cnt);
@@ -212,9 +230,10 @@ static void print_group_info(const rd_kafka_ConsumerGroupDescription_t *group) {
rd_kafka_Node_port(coordinator));
}
printf(
- "Group \"%s\", partition assignor \"%s\", "
- " state %s%s, with %" PRId32 " member(s)\n",
+ "Group \"%s\", partition assignor \"%s\", type \"%s\""
+ " state \"%s\"%s, with %" PRId32 " member(s)\n",
group_id, partition_assignor,
+ rd_kafka_consumer_group_type_name(type),
rd_kafka_consumer_group_state_name(state), coordinator_desc,
member_cnt);
for (j = 0; j < authorized_operations_cnt; j++) {
diff --git a/examples/elect_leaders.c b/examples/elect_leaders.c
new file mode 100644
index 0000000000..84015580ca
--- /dev/null
+++ b/examples/elect_leaders.c
@@ -0,0 +1,317 @@
+/*
+ * librdkafka - Apache Kafka C library
+ *
+ * Copyright (c) 2024, Confluent Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are met:
+ *
+ * 1. Redistributions of source code must retain the above copyright notice,
+ * this list of conditions and the following disclaimer.
+ * 2. Redistributions in binary form must reproduce the above copyright notice,
+ * this list of conditions and the following disclaimer in the documentation
+ * and/or other materials provided with the distribution.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+ * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+ * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ * ARE DISCLAIMED. IN NO EVENT SH THE COPYRIGHT OWNER OR CONTRIBUTORS BE
+ * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+ * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+ * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+ * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+ * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+ * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
+ * POSSIBILITY OF SUCH DAMAGE.
+ */
+
+/**
+ * Example utility that shows how to use Elect Leaders (AdminAPI)
+ * to trigger preffered or unclean elections for
+ * one or more topic partitions.
+ */
+
+#include
+#include
+#include
+#include
+
+#ifdef _WIN32
+#include "../win32/wingetopt.h"
+#else
+#include
+#endif
+
+/* Typical include path would be , but this program
+ * is builtin from within the librdkafka source tree and thus differs. */
+#include "rdkafka.h"
+
+const char *argv0;
+
+static rd_kafka_queue_t *queue = NULL; /** Admin result queue.
+ * This is a global so we can
+ * yield in stop() */
+static volatile sig_atomic_t run = 1;
+
+/**
+ * @brief Signal termination of program
+ */
+static void stop(int sig) {
+ if (!run) {
+ fprintf(stderr, "%% Forced termination\n");
+ exit(2);
+ }
+ run = 0;
+ if (queue)
+ rd_kafka_queue_yield(queue);
+}
+
+static void usage(const char *reason, ...) {
+
+ fprintf(stderr,
+ "Elect Leaders usage examples\n"
+ "\n"
+ "Usage: %s "
+ " ...\n"
+ "\n"
+ "Options:\n"
+ " -b Bootstrap server list to connect to.\n"
+ " -X Set librdkafka configuration property.\n"
+ " See CONFIGURATION.md for full list.\n"
+ " -d Enable librdkafka debugging (%s).\n"
+ "\n",
+ argv0, rd_kafka_get_debug_contexts());
+
+ if (reason) {
+ va_list ap;
+ char reasonbuf[512];
+
+ va_start(ap, reason);
+ vsnprintf(reasonbuf, sizeof(reasonbuf), reason, ap);
+ va_end(ap);
+
+ fprintf(stderr, "ERROR: %s\n", reasonbuf);
+ }
+
+ exit(reason ? 1 : 0);
+}
+
+
+#define fatal(...) \
+ do { \
+ fprintf(stderr, "ERROR: "); \
+ fprintf(stderr, __VA_ARGS__); \
+ fprintf(stderr, "\n"); \
+ exit(2); \
+ } while (0)
+
+
+/**
+ * @brief Set config property. Exit on failure.
+ */
+static void conf_set(rd_kafka_conf_t *conf, const char *name, const char *val) {
+ char errstr[512];
+
+ if (rd_kafka_conf_set(conf, name, val, errstr, sizeof(errstr)) !=
+ RD_KAFKA_CONF_OK)
+ fatal("Failed to set %s=%s: %s", name, val, errstr);
+}
+
+static void
+print_elect_leaders_result(const rd_kafka_ElectLeaders_result_t *result) {
+ const rd_kafka_topic_partition_result_t **results;
+ size_t results_cnt;
+ size_t i;
+
+ results = rd_kafka_ElectLeaders_result_partitions(result, &results_cnt);
+ printf("ElectLeaders response has %zu partition(s):\n", results_cnt);
+ for (i = 0; i < results_cnt; i++) {
+ const rd_kafka_topic_partition_t *partition =
+ rd_kafka_topic_partition_result_partition(results[i]);
+ const rd_kafka_error_t *err =
+ rd_kafka_topic_partition_result_error(results[i]);
+ if (rd_kafka_error_code(err)) {
+ printf("%% ElectLeaders failed for %s [%" PRId32
+ "] : %s\n",
+ partition->topic, partition->partition,
+ rd_kafka_error_string(err));
+ } else {
+ printf("%% ElectLeaders succeeded for %s [%" PRId32
+ "]\n",
+ partition->topic, partition->partition);
+ }
+ }
+}
+
+/**
+ * @brief Parse an integer or fail.
+ */
+int64_t parse_int(const char *what, const char *str) {
+ char *end;
+ unsigned long n = strtoull(str, &end, 0);
+
+ if (end != str + strlen(str)) {
+ fprintf(stderr, "%% Invalid input for %s: %s: not an integer\n",
+ what, str);
+ exit(1);
+ }
+
+ return (int64_t)n;
+}
+
+static void cmd_elect_leaders(rd_kafka_conf_t *conf, int argc, char **argv) {
+ rd_kafka_t *rk;
+ char errstr[512];
+ rd_kafka_AdminOptions_t *options;
+ rd_kafka_event_t *event = NULL;
+ rd_kafka_topic_partition_list_t *partitions = NULL;
+ rd_kafka_ElectionType_t election_type;
+ rd_kafka_ElectLeaders_t *elect_leaders;
+ int i;
+ int retval = 0;
+
+ if ((argc - 1) % 2 != 0) {
+ usage("Invalid number of arguments");
+ }
+
+ election_type = parse_int("election_type", argv[0]);
+
+ argc--;
+ argv++;
+ if (argc > 0) {
+ partitions = rd_kafka_topic_partition_list_new(argc / 2);
+ for (i = 0; i < argc; i += 2) {
+ rd_kafka_topic_partition_list_add(
+ partitions, argv[i],
+ parse_int("partition", argv[i + 1]));
+ }
+ }
+
+ elect_leaders = rd_kafka_ElectLeaders_new(election_type, partitions);
+
+ if (partitions) {
+ rd_kafka_topic_partition_list_destroy(partitions);
+ }
+
+ /*
+ * Create consumer instance
+ * NOTE: rd_kafka_new() takes ownership of the conf object
+ * and the application must not reference it again after
+ * this call.
+ */
+ rk = rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, sizeof(errstr));
+ if (!rk)
+ fatal("Failed to create new consumer: %s", errstr);
+
+ /*
+ * Elect Leaders
+ */
+ queue = rd_kafka_queue_new(rk);
+
+ /* Signal handler for clean shutdown */
+ signal(SIGINT, stop);
+
+
+ options = rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_ELECTLEADERS);
+
+ if (rd_kafka_AdminOptions_set_request_timeout(
+ options, 30 * 1000 /* 30s */, errstr, sizeof(errstr))) {
+ fprintf(stderr, "%% Failed to set timeout: %s\n", errstr);
+ goto exit;
+ }
+
+ if (rd_kafka_AdminOptions_set_operation_timeout(
+ options, 30 * 1000 /* 30s */, errstr, sizeof(errstr))) {
+ fprintf(stderr, "%% Failed to set operation timeout: %s\n",
+ errstr);
+ goto exit;
+ }
+
+ rd_kafka_ElectLeaders(rk, elect_leaders, options, queue);
+
+ rd_kafka_ElectLeaders_destroy(elect_leaders);
+ rd_kafka_AdminOptions_destroy(options);
+
+ /* Wait for results */
+ event = rd_kafka_queue_poll(queue, -1 /* indefinitely but limited by
+ * the request timeout set
+ * above */);
+
+ if (!event) {
+ /* User hit Ctrl-C,
+ * see yield call in stop() signal handler */
+ fprintf(stderr, "%% Cancelled by user\n");
+
+ } else if (rd_kafka_event_error(event)) {
+ rd_kafka_resp_err_t err = rd_kafka_event_error(event);
+ /* ElectLeaders request failed */
+ fprintf(stderr, "%% ElectLeaders failed[%" PRId32 "]: %s\n",
+ err, rd_kafka_event_error_string(event));
+ retval = 1;
+ goto exit;
+ } else {
+ /* ElectLeaders request succeeded */
+ const rd_kafka_ElectLeaders_result_t *result;
+ result = rd_kafka_event_ElectLeaders_result(event);
+ print_elect_leaders_result(result);
+ }
+
+
+exit:
+ if (event)
+ rd_kafka_event_destroy(event);
+
+ rd_kafka_queue_destroy(queue);
+ /* Destroy the client instance */
+ rd_kafka_destroy(rk);
+
+ exit(retval);
+}
+
+
+int main(int argc, char **argv) {
+ rd_kafka_conf_t *conf; /**< Client configuration object */
+ int opt;
+ argv0 = argv[0];
+
+ /*
+ * Create Kafka client configuration place-holder
+ */
+ conf = rd_kafka_conf_new();
+
+ /*
+ * Parse common options
+ */
+ while ((opt = getopt(argc, argv, "b:X:d:")) != -1) {
+ switch (opt) {
+ case 'b':
+ conf_set(conf, "bootstrap.servers", optarg);
+ break;
+
+ case 'X': {
+ char *name = optarg, *val;
+
+ if (!(val = strchr(name, '=')))
+ fatal("-X expects a name=value argument");
+
+ *val = '\0';
+ val++;
+
+ conf_set(conf, name, val);
+ break;
+ }
+
+ case 'd':
+ conf_set(conf, "debug", optarg);
+ break;
+
+ default:
+ usage("Unknown option %c", (char)opt);
+ }
+ }
+
+ cmd_elect_leaders(conf, argc - optind, &argv[optind]);
+
+ return 0;
+}
diff --git a/examples/incremental_alter_configs.c b/examples/incremental_alter_configs.c
index 40a16cf842..b63b414795 100644
--- a/examples/incremental_alter_configs.c
+++ b/examples/incremental_alter_configs.c
@@ -183,11 +183,10 @@ cmd_incremental_alter_configs(rd_kafka_conf_t *conf, int argc, char **argv) {
rd_kafka_ConfigResource_t *config;
rd_kafka_AlterConfigOpType_t op_type;
rd_kafka_ResourceType_t restype =
- !strcmp(restype_s, "TOPIC")
- ? RD_KAFKA_RESOURCE_TOPIC
- : !strcmp(restype_s, "BROKER")
- ? RD_KAFKA_RESOURCE_BROKER
- : RD_KAFKA_RESOURCE_UNKNOWN;
+ !strcmp(restype_s, "TOPIC") ? RD_KAFKA_RESOURCE_TOPIC
+ : !strcmp(restype_s, "BROKER") ? RD_KAFKA_RESOURCE_BROKER
+ : !strcmp(restype_s, "GROUP") ? RD_KAFKA_RESOURCE_GROUP
+ : RD_KAFKA_RESOURCE_UNKNOWN;
if (restype == RD_KAFKA_RESOURCE_UNKNOWN) {
usage("Invalid resource type: %s", restype_s);
diff --git a/examples/kafkatest_verifiable_client.cpp b/examples/kafkatest_verifiable_client.cpp
index bdb8607a33..1afd1347ed 100644
--- a/examples/kafkatest_verifiable_client.cpp
+++ b/examples/kafkatest_verifiable_client.cpp
@@ -194,14 +194,11 @@ static void errorString(const std::string &name,
const std::string &topic,
const std::string *key,
const std::string &value) {
- std::cout << "{ "
- << "\"name\": \"" << name << "\", "
- << "\"_time\": \"" << now() << "\", "
- << "\"message\": \"" << errmsg << "\", "
- << "\"topic\": \"" << topic << "\", "
- << "\"key\": \"" << (key ? *key : "NULL") << "\", "
- << "\"value\": \"" << value << "\" "
- << "}" << std::endl;
+ std::cout << "{ " << "\"name\": \"" << name << "\", " << "\"_time\": \""
+ << now() << "\", " << "\"message\": \"" << errmsg << "\", "
+ << "\"topic\": \"" << topic << "\", " << "\"key\": \""
+ << (key ? *key : "NULL") << "\", " << "\"value\": \"" << value
+ << "\" " << "}" << std::endl;
}
@@ -211,15 +208,12 @@ static void successString(const std::string &name,
int64_t offset,
const std::string *key,
const std::string &value) {
- std::cout << "{ "
- << "\"name\": \"" << name << "\", "
- << "\"_time\": \"" << now() << "\", "
- << "\"topic\": \"" << topic << "\", "
+ std::cout << "{ " << "\"name\": \"" << name << "\", " << "\"_time\": \""
+ << now() << "\", " << "\"topic\": \"" << topic << "\", "
<< "\"partition\": " << partition << ", "
- << "\"offset\": " << offset << ", "
- << "\"key\": \"" << (key ? *key : "NULL") << "\", "
- << "\"value\": \"" << value << "\" "
- << "}" << std::endl;
+ << "\"offset\": " << offset << ", " << "\"key\": \""
+ << (key ? *key : "NULL") << "\", " << "\"value\": \"" << value
+ << "\" " << "}" << std::endl;
}
@@ -231,15 +225,11 @@ static void offsetStatus(bool success,
const std::string &errstr) {
std::cout << "{ "
"\"name\": \"offsets_committed\", "
- << "\"success\": " << success << ", "
- << "\"offsets\": [ "
- << " { "
+ << "\"success\": " << success << ", " << "\"offsets\": [ " << " { "
<< " \"topic\": \"" << topic << "\", "
<< " \"partition\": " << partition << ", "
- << " \"offset\": " << (int)offset << ", "
- << " \"error\": \"" << errstr << "\" "
- << " } "
- << "] }" << std::endl;
+ << " \"offset\": " << (int)offset << ", " << " \"error\": \""
+ << errstr << "\" " << " } " << "] }" << std::endl;
}
#endif
@@ -341,8 +331,7 @@ static void report_records_consumed(int immediate) {
<< "\"count\": "
<< (state.consumer.consumedMessages -
state.consumer.consumedMessagesLastReported)
- << ", "
- << "\"partitions\": [ ";
+ << ", " << "\"partitions\": [ ";
for (std::map::iterator ii = assignments->begin();
ii != assignments->end(); ii++) {
@@ -352,8 +341,7 @@ static void report_records_consumed(int immediate) {
<< " \"topic\": \"" << a->topic << "\", "
<< " \"partition\": " << a->partition << ", "
<< " \"minOffset\": " << a->minOffset << ", "
- << " \"maxOffset\": " << a->maxOffset << " "
- << " } ";
+ << " \"maxOffset\": " << a->maxOffset << " " << " } ";
a->minOffset = -1;
}
@@ -378,8 +366,7 @@ class ExampleOffsetCommitCb : public RdKafka::OffsetCommitCb {
*/
report_records_consumed(1);
- std::cout << "{ "
- << "\"name\": \"offsets_committed\", "
+ std::cout << "{ " << "\"name\": \"offsets_committed\", "
<< "\"success\": " << (err ? "false" : "true") << ", "
<< "\"error\": \"" << (err ? RdKafka::err2str(err) : "") << "\", "
<< "\"_autocommit\": "
@@ -387,15 +374,14 @@ class ExampleOffsetCommitCb : public RdKafka::OffsetCommitCb {
<< "\"offsets\": [ ";
assert(offsets.size() > 0);
for (unsigned int i = 0; i < offsets.size(); i++) {
- std::cout << (i == 0 ? "" : ", ") << "{ "
- << " \"topic\": \"" << offsets[i]->topic() << "\", "
+ std::cout << (i == 0 ? "" : ", ") << "{ " << " \"topic\": \""
+ << offsets[i]->topic() << "\", "
<< " \"partition\": " << offsets[i]->partition() << ", "
<< " \"offset\": " << (int)offsets[i]->offset() << ", "
<< " \"error\": \""
<< (offsets[i]->err() ? RdKafka::err2str(offsets[i]->err())
: "")
- << "\" "
- << " }";
+ << "\" " << " }";
}
std::cout << " ] }" << std::endl;
}
@@ -427,8 +413,8 @@ static void do_commit(RdKafka::KafkaConsumer *consumer, int immediate) {
RdKafka::ErrorCode err;
err = consumer->commitSync(&ex_offset_commit_cb);
- std::cerr << now() << ": "
- << "sync commit returned " << RdKafka::err2str(err) << std::endl;
+ std::cerr << now() << ": " << "sync commit returned " << RdKafka::err2str(err)
+ << std::endl;
state.consumer.consumedMessagesAtLastCommit = state.consumer.consumedMessages;
}
@@ -526,8 +512,8 @@ class ExampleRebalanceCb : public RdKafka::RebalanceCb {
const std::vector &partitions) {
std::ostringstream out;
for (unsigned int i = 0; i < partitions.size(); i++)
- out << (i == 0 ? "" : ", ") << "{ "
- << " \"topic\": \"" << partitions[i]->topic() << "\", "
+ out << (i == 0 ? "" : ", ") << "{ " << " \"topic\": \""
+ << partitions[i]->topic() << "\", "
<< " \"partition\": " << partitions[i]->partition() << " }";
return out.str();
}
@@ -549,13 +535,11 @@ class ExampleRebalanceCb : public RdKafka::RebalanceCb {
consumer->unassign();
}
- std::cout << "{ "
- << "\"name\": \"partitions_"
+ std::cout << "{ " << "\"name\": \"partitions_"
<< (err == RdKafka::ERR__ASSIGN_PARTITIONS ? "assigned"
: "revoked")
- << "\", "
- << "\"partitions\": [ " << part_list_json(partitions) << "] }"
- << std::endl;
+ << "\", " << "\"partitions\": [ " << part_list_json(partitions)
+ << "] }" << std::endl;
}
};
diff --git a/examples/list_consumer_groups.c b/examples/list_consumer_groups.c
index 13656cd66d..060554af5f 100644
--- a/examples/list_consumer_groups.c
+++ b/examples/list_consumer_groups.c
@@ -73,7 +73,8 @@ static void usage(const char *reason, ...) {
fprintf(stderr,
"List groups usage examples\n"
"\n"
- "Usage: %s ...\n"
+ "Usage: %s [ ...] "
+ " [ ...]\n"
"\n"
"Options:\n"
" -b Bootstrap server list to connect to.\n"
@@ -145,12 +146,15 @@ static int print_groups_info(const rd_kafka_ListConsumerGroups_result_t *list) {
int is_simple_consumer_group =
rd_kafka_ConsumerGroupListing_is_simple_consumer_group(
group);
+ rd_kafka_consumer_group_type_t type =
+ rd_kafka_ConsumerGroupListing_type(group);
printf("Group \"%s\", is simple %" PRId32
", "
- "state %s",
+ "state %s, type %s",
group_id, is_simple_consumer_group,
- rd_kafka_consumer_group_state_name(state));
+ rd_kafka_consumer_group_state_name(state),
+ rd_kafka_consumer_group_type_name(type));
printf("\n");
}
for (i = 0; i < result_error_cnt; i++) {
@@ -184,24 +188,46 @@ int64_t parse_int(const char *what, const char *str) {
static void
cmd_list_consumer_groups(rd_kafka_conf_t *conf, int argc, char **argv) {
rd_kafka_t *rk;
- const char **states_str = NULL;
char errstr[512];
rd_kafka_AdminOptions_t *options;
rd_kafka_event_t *event = NULL;
rd_kafka_error_t *error = NULL;
int i;
- int retval = 0;
- int states_cnt = 0;
+ int retval = 0;
+ int states_cnt = 0;
+ int types_cnt = 0;
+ const int min_argc = 2;
+
rd_kafka_consumer_group_state_t *states;
+ rd_kafka_consumer_group_type_t *types;
+ /*
+ * Argument validation
+ */
+ if (argc < min_argc)
+ usage("Expected at least %d arguments", min_argc);
+ else {
+ states_cnt = parse_int("state count", argv[0]);
+ if (argc < states_cnt + 2) {
+ usage("Expected %d state code(s) after states count",
+ states_cnt);
+ }
- if (argc >= 1) {
- states_str = (const char **)&argv[0];
- states_cnt = argc;
+ types_cnt = parse_int("type count", argv[1 + states_cnt]);
+ if (argc < 1 + states_cnt + 1 + types_cnt) {
+ usage("Expected %d type(s) after type count",
+ types_cnt);
+ }
}
+
states = calloc(states_cnt, sizeof(rd_kafka_consumer_group_state_t));
for (i = 0; i < states_cnt; i++) {
- states[i] = parse_int("state code", states_str[i]);
+ states[i] = parse_int("state code", argv[i + 1]);
+ }
+
+ types = calloc(types_cnt, sizeof(rd_kafka_consumer_group_type_t));
+ for (i = 0; i < types_cnt; i++) {
+ types[i] = parse_int("type code", argv[i + states_cnt + 2]);
}
/*
@@ -235,10 +261,17 @@ cmd_list_consumer_groups(rd_kafka_conf_t *conf, int argc, char **argv) {
options, states, states_cnt))) {
fprintf(stderr, "%% Failed to set states: %s\n",
rd_kafka_error_string(error));
- rd_kafka_error_destroy(error);
goto exit;
}
free(states);
+ if ((error = rd_kafka_AdminOptions_set_match_consumer_group_types(
+ options, types, types_cnt))) {
+ fprintf(stderr, "%% Failed to set types: %s\n",
+ rd_kafka_error_string(error));
+ goto exit;
+ }
+ free(types);
+
rd_kafka_ListConsumerGroups(rk, options, queue);
rd_kafka_AdminOptions_destroy(options);
@@ -273,6 +306,8 @@ cmd_list_consumer_groups(rd_kafka_conf_t *conf, int argc, char **argv) {
exit:
+ if (error)
+ rd_kafka_error_destroy(error);
if (event)
rd_kafka_event_destroy(event);
rd_kafka_queue_destroy(queue);
diff --git a/examples/openssl_engine_example.cpp b/examples/openssl_engine_example.cpp
index 7279747176..07de03453f 100644
--- a/examples/openssl_engine_example.cpp
+++ b/examples/openssl_engine_example.cpp
@@ -74,10 +74,9 @@ class PrintingSSLVerifyCb : public RdKafka::SslCertificateVerifyCb {
const char *buf,
size_t size,
std::string &errstr) {
- std::cout << "ssl_cert_verify_cb :"
- << ": broker_name=" << broker_name << ", broker_id=" << broker_id
- << ", x509_error=" << *x509_error << ", depth=" << depth
- << ", buf size=" << size << std::endl;
+ std::cout << "ssl_cert_verify_cb :" << ": broker_name=" << broker_name
+ << ", broker_id=" << broker_id << ", x509_error=" << *x509_error
+ << ", depth=" << depth << ", buf size=" << size << std::endl;
return true;
}
diff --git a/examples/producer.cpp b/examples/producer.cpp
index 76560eb6be..ea8f97784a 100755
--- a/examples/producer.cpp
+++ b/examples/producer.cpp
@@ -133,8 +133,8 @@ int main(int argc, char **argv) {
/*
* Read messages from stdin and produce to broker.
*/
- std::cout << "% Type message value and hit enter "
- << "to produce message." << std::endl;
+ std::cout << "% Type message value and hit enter " << "to produce message."
+ << std::endl;
for (std::string line; run && std::getline(std::cin, line);) {
if (line.empty()) {
diff --git a/examples/user_scram.c b/examples/user_scram.c
index 95d6809b40..8b81737338 100644
--- a/examples/user_scram.c
+++ b/examples/user_scram.c
@@ -138,11 +138,10 @@ int64_t parse_int(const char *what, const char *str) {
}
rd_kafka_ScramMechanism_t parse_mechanism(const char *arg) {
- return !strcmp(arg, "SCRAM-SHA-256")
- ? RD_KAFKA_SCRAM_MECHANISM_SHA_256
- : !strcmp(arg, "SCRAM-SHA-512")
- ? RD_KAFKA_SCRAM_MECHANISM_SHA_512
- : RD_KAFKA_SCRAM_MECHANISM_UNKNOWN;
+ return !strcmp(arg, "SCRAM-SHA-256") ? RD_KAFKA_SCRAM_MECHANISM_SHA_256
+ : !strcmp(arg, "SCRAM-SHA-512")
+ ? RD_KAFKA_SCRAM_MECHANISM_SHA_512
+ : RD_KAFKA_SCRAM_MECHANISM_UNKNOWN;
}
static void print_descriptions(
diff --git a/examples/win_ssl_cert_store.cpp b/examples/win_ssl_cert_store.cpp
index 5158f961b1..c4a37c9c9f 100644
--- a/examples/win_ssl_cert_store.cpp
+++ b/examples/win_ssl_cert_store.cpp
@@ -211,9 +211,10 @@ class PrintingSSLVerifyCb : public RdKafka::SslCertificateVerifyCb {
CertGetNameStringA(ctx, CERT_NAME_FRIENDLY_DISPLAY_TYPE,
CERT_NAME_ISSUER_FLAG, NULL, issuer, sizeof(issuer));
- std::cerr << "Broker " << broker_name << " (" << broker_id << "): "
- << "certificate depth " << depth << ", X509 error " << *x509_error
- << ", subject " << subject << ", issuer " << issuer << std::endl;
+ std::cerr << "Broker " << broker_name << " (" << broker_id
+ << "): " << "certificate depth " << depth << ", X509 error "
+ << *x509_error << ", subject " << subject << ", issuer " << issuer
+ << std::endl;
if (ctx)
CertFreeCertificateContext(ctx);
diff --git a/mklove/modules/configure.libcurl b/mklove/modules/configure.libcurl
index 3a5f15b92a..174a4d8b52 100644
--- a/mklove/modules/configure.libcurl
+++ b/mklove/modules/configure.libcurl
@@ -45,8 +45,8 @@ void foo (void) {
function install_source {
local name=$1
local destdir=$2
- local ver=8.8.0
- local checksum="77c0e1cd35ab5b45b659645a93b46d660224d0024f1185e8a95cdb27ae3d787d"
+ local ver=8.10.1
+ local checksum="d15ebab765d793e2e96db090f0e172d127859d78ca6f6391d7eafecfd894bbc0"
echo "### Installing $name $ver from source to $destdir"
if [[ ! -f Makefile ]]; then
@@ -86,7 +86,7 @@ function install_source {
--disable-manual \
--disable-ldap{,s} \
--disable-libcurl-option \
- --without-{librtmp,libidn2,winidn,nghttp2,nghttp3,ngtcp2,quiche,brotli} &&
+ --without-{librtmp,libidn2,winidn,nghttp2,nghttp3,ngtcp2,quiche,brotli,libpsl} &&
time make CPPFLAGS="$CPPFLAGS" -j &&
make DESTDIR="${destdir}" prefix=/usr install
local ret=$?
diff --git a/mklove/modules/configure.libssl b/mklove/modules/configure.libssl
index 019e6c60b5..3ef5cab7d0 100644
--- a/mklove/modules/configure.libssl
+++ b/mklove/modules/configure.libssl
@@ -91,8 +91,8 @@ function manual_checks {
function libcrypto_install_source {
local name=$1
local destdir=$2
- local ver=3.0.13
- local checksum="88525753f79d3bec27d2fa7c66aa0b92b3aa9498dafd93d7cfa4b3780cdae313"
+ local ver=3.0.15
+ local checksum="23c666d0edf20f14249b3d8f0368acaee9ab585b09e1de82107c66e1f3ec9533"
local url=https://www.openssl.org/source/openssl-${ver}.tar.gz
local conf_args="--prefix=/usr --openssldir=/usr/lib/ssl no-shared no-zlib"
diff --git a/packaging/RELEASE.md b/packaging/RELEASE.md
index 36cf38198a..f94ddce85e 100644
--- a/packaging/RELEASE.md
+++ b/packaging/RELEASE.md
@@ -45,6 +45,14 @@ Error strings must not contain a trailing period.
will be a breaking API change.
+### Max Apache Kafka RPC versions
+
+Copy the first three columns of the table in `INTRODUCTION.md` to a temporary file
+and run the script
+`./packaging/tools/update_rpcs_max_versions.py `
+for getting the updated columns in stdout.
+
+
## Run regression tests
**Build tests:**
diff --git a/packaging/cp/README.md b/packaging/cp/README.md
index 422d8bb158..2959763897 100644
--- a/packaging/cp/README.md
+++ b/packaging/cp/README.md
@@ -4,10 +4,13 @@ This small set of scripts verifies the librdkafka packages that
are part of the Confluent Platform.
The base_url is the http S3 bucket path to the a PR job, or similar.
+Pass the platform (e.g. linux/arm64) and the expected librdkafka version too.
## How to use
- $ ./verify-packages.sh 7.6 https://packages.confluent.io
+```
+$ ./verify-packages.sh https://packages.confluent.io linux/amd64 2.8.0
+```
Requires docker and patience.
diff --git a/packaging/cp/check_features.c b/packaging/cp/check_features.c
index 4229402fd6..c123f5f163 100644
--- a/packaging/cp/check_features.c
+++ b/packaging/cp/check_features.c
@@ -15,9 +15,17 @@ int main(int argc, char **argv) {
printf("librdkafka %s (0x%x, define: 0x%x)\n", rd_kafka_version_str(),
rd_kafka_version(), RD_KAFKA_VERSION);
- if (argc > 1 && !(argc & 1)) {
- printf("Usage: %s [config.property config-value ..]\n",
- argv[0]);
+ if (argc < 2 || (argc > 2 && (argc & 1))) {
+ printf(
+ "Usage: %s [config.property config-value "
+ "..]\n",
+ argv[0]);
+ return 1;
+ }
+
+ if (!strstr(rd_kafka_version_str(), argv[1])) {
+ printf("ERROR: version string mismatch: expected %s, got %s\n",
+ argv[1], rd_kafka_version_str());
return 1;
}
@@ -46,7 +54,7 @@ int main(int argc, char **argv) {
printf("all expected features matched: %s\n", expected_features);
/* Apply config from argv key value pairs */
- for (i = 1; i + 1 < argc; i += 2) {
+ for (i = 2; i + 1 < argc; i += 2) {
printf("verifying config %s=%s\n", argv[i], argv[i + 1]);
if (rd_kafka_conf_set(conf, argv[i], argv[i + 1], errstr,
sizeof(errstr)) != RD_KAFKA_CONF_OK) {
diff --git a/packaging/cp/verify-deb.sh b/packaging/cp/verify-deb.sh
index e394627d89..7c642d1392 100755
--- a/packaging/cp/verify-deb.sh
+++ b/packaging/cp/verify-deb.sh
@@ -3,22 +3,22 @@
set -e
-cpver=$1
-base_url=$2
+base_url=$1
+version=$2
if [[ -z $base_url ]]; then
- echo "Usage: $0 "
+ echo "Usage: $0 "
exit 1
fi
apt-get update
apt-get install -y apt-transport-https wget gnupg2 lsb-release
-wget -qO - ${base_url}/deb/${cpver}/archive.key | apt-key add -
+wget -qO - ${base_url}/clients/deb/archive.key | apt-key add -
release=$(lsb_release -cs)
cat >/etc/apt/sources.list.d/Confluent.list < "
+if [[ -z $base_url || -z $version ]]; then
+ echo "Usage: $0 "
echo ""
- echo " is the Major.minor version of CP, e.g., 5.3"
echo " is the release base bucket URL"
+ echo " is platform to verify (e.g. linux/amd64)"
+ echo " is the expected version"
exit 1
fi
thisdir="$( cd "$(dirname "$0")" ; pwd -P )"
-echo "#### Verifying RPM packages ####"
-docker run -v $thisdir:/v rockylinux:8 /v/verify-rpm.sh $cpver $base_url
-docker run -v $thisdir:/v rockylinux:9 /v/verify-rpm.sh $cpver $base_url
-rpm_status=$?
-
-echo "#### Verifying Debian packages ####"
-docker run -v $thisdir:/v debian:10 /v/verify-deb.sh $cpver $base_url
-docker run -v $thisdir:/v debian:11 /v/verify-deb.sh $cpver $base_url
-docker run -v $thisdir:/v debian:12 /v/verify-deb.sh $cpver $base_url
-docker run -v $thisdir:/v ubuntu:20.04 /v/verify-deb.sh $cpver $base_url
-docker run -v $thisdir:/v ubuntu:22.04 /v/verify-deb.sh $cpver $base_url
-deb_status=$?
+verify_debian() {
+ local version=$2
+ docker run -v $thisdir:/v $1 /v/verify-deb.sh $base_url $version
+ deb_status=$?
+ if [[ $deb_status == 0 ]]; then
+ echo "SUCCESS: Debian based $1 $2 packages verified"
+ else
+ echo "ERROR: Debian based $1 $2 package verification failed"
+ exit 1
+ fi
+}
+verify_rpm() {
+ local version=$2
+ docker run -v $thisdir:/v $1 /v/verify-rpm.sh $base_url $version
+ rpm_status=$?
+ if [[ $rpm_status == 0 ]]; then
+ echo "SUCCESS: RPM $1 $2 packages verified"
+ else
+ echo "ERROR: RPM $1 $2 package verification failed"
+ exit 1
+ fi
+}
-if [[ $rpm_status == 0 ]]; then
- echo "SUCCESS: RPM packages verified"
-else
- echo "ERROR: RPM package verification failed"
-fi
+verify_rpm_distros() {
+ local platform=$1
+ local version=$2
+ echo "#### Verifying RPM packages for $platform ####"
+ # Last RHEL 8 version is 2.4.0
+ verify_rpm rockylinux:8 "2.4.0"
+ verify_rpm rockylinux:9 $version
+}
-if [[ $deb_status == 0 ]]; then
- echo "SUCCESS: Debian packages verified"
-else
- echo "ERROR: Debian package verification failed"
-fi
+verify_debian_distros() {
+ local platform=$1
+ local version=$2
+ echo "#### Verifying Debian packages for $platform ####"
+ verify_debian debian:11 $version
+ verify_debian debian:12 $version
+ verify_debian ubuntu:20.04 $version
+ verify_debian ubuntu:22.04 $version
+ verify_debian ubuntu:24.04 $version
+}
-if [[ $deb_status != 0 || $rpm_status != 0 ]]; then
- exit 1
-fi
+verify_distros() {
+ verify_rpm_distros $1 $2
+ verify_debian_distros $1 $2
+}
+verify_distros $platform $version
diff --git a/packaging/cp/verify-rpm.sh b/packaging/cp/verify-rpm.sh
index d199524139..6824353843 100755
--- a/packaging/cp/verify-rpm.sh
+++ b/packaging/cp/verify-rpm.sh
@@ -3,22 +3,15 @@
set -e
-cpver=$1
-base_url=$2
+base_url=$1
+version=$2
if [[ -z $base_url ]]; then
- echo "Usage: $0 "
+ echo "Usage: $0 "
exit 1
fi
cat >/etc/yum.repos.d/Confluent.repo < [...]"
+ exit 1
+fi
+
+IMAGE=$1
+SCRIPT_DIR=$(dirname "$0")
+ENTRYPOINT=${2}
+REST=${@:3}
+
+if [ $(which cache) ]; then
+ cache restore ${IMAGE}.tar
+fi
+if [ ! -f ./${IMAGE}.tar ]; then
+ docker build -f $SCRIPT_DIR/Dockerfile -t $IMAGE --build-arg UID=$UID .
+ docker save $IMAGE -o ./${IMAGE}.tar
+
+ if [ $(which cache) ]; then
+ cache store ${IMAGE}.tar ./${IMAGE}.tar
+ fi
+else
+ docker load -i ./${IMAGE}.tar
+fi
+
+docker run --rm --entrypoint $ENTRYPOINT \
+ -v .:/librdkafka -w /librdkafka -e CI -u $UID:$UID ${IMAGE} ${REST}
diff --git a/packaging/tools/run-integration-tests.sh b/packaging/tools/run-integration-tests.sh
new file mode 100755
index 0000000000..d700d2d58e
--- /dev/null
+++ b/packaging/tools/run-integration-tests.sh
@@ -0,0 +1,31 @@
+#!/bin/bash
+set -e
+
+if [ "$#" -lt 2 ]; then
+ echo "Usage: $0 []"
+ echo "Example: $0 3.9.0 7.9.0"
+ exit 1
+fi
+
+KAFKA_VERSION=$1
+CP_VERSION=$2
+if [ -n "$3" ]; then
+ export TEST_CONSUMER_GROUP_PROTOCOL=$3
+fi
+
+source /home/user/venv/bin/activate
+./configure --install-deps --enable-werror --enable-devel
+./packaging/tools/rdutcoverage.sh
+make copyright-check
+make -j all examples check
+echo "Verifying that CONFIGURATION.md does not have manual changes"
+git diff --exit-code CONFIGURATION.md
+examples/rdkafka_example -X builtin.features
+ldd src/librdkafka.so.1
+ldd src-cpp/librdkafka++.so.1
+make -j -C tests build
+make -C tests run_local_quick
+DESTDIR="$PWD/dest" make install
+(cd tests && python3 -m trivup.clusters.KafkaCluster --kraft \
+ --version ${KAFKA_VERSION} \
+ --cpversion ${CP_VERSION} --cmd 'make quick')
diff --git a/packaging/tools/run-style-check.sh b/packaging/tools/run-style-check.sh
new file mode 100755
index 0000000000..f739d320cb
--- /dev/null
+++ b/packaging/tools/run-style-check.sh
@@ -0,0 +1,4 @@
+#!/bin/bash
+set -e
+source /home/user/venv/bin/activate
+make style-check
diff --git a/packaging/tools/style-format.sh b/packaging/tools/style-format.sh
index c59ecbe6a1..ebd61ad10f 100755
--- a/packaging/tools/style-format.sh
+++ b/packaging/tools/style-format.sh
@@ -3,11 +3,12 @@
# Check or apply/fix the project coding style to all files passed as arguments.
# Uses clang-format for C/C++ and flake8 for Python.
#
-# Requires clang-format version 10 (apt install clang-format-10).
-#
-CLANG_FORMAT=${CLANG_FORMAT:-clang-format}
+
+# Requires clang-format version 18 (apt install clang-format-18).
+CLANG_FORMAT=${CLANG_FORMAT:-clang-format-18}
+CLANG_FORMAT_REQUIRED_VERSION=18
set -e
@@ -27,8 +28,8 @@ else
fi
clang_format_version=$(${CLANG_FORMAT} --version | sed -Ee 's/.*version ([[:digit:]]+)\.[[:digit:]]+\.[[:digit:]]+.*/\1/')
-if [[ $clang_format_version != "10" ]] ; then
- echo "$0: clang-format version 10, '$clang_format_version' detected"
+if [[ $clang_format_version != "$CLANG_FORMAT_REQUIRED_VERSION" ]] ; then
+ echo "$0: clang-format version ${CLANG_FORMAT_REQUIRED_VERSION}, '$clang_format_version' detected"
exit 1
fi
diff --git a/packaging/tools/update_rpcs_max_versions.py b/packaging/tools/update_rpcs_max_versions.py
new file mode 100755
index 0000000000..4b98a0f51c
--- /dev/null
+++ b/packaging/tools/update_rpcs_max_versions.py
@@ -0,0 +1,100 @@
+#!/usr/bin/env python
+import json
+import sys
+import re
+
+# This script updates the Apache Kafka RPCs max versions.
+# It reads the input from `input_file`, which should be a table
+# looking like the table in `INTRODUCTION.md`.
+# Should look like this (without the first space after the comment):
+# | 0 | Produce | 12 | 10 |
+# | 1 | Fetch | 17 | 16 |
+# | 2 | ListOffsets | 10 | 7 |
+# | 3 | Metadata | 13 | 12 |
+# | 8 | OffsetCommit | 9 | 9 |
+# | 9 | OffsetFetch | 9 | 9 |
+# | 10 | FindCoordinator | 6 | 2 |
+# | 11 | JoinGroup | 9 | 5 |
+# | 12 | Heartbeat | 4 | 3 |
+# | 13 | LeaveGroup | 5 | 1 |
+# | 14 | SyncGroup | 5 | 3 |
+# | 15 | DescribeGroups | 6 | 4 |
+# | 16 | ListGroups | 5 | 4 |
+# | 17 | SaslHandshake | 1 | 1 |
+# | 18 | ApiVersions | 4 | 3 |
+# | 19 | CreateTopics | 7 | 4 |
+# | 20 | DeleteTopics | 6 | 1 |
+# | 21 | DeleteRecords | 2 | 1 |
+# | 22 | InitProducerId | 5 | 4 |
+# | 23 | OffsetForLeaderEpoch | 4 | 2 |
+# | 24 | AddPartitionsToTxn | 5 | 0 |
+# | 25 | AddOffsetsToTxn | 4 | 0 |
+# | 26 | EndTxn | 5 | 1 |
+# | 28 | TxnOffsetCommit | 5 | 3 |
+# | 29 | DescribeAcls | 3 | 1 |
+# | 30 | CreateAcls | 3 | 1 |
+# | 31 | DeleteAcls | 3 | 1 |
+# | 32 | DescribeConfigs | 4 | 1 |
+# | 33 | AlterConfigs | 2 | 2 |
+# | 36 | SaslAuthenticate | 2 | 1 |
+# | 37 | CreatePartitions | 3 | 0 |
+# | 42 | DeleteGroups | 2 | 1 |
+# | 43 | ElectLeaders | 2 | 2 |
+# | 44 | IncrementalAlterConfigs | 1 | 1 |
+# | 47 | OffsetDelete | 0 | 0 |
+# | 50 | DescribeUserScramCredentials | 0 | 0 |
+# | 51 | AlterUserScramCredentials | 0 | 0 |
+# | 68 | ConsumerGroupHeartbeat | 1 | 1 |
+# | 69 | ConsumerGroupDescribe | 1 | 0 |
+# | 71 | GetTelemetrySubscriptions | 0 | 0 |
+# | 72 | PushTelemetry | 0 | 0 |
+#
+# Output will be the same with max versions updated
+# Should pass Apache Kafka root folder as first argument and the input file
+# as second argument.
+ak_folder = sys.argv[1]
+input_file = sys.argv[2]
+
+if len(sys.argv) != 3:
+ print("Usage: python3 update_rpcs_max_versions.py "
+ "")
+ sys.exit(1)
+
+with open(input_file, 'r') as input:
+ lines = input.readlines()
+ max_first_column = 0
+ max_second_column = 0
+ max_third_column = 0
+ apis = []
+ for line in lines:
+ line = re.sub('^\\s*\\|\\s*', '', line)
+ pipe_char = line.find('|')
+ max_first_column = max(max_first_column, pipe_char)
+ api_num = int(line[0:pipe_char])
+ line = line[pipe_char + 1:]
+ line = re.sub('^\\s*', '', line)
+ pipe_char = line.find('|')
+ max_second_column = max(max_second_column, pipe_char)
+ api = line[0:pipe_char].strip()
+ line = line[pipe_char + 1:].lstrip()
+ pipe_char = line.find('|')
+ max_third_column = max(max_third_column, pipe_char)
+ rest = line[pipe_char + 1:].strip()
+ apis.append((api_num, api, rest))
+
+ for api_num, api, rest in apis:
+ with open(f'{ak_folder}/clients/src/main/resources/common/message/'
+ f'{api}Request.json',
+ 'r') as f:
+ text = f.readlines()
+ text = "".join([line for line in text
+ if '#' not in line and '//' not in line])
+ json_object = json.loads(text)
+ max_version = json_object["validVersions"].split("-")[-1]
+ print('| ', end='')
+ print(str(api_num).ljust(max_first_column), end='')
+ print('| ', end='')
+ print(api.ljust(max_second_column), end='')
+ print('| ', end='')
+ print(str(max_version).ljust(max_third_column) + '|', end='')
+ print(f' {rest}')
diff --git a/service.yml b/service.yml
index b15226a30c..9c8da7969b 100644
--- a/service.yml
+++ b/service.yml
@@ -16,3 +16,157 @@ semaphore:
- /semaphore.*/
- /dev_.*/
- /feature\/.*/
+ tasks:
+ - name: verify-linux-packages
+ branch: master
+ scheduled: false
+ pipeline_file: .semaphore/verify-linux-packages.yml
+ parameters:
+ - name: LIBRDKAFKA_VERSION
+ required: true
+ description: librdkafka version to verify
+ - name: run-all-tests
+ branch: master
+ scheduled: false
+ pipeline_file: .semaphore/run-all-tests.yml
+ parameters:
+ - name: GIT_REF
+ required: false
+ description: "librdkafka git ref to test."
+
+ - name: TEST_KAFKA_GIT_REF
+ required: true
+ default_value: ""
+ description: "Kafka version to test against. When a valid version, the corresponding
+ package is downloaded. [/][@] is also supported
+ to clone from GitHub and build from source."
+
+ - name: TEST_CP_VERSION
+ required: false
+ description: "CP version to test against"
+
+ - name: TEST_KAFKA_VERSION
+ required: false
+ description: "Kafka version to assume in this branch. If empty (default)
+ uses TEST_KAFKA_GIT_REF when a valid version, otherwise 9.9.9."
+
+ - name: TEST_KRAFT
+ required: true
+ default_value: "True"
+ description: "Run a KRaft cluster."
+ options:
+ - "True"
+ - "False"
+
+ - name: TEST_TYPE
+ required: true
+ default_value: "plaintext,ssl"
+ description: "Run PLAINTEXT or SSL tests or both. Defaults to both."
+ options:
+ - "plaintext,ssl"
+ - "plaintext"
+ - "ssl"
+
+ - name: TEST_SSL_INTERMEDIATE_CA
+ required: false
+ description: "Run SSL tests with an intermediate CA certificate."
+ options:
+ - "True"
+ - "False"
+
+ - name: TEST_SASL
+ required: false
+ description: "SASL mechanism to use."
+ options:
+ - ""
+ - "PLAIN"
+ - "SCRAM-SHA-256"
+ - "SCRAM-SHA-512"
+ - "GSSAPI"
+ - "OAUTHBEARER"
+
+ - name: TEST_QUICK
+ required: true
+ default_value: "False"
+ description: "Run only quick tests."
+ options:
+ - "True"
+ - "False"
+
+ - name: TEST_LOCAL
+ required: true
+ default_value: "All"
+ description: "Run only local, only non-local or all tests."
+ options:
+ - "All"
+ - "Local"
+ - "Non-local"
+
+ - name: TEST_PARALLEL
+ required: false
+ description: "Number of parallel tests to run. Empty means default value."
+
+ - name: TEST_MODE
+ required: false
+ description: "Test mode to use. Empty means default value."
+
+ - name: TESTS
+ required: false
+ description: "Run specified tests only. Empty means run all tests.
+ Can be a comma-separated list of test numbers or ranges,
+ even overlapping. Example:
+ 1,3,9-11,10-15,17-,-10"
+
+ - name: TESTS_BATCH_SIZE
+ required: false
+ description: "Run tests in batches of this size.
+ Empty means no batches, run all tests at once."
+
+ - name: TEST_ITERATIONS
+ required: false
+ description: "Run specified tests this many times.
+ Empty means run tests once."
+
+ - name: TEST_ASSERT
+ required: true
+ default_value: "False"
+ description: "Assert on failures"
+ options:
+ - "True"
+ - "False"
+
+ - name: TEST_DEBUG
+ required: false
+ description: "Enable debug output for tests."
+
+ - name: TEST_LEVEL
+ required: false
+ description: "Test verbosity level. Default is 2. Increase to 3 or 4 for more output."
+
+ - name: TEST_CONF
+ required: false
+ description: "Additional broker configuration as a JSON array. Example:
+ [group.coordinator.rebalance.protocols=classic,consumer]"
+
+ - name: TEST_TRIVUP_PARAMETERS
+ required: false
+ description: "Additional trivup parameters."
+
+ - name: TEST_RUNNER_PARAMETERS
+ required: false
+ description: "Additional test runner parameters."
+
+ - name: TEST_ENV_VARIABLES
+ required: false
+ description: "Additional environment variables to set for tests.
+ Example: A=1,B=2,C=3"
+
+ - name: TEST_ARCHES
+ required: true
+ default_value: "x86_64,aarch64"
+ description: "Test on these architectures: x86_64, aarch64 or both"
+ options:
+ - "x86_64,aarch64"
+ - "x86_64"
+ - "aarch64"
+
diff --git a/src-cpp/rdkafkacpp.h b/src-cpp/rdkafkacpp.h
index eb04afa1f5..c23a93d718 100644
--- a/src-cpp/rdkafkacpp.h
+++ b/src-cpp/rdkafkacpp.h
@@ -112,7 +112,7 @@ namespace RdKafka {
* @remark This value should only be used during compile time,
* for runtime checks of version use RdKafka::version()
*/
-#define RD_KAFKA_VERSION 0x020500ff
+#define RD_KAFKA_VERSION 0x020c01ff
/**
* @brief Returns the librdkafka version as integer.
@@ -327,6 +327,11 @@ enum ErrorCode {
ERR__AUTO_OFFSET_RESET = -140,
/** Partition log truncation detected */
ERR__LOG_TRUNCATION = -139,
+ /** A different record in the batch was invalid
+ * and this message failed persisting. */
+ ERR__INVALID_DIFFERENT_RECORD = -138,
+ /** Broker is going away but client isn't terminating */
+ ERR__DESTROY_BROKER = -137,
/** End internal error codes */
ERR__END = -100,
@@ -544,7 +549,28 @@ enum ErrorCode {
/** Unable to update finalized features due to server error */
ERR_FEATURE_UPDATE_FAILED = 96,
/** Request principal deserialization failed during forwarding */
- ERR_PRINCIPAL_DESERIALIZATION_FAILURE = 97
+ ERR_PRINCIPAL_DESERIALIZATION_FAILURE = 97,
+ /** Unknown Topic Id */
+ ERR_UNKNOWN_TOPIC_ID = 100,
+ /** The member epoch is fenced by the group coordinator */
+ ERR_FENCED_MEMBER_EPOCH = 110,
+ /** The instance ID is still used by another member in the
+ * consumer group */
+ ERR_UNRELEASED_INSTANCE_ID = 111,
+ /** The assignor or its version range is not supported by the consumer
+ * group */
+ ERR_UNSUPPORTED_ASSIGNOR = 112,
+ /** The member epoch is stale */
+ ERR_STALE_MEMBER_EPOCH = 113,
+ /** Client sent a push telemetry request with an invalid or outdated
+ * subscription ID. */
+ ERR_UNKNOWN_SUBSCRIPTION_ID = 117,
+ /** Client sent a push telemetry request larger than the maximum size
+ * the broker will accept. */
+ ERR_TELEMETRY_TOO_LARGE = 118,
+ /** Client metadata is stale,
+ * client should rebootstrap to obtain new metadata. */
+ ERR_REBOOTSTRAP_REQUIRED = 129
};
diff --git a/src/rd.h b/src/rd.h
index 559f37d45e..b24d370faa 100644
--- a/src/rd.h
+++ b/src/rd.h
@@ -53,6 +53,7 @@
#include
#include
#include
+#include
#include "tinycthread.h"
#include "rdsysqueue.h"
@@ -220,12 +221,18 @@ static RD_INLINE RD_UNUSED char *rd_strndup(const char *s, size_t len) {
/* Round/align X upwards to STRIDE, which must be power of 2. */
-#define RD_ROUNDUP(X, STRIDE) (((X) + ((STRIDE)-1)) & ~(STRIDE - 1))
+#define RD_ROUNDUP(X, STRIDE) (((X) + ((STRIDE) - 1)) & ~(STRIDE - 1))
#define RD_ARRAY_SIZE(A) (sizeof((A)) / sizeof(*(A)))
#define RD_ARRAYSIZE(A) RD_ARRAY_SIZE(A)
#define RD_SIZEOF(TYPE, MEMBER) sizeof(((TYPE *)NULL)->MEMBER)
#define RD_OFFSETOF(TYPE, MEMBER) ((size_t) & (((TYPE *)NULL)->MEMBER))
+/** Array foreach */
+#define RD_ARRAY_FOREACH_INDEX(ELEM, ARRAY, INDEX) \
+ for ((INDEX = 0, (ELEM) = (ARRAY)[INDEX]); \
+ INDEX < RD_ARRAY_SIZE(ARRAY); \
+ (ELEM) = \
+ (++INDEX < RD_ARRAY_SIZE(ARRAY) ? (ARRAY)[INDEX] : (ELEM)))
/**
* Returns the 'I'th array element from static sized array 'A'
@@ -427,7 +434,7 @@ static RD_INLINE RD_UNUSED int rd_refcnt_get(rd_refcnt_t *R) {
#define RD_INTERFACE_CALL(i, name, ...) (i->name(i->opaque, __VA_ARGS__))
-#define RD_CEIL_INTEGER_DIVISION(X, DEN) (((X) + ((DEN)-1)) / (DEN))
+#define RD_CEIL_INTEGER_DIVISION(X, DEN) (((X) + ((DEN) - 1)) / (DEN))
/**
* @brief Utility types to hold memory,size tuple.
@@ -438,4 +445,140 @@ typedef struct rd_chariov_s {
size_t size;
} rd_chariov_t;
+/**
+ * @brief Read the file at \p file_path in binary mode and return its contents.
+ * The returned buffer is NULL-terminated,
+ * the size parameter will contain the actual file size.
+ *
+ * @param file_path Path to the file to read.
+ * @param size Pointer to store the file size (optional).
+ * @param max_size Maximum file size to read (0 for no limit) (optional).
+ *
+ * @returns Newly allocated buffer containing the file contents.
+ * NULL on error (file not found, too large, etc).
+ *
+ * @remark The returned pointer ownership is transferred to the caller.
+ *
+ * @locality Any thread
+ */
+static RD_INLINE RD_UNUSED char *
+rd_file_read(const char *file_path, size_t *size, size_t max_size) {
+ FILE *file;
+ char *buf = NULL;
+ size_t file_size;
+ size_t read_size;
+ if (!size)
+ size = &read_size;
+
+#ifndef _WIN32
+ file = fopen(file_path, "rb");
+#else
+ file = NULL;
+ errno = fopen_s(&file, file_path, "rb");
+#endif
+ if (!file)
+ return NULL;
+
+ if (fseek(file, 0, SEEK_END) != 0)
+ goto err;
+
+ file_size = (size_t)ftell(file);
+ if (file_size < 0)
+ goto err;
+
+ if (fseek(file, 0, SEEK_SET) != 0)
+ goto err;
+
+ /* Check if file is too large */
+ if (max_size > 0 && file_size > max_size)
+ goto err;
+
+ /* Allocate buffer with extra byte for NULL terminator */
+ buf = (char *)rd_malloc(file_size + 1);
+ read_size = fread(buf, 1, file_size, file);
+
+ if (read_size != file_size)
+ goto err;
+
+ /* NULL terminate the buffer */
+ buf[file_size] = '\0';
+ *size = file_size;
+ fclose(file);
+ return buf;
+err:
+ fclose(file);
+ if (buf)
+ rd_free(buf);
+ return NULL;
+}
+
+static RD_INLINE RD_UNUSED FILE *
+rd_file_mkstemp(const char *prefix,
+ const char *mode,
+ char *tempfile_path_out,
+ size_t tempfile_path_out_size) {
+ FILE *tempfile;
+
+#ifdef _WIN32
+ char tempfolder_path[MAX_PATH];
+ char tempfile_path[MAX_PATH];
+ if (!GetTempPathA(MAX_PATH, tempfolder_path))
+ return NULL; /* Failed to get temp folder path */
+
+
+ if (!GetTempFileNameA(tempfolder_path, "TMP", 1, tempfile_path))
+ return NULL; /* Failed to create temp file name */
+
+ tempfile = fopen(tempfile_path, mode);
+#else
+ int tempfile_fd;
+ char tempfile_path[512];
+ rd_snprintf(tempfile_path, sizeof(tempfile_path), "/tmp/%sXXXXXX",
+ prefix);
+ tempfile_fd = mkstemp(tempfile_path);
+ if (tempfile_fd < 0)
+ return NULL;
+
+ tempfile = fdopen(tempfile_fd, mode);
+#endif
+
+ if (!tempfile)
+ return NULL;
+
+ if (tempfile_path_out)
+ rd_snprintf(tempfile_path_out, tempfile_path_out_size, "%s",
+ tempfile_path);
+ return tempfile;
+}
+
+/**
+ * @brief Retrive stat for a \p path .
+ *
+ * @param path Path to the file or directory.
+ * @param is_dir Pointer to store if the \p path is a directory (optional).
+ *
+ * @return `rd_true` if the path exists.
+ */
+static RD_INLINE RD_UNUSED rd_bool_t rd_file_stat(const char *path,
+ rd_bool_t *is_dir) {
+#ifdef _WIN32
+ struct _stat st;
+ if (_stat(path, &st) == 0) {
+ if (is_dir)
+ *is_dir = st.st_mode & S_IFDIR;
+ return rd_true;
+ }
+#else
+ struct stat st;
+ if (stat(path, &st) == 0) {
+ if (is_dir)
+ *is_dir = S_ISDIR(st.st_mode);
+ return rd_true;
+ }
+#endif
+ if (is_dir)
+ *is_dir = rd_false;
+ return rd_false;
+}
+
#endif /* _RD_H_ */
diff --git a/src/rdaddr.h b/src/rdaddr.h
index 7e86a549a8..09105fdf14 100644
--- a/src/rdaddr.h
+++ b/src/rdaddr.h
@@ -54,14 +54,13 @@ typedef union {
#define sinx_family in.sin_family
#define sinx_addr in.sin_addr
#define RD_SOCKADDR_INX_LEN(sinx) \
- ((sinx)->sinx_family == AF_INET \
- ? sizeof(struct sockaddr_in) \
- : (sinx)->sinx_family == AF_INET6 ? sizeof(struct sockaddr_in6) \
- : sizeof(rd_sockaddr_inx_t))
+ ((sinx)->sinx_family == AF_INET ? sizeof(struct sockaddr_in) \
+ : (sinx)->sinx_family == AF_INET6 ? sizeof(struct sockaddr_in6) \
+ : sizeof(rd_sockaddr_inx_t))
#define RD_SOCKADDR_INX_PORT(sinx) \
- ((sinx)->sinx_family == AF_INET \
- ? (sinx)->in.sin_port \
- : (sinx)->sinx_family == AF_INET6 ? (sinx)->in6.sin6_port : 0)
+ ((sinx)->sinx_family == AF_INET ? (sinx)->in.sin_port \
+ : (sinx)->sinx_family == AF_INET6 ? (sinx)->in6.sin6_port \
+ : 0)
#define RD_SOCKADDR_INX_PORT_SET(sinx, port) \
do { \
diff --git a/src/rdatomic.h b/src/rdatomic.h
index 4b97dd7d08..9194a440c2 100644
--- a/src/rdatomic.h
+++ b/src/rdatomic.h
@@ -111,6 +111,10 @@ static RD_INLINE int32_t RD_UNUSED rd_atomic32_get(rd_atomic32_t *ra) {
#endif
}
+/**
+ * @brief Set the atomic value to \p v and return the previous value to
+ * that can be used to determine if the value was changed.
+ */
static RD_INLINE int32_t RD_UNUSED rd_atomic32_set(rd_atomic32_t *ra,
int32_t v) {
#ifdef _WIN32
@@ -118,15 +122,14 @@ static RD_INLINE int32_t RD_UNUSED rd_atomic32_set(rd_atomic32_t *ra,
#elif !HAVE_ATOMICS_32
int32_t r;
mtx_lock(&ra->lock);
- r = ra->val = v;
+ r = rd->val;
+ ra->val = v;
mtx_unlock(&ra->lock);
return r;
#elif HAVE_ATOMICS_32_ATOMIC
- __atomic_store_n(&ra->val, v, __ATOMIC_SEQ_CST);
- return v;
+ return __atomic_exchange_n(&ra->val, v, __ATOMIC_SEQ_CST);
#elif HAVE_ATOMICS_32_SYNC
- (void)__sync_lock_test_and_set(&ra->val, v);
- return v;
+ return __sync_lock_test_and_set(&ra->val, v);
#else
return ra->val = v; // FIXME
#endif
@@ -200,7 +203,10 @@ static RD_INLINE int64_t RD_UNUSED rd_atomic64_get(rd_atomic64_t *ra) {
#endif
}
-
+/**
+ * @brief Set the atomic value to \p v and return the previous value to
+ * that can be used to determine if the value was changed.
+ */
static RD_INLINE int64_t RD_UNUSED rd_atomic64_set(rd_atomic64_t *ra,
int64_t v) {
#ifdef _WIN32
@@ -208,16 +214,14 @@ static RD_INLINE int64_t RD_UNUSED rd_atomic64_set(rd_atomic64_t *ra,
#elif !HAVE_ATOMICS_64
int64_t r;
mtx_lock(&ra->lock);
- ra->val = v;
r = ra->val;
+ ra->val = v;
mtx_unlock(&ra->lock);
return r;
#elif HAVE_ATOMICS_64_ATOMIC
- __atomic_store_n(&ra->val, v, __ATOMIC_SEQ_CST);
- return v;
+ return __atomic_exchange_n(&ra->val, v, __ATOMIC_SEQ_CST);
#elif HAVE_ATOMICS_64_SYNC
- (void)__sync_lock_test_and_set(&ra->val, v);
- return v;
+ return __sync_lock_test_and_set(&ra->val, v);
#else
return ra->val = v; // FIXME
#endif
diff --git a/src/rdavg.h b/src/rdavg.h
index 55469e2466..c187aa91fc 100644
--- a/src/rdavg.h
+++ b/src/rdavg.h
@@ -45,8 +45,9 @@ typedef struct rd_avg_s {
} ra_v;
mtx_t ra_lock;
int ra_enabled;
- enum { RD_AVG_GAUGE,
- RD_AVG_COUNTER,
+ enum {
+ RD_AVG_GAUGE,
+ RD_AVG_COUNTER,
} ra_type;
#if WITH_HDRHISTOGRAM
rd_hdr_histogram_t *ra_hdr;
diff --git a/src/rdbase64.c b/src/rdbase64.c
index aaf2fb138e..7d87650903 100644
--- a/src/rdbase64.c
+++ b/src/rdbase64.c
@@ -32,7 +32,7 @@
#include
#else
-#define conv_bin2ascii(a, table) ((table)[(a)&0x3f])
+#define conv_bin2ascii(a, table) ((table)[(a) & 0x3f])
static const unsigned char data_bin2ascii[65] =
"ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/";
@@ -119,6 +119,37 @@ char *rd_base64_encode_str(const rd_chariov_t *in) {
return out.ptr;
}
+/**
+ * @brief Base64 encode binary input \p in and return a newly allocated,
+ * base64-encoded string with URL-safe characters.
+ * @returns a newly allocated, base64-encoded string or NULL in case of some
+ * issue with the conversion or the conversion is not supported.
+ *
+ * @remark Returned string must be freed after use.
+ */
+char *rd_base64_encode_str_urlsafe(const rd_chariov_t *in) {
+ rd_chariov_t out;
+ char *p;
+ rd_base64_encode(in, &out);
+
+ /* Replace + with - and / with _ */
+ for (p = out.ptr; *p; p++) {
+ if (*p == '+')
+ *p = '-';
+ else if (*p == '/')
+ *p = '_';
+ }
+
+ /* Remove padding '=' characters */
+ int newlen = strlen(out.ptr);
+ while (newlen > 0 && out.ptr[newlen - 1] == '=') {
+ out.ptr[newlen - 1] = '\0';
+ newlen--;
+ }
+
+ out.size = newlen;
+ return out.ptr;
+}
/**
* @brief Base64 decode input string \p in. Ignores leading and trailing
@@ -166,4 +197,4 @@ int rd_base64_decode(const rd_chariov_t *in, rd_chariov_t *out) {
#else
return -2;
#endif
-}
\ No newline at end of file
+}
diff --git a/src/rdbase64.h b/src/rdbase64.h
index fd9e7a209f..1fb1288548 100644
--- a/src/rdbase64.h
+++ b/src/rdbase64.h
@@ -36,6 +36,8 @@ void rd_base64_encode(const rd_chariov_t *in, rd_chariov_t *out);
char *rd_base64_encode_str(const rd_chariov_t *in);
+char *rd_base64_encode_str_urlsafe(const rd_chariov_t *in);
+
int rd_base64_decode(const rd_chariov_t *in, rd_chariov_t *out);
#endif /* _RDBASE64_H_ */
\ No newline at end of file
diff --git a/src/rdendian.h b/src/rdendian.h
index 8a1c4148ce..55e6f2803c 100644
--- a/src/rdendian.h
+++ b/src/rdendian.h
@@ -125,16 +125,17 @@
#define be32toh(x) (x)
#define be16toh(x) (x)
#define le32toh(x) \
- ((((x)&0xff) << 24) | (((x)&0xff00) << 8) | (((x)&0xff0000) >> 8) | \
- (((x)&0xff000000) >> 24))
+ ((((x) & 0xff) << 24) | (((x) & 0xff00) << 8) | \
+ (((x) & 0xff0000) >> 8) | (((x) & 0xff000000) >> 24))
#define le64toh(x) \
- ((((x)&0x00000000000000ffL) << 56) | \
- (((x)&0x000000000000ff00L) << 40) | \
- (((x)&0x0000000000ff0000L) << 24) | \
- (((x)&0x00000000ff000000L) << 8) | (((x)&0x000000ff00000000L) >> 8) | \
- (((x)&0x0000ff0000000000L) >> 24) | \
- (((x)&0x00ff000000000000L) >> 40) | \
- (((x)&0xff00000000000000L) >> 56))
+ ((((x) & 0x00000000000000ffL) << 56) | \
+ (((x) & 0x000000000000ff00L) << 40) | \
+ (((x) & 0x0000000000ff0000L) << 24) | \
+ (((x) & 0x00000000ff000000L) << 8) | \
+ (((x) & 0x000000ff00000000L) >> 8) | \
+ (((x) & 0x0000ff0000000000L) >> 24) | \
+ (((x) & 0x00ff000000000000L) >> 40) | \
+ (((x) & 0xff00000000000000L) >> 56))
#else
#include
#endif
diff --git a/src/rdhttp.c b/src/rdhttp.c
index cea2d1c97d..d30ef2a386 100644
--- a/src/rdhttp.c
+++ b/src/rdhttp.c
@@ -40,6 +40,10 @@
#include
#include "rdhttp.h"
+#if WITH_SSL
+#include "rdkafka_ssl.h"
+#endif
+
/** Maximum response size, increase as necessary. */
#define RD_HTTP_RESPONSE_SIZE_MAX 1024 * 1024 * 500 /* 500kb */
@@ -128,8 +132,171 @@ rd_http_req_write_cb(char *ptr, size_t size, size_t nmemb, void *userdata) {
return nmemb;
}
-rd_http_error_t *rd_http_req_init(rd_http_req_t *hreq, const char *url) {
+#if WITH_SSL
+/**
+ * @brief Callback function for setting up the SSL_CTX for HTTPS requests.
+ *
+ * This function sets the default CA paths for the SSL_CTX, and if that fails,
+ * it attempts to probe and set a default CA location. If `probe` is forced
+ * it skips the default CA paths and directly probes for CA certificates.
+ *
+ * On Windows, it attempts to load CA root certificates from the
+ * configured Windows certificate stores before falling back to the default.
+ *
+ * @return `CURLE_OK` on success, or `CURLE_SSL_CACERT_BADFILE` on failure.
+ */
+static CURLcode
+rd_http_ssl_ctx_function(CURL *curl, void *sslctx, void *userptr) {
+ SSL_CTX *ctx = (SSL_CTX *)sslctx;
+ rd_kafka_t *rk = (rd_kafka_t *)userptr;
+ int r = -1;
+ rd_bool_t force_probe =
+ !rd_strcmp(rk->rk_conf.https.ca_location, "probe");
+ rd_bool_t use_probe = force_probe;
+
+#if WITH_STATIC_LIB_libcrypto
+ /* We fallback to `probe` when statically linked. */
+ use_probe = rd_true;
+#endif
+
+#ifdef _WIN32
+ /* Attempt to load CA root certificates from the
+ * configured Windows certificate stores. */
+ r = rd_kafka_ssl_win_load_cert_stores(rk, "https", ctx,
+ rk->rk_conf.ssl.ca_cert_stores);
+ if (r == 0) {
+ rd_kafka_log(rk, LOG_NOTICE, "CERTSTORE",
+ "No CA certificates loaded for `https` from "
+ "Windows certificate stores: "
+ "falling back to default OpenSSL CA paths");
+ r = -1;
+ } else if (r == -1)
+ rd_kafka_log(rk, LOG_NOTICE, "CERTSTORE",
+ "Failed to load CA certificates for `https` from "
+ "Windows certificate stores: "
+ "falling back to default OpenSSL CA paths");
+
+ if (r != -1) {
+ rd_kafka_dbg(rk, SECURITY, "SSL",
+ "Successfully loaded CA certificates for `https` "
+ "from Windows certificate stores");
+ return CURLE_OK; /* Success, CA certs loaded on Windows */
+ }
+#endif
+
+ if (!force_probe) {
+ /* Previous default behavior: use predefined paths set when
+ * building OpenSSL. */
+ char errstr[512];
+ r = SSL_CTX_set_default_verify_paths(ctx);
+ if (r == 1) {
+ rd_kafka_dbg(rk, SECURITY, "SSL",
+ "SSL_CTX_set_default_verify_paths() "
+ "for `https` "
+ "succeeded");
+ return CURLE_OK; /* Success */
+ }
+
+ /* Read error and clear the error stack. */
+ rd_kafka_ssl_error0(rk, NULL, "https", errstr, sizeof(errstr));
+ rd_kafka_dbg(rk, SECURITY, "SSL",
+ "SSL_CTX_set_default_verify_paths() "
+ "for `https` "
+ "failed: %s",
+ errstr);
+ }
+
+ if (use_probe) {
+ /* We asked for probing or we're using
+ * a statically linked version of OpenSSL. */
+
+ r = rd_kafka_ssl_probe_and_set_default_ca_location(rk, "https",
+ ctx);
+ if (r == 0)
+ return CURLE_OK;
+ }
+
+ return CURLE_SSL_CACERT_BADFILE;
+}
+
+static void rd_http_ssl_configure(rd_kafka_t *rk, CURL *hreq_curl) {
+ rd_bool_t force_probe =
+ !rd_strcmp(rk->rk_conf.https.ca_location, "probe");
+
+ if (!force_probe && rk->rk_conf.https.ca_location) {
+ rd_bool_t is_dir;
+ rd_kafka_dbg(rk, SECURITY, "SSL",
+ "Setting `https` CA certs from "
+ "configured location: %s",
+ rk->rk_conf.https.ca_location);
+ if (rd_file_stat(rk->rk_conf.https.ca_location, &is_dir)) {
+ if (is_dir) {
+ curl_easy_setopt(hreq_curl, CURLOPT_CAPATH,
+ rk->rk_conf.https.ca_location);
+ curl_easy_setopt(hreq_curl, CURLOPT_CAINFO,
+ NULL);
+ } else {
+ curl_easy_setopt(hreq_curl, CURLOPT_CAPATH,
+ NULL);
+ curl_easy_setopt(hreq_curl, CURLOPT_CAINFO,
+ rk->rk_conf.https.ca_location);
+ }
+ } else {
+ /* Path doesn't exist, don't set any trusted
+ * certificate. */
+ curl_easy_setopt(hreq_curl, CURLOPT_CAINFO, NULL);
+ curl_easy_setopt(hreq_curl, CURLOPT_CAPATH, NULL);
+ }
+ } else if (!force_probe && rk->rk_conf.https.ca_pem) {
+#if CURL_AT_LEAST_VERSION(7, 77, 0)
+ struct curl_blob ca_blob = {
+ .data = rk->rk_conf.https.ca_pem,
+ .len = strlen(rk->rk_conf.https.ca_pem),
+ .flags = CURL_BLOB_COPY};
+ rd_kafka_dbg(rk, SECURITY, "SSL",
+ "Setting `https` CA certs from "
+ "configured PEM string");
+ curl_easy_setopt(hreq_curl, CURLOPT_CAINFO_BLOB, &ca_blob);
+#endif
+ /* Only the blob should be set, no default paths. */
+ curl_easy_setopt(hreq_curl, CURLOPT_CAINFO, NULL);
+ curl_easy_setopt(hreq_curl, CURLOPT_CAPATH, NULL);
+ } else {
+ curl_easy_setopt(hreq_curl, CURLOPT_SSL_CTX_FUNCTION,
+ rd_http_ssl_ctx_function);
+ curl_easy_setopt(hreq_curl, CURLOPT_SSL_CTX_DATA, rk);
+ }
+}
+#endif
+
+/**
+ * @brief Check if the error returned from HTTP(S) is temporary or not.
+ *
+ * @returns If the \p error_code is temporary, return rd_true,
+ * otherwise return rd_false.
+ *
+ * @locality Any thread.
+ * @locks None.
+ * @locks_acquired None.
+ */
+static rd_bool_t rd_http_is_failure_temporary(int error_code) {
+ switch (error_code) {
+ case 408: /**< Request timeout */
+ case 425: /**< Too early */
+ case 429: /**< Too many requests */
+ case 500: /**< Internal server error */
+ case 502: /**< Bad gateway */
+ case 503: /**< Service unavailable */
+ case 504: /**< Gateway timeout */
+ return rd_true;
+
+ default:
+ return rd_false;
+ }
+}
+rd_http_error_t *
+rd_http_req_init(rd_kafka_t *rk, rd_http_req_t *hreq, const char *url) {
memset(hreq, 0, sizeof(*hreq));
hreq->hreq_curl = curl_easy_init();
@@ -139,8 +306,15 @@ rd_http_error_t *rd_http_req_init(rd_http_req_t *hreq, const char *url) {
hreq->hreq_buf = rd_buf_new(1, 1024);
curl_easy_setopt(hreq->hreq_curl, CURLOPT_URL, url);
+#if CURL_AT_LEAST_VERSION(7, 85, 0)
+ curl_easy_setopt(hreq->hreq_curl, CURLOPT_PROTOCOLS_STR, "http,https");
+#else
+ /* As of 06/10/2025 Debian 10 and CentOS Stream 9 ship with
+ * older CURL versions, remove this condition once they're not supported
+ * anymore. */
curl_easy_setopt(hreq->hreq_curl, CURLOPT_PROTOCOLS,
CURLPROTO_HTTP | CURLPROTO_HTTPS);
+#endif
curl_easy_setopt(hreq->hreq_curl, CURLOPT_MAXREDIRS, 16);
curl_easy_setopt(hreq->hreq_curl, CURLOPT_TIMEOUT, 30);
curl_easy_setopt(hreq->hreq_curl, CURLOPT_ERRORBUFFER,
@@ -150,6 +324,10 @@ rd_http_error_t *rd_http_req_init(rd_http_req_t *hreq, const char *url) {
rd_http_req_write_cb);
curl_easy_setopt(hreq->hreq_curl, CURLOPT_WRITEDATA, (void *)hreq);
+#if WITH_SSL
+ rd_http_ssl_configure(rk, hreq->hreq_curl);
+#endif
+
return NULL;
}
@@ -191,35 +369,113 @@ const char *rd_http_req_get_content_type(rd_http_req_t *hreq) {
/**
* @brief Perform a blocking HTTP(S) request to \p url.
+ * Retries the request \p retries times with linear backoff.
+ * Interval of \p retry_ms milliseconds is used between retries.
+ *
+ * @param url The URL to perform the request to.
+ * @param headers_array Array of HTTP(S) headers to set, each element
+ * is a string in the form "key: value"
+ * @param headers_array_cnt Number of elements in \p headers_array.
+ * @param timeout_s Timeout in seconds for the request, 0 means default
+ * `rd_http_req_init()` timeout.
+ * @param retries Number of retries to perform on failure.
+ * @param retry_ms Milliseconds to wait between retries.
+ * @param rbufp (out) Pointer to a buffer that will be filled with the response.
+ * @param content_type (out, optional) Pointer to a string that will be filled
+ * with the content type of the response, if not NULL.
+ * @param response_code (out, optional) Pointer to an integer that will be
+ * filled with the HTTP response code, if not NULL.
*
- * Returns the response (even if there's a HTTP error code returned)
- * in \p *rbufp.
+ * @return Returns NULL on success (HTTP response code < 400), or an error
+ * object on transport or HTTP error.
*
- * Returns NULL on success (HTTP response code < 400), or an error
- * object on transport or HTTP error - this error object must be destroyed
- * by calling rd_http_error_destroy(). In case of HTTP error the \p *rbufp
- * may be filled with the error response.
+ * @remark Returned error object, when non-NULL, must be destroyed
+ * by calling rd_http_error_destroy().
+ *
+ * @locality Any thread.
+ * @locks None.
+ * @locks_acquired None.
*/
-rd_http_error_t *rd_http_get(const char *url, rd_buf_t **rbufp) {
+rd_http_error_t *rd_http_get(rd_kafka_t *rk,
+ const char *url,
+ char **headers_array,
+ size_t headers_array_cnt,
+ int timeout_s,
+ int retries,
+ int retry_ms,
+ rd_buf_t **rbufp,
+ char **content_type,
+ int *response_code) {
rd_http_req_t hreq;
- rd_http_error_t *herr;
+ rd_http_error_t *herr = NULL;
+ struct curl_slist *headers = NULL;
+ char *header;
+ int i;
+ size_t len, j;
*rbufp = NULL;
+ if (content_type)
+ *content_type = NULL;
+ if (response_code)
+ *response_code = -1;
- herr = rd_http_req_init(&hreq, url);
+ herr = rd_http_req_init(rk, &hreq, url);
if (unlikely(herr != NULL))
return herr;
- herr = rd_http_req_perform_sync(&hreq);
- if (herr) {
- rd_http_req_destroy(&hreq);
- return herr;
+ for (j = 0; j < headers_array_cnt; j++) {
+ header = headers_array[j];
+ if (header && *header)
+ headers = curl_slist_append(headers, header);
+ }
+ curl_easy_setopt(hreq.hreq_curl, CURLOPT_HTTPHEADER, headers);
+ if (timeout_s > 0)
+ curl_easy_setopt(hreq.hreq_curl, CURLOPT_TIMEOUT, timeout_s);
+
+ for (i = 0; i <= retries; i++) {
+ if (rd_kafka_terminating(rk)) {
+ herr = rd_http_error_new(-1, "Terminating");
+ goto done;
+ }
+
+ herr = rd_http_req_perform_sync(&hreq);
+ len = rd_buf_len(hreq.hreq_buf);
+
+ if (!herr) {
+ if (len > 0)
+ break; /* Success */
+ /* Empty response */
+ goto done;
+ }
+
+ /* Retry if HTTP(S) request returns temporary error and there
+ * are remaining retries, else fail. */
+ if (i == retries || !rd_http_is_failure_temporary(herr->code)) {
+ goto done;
+ }
+
+ /* Retry */
+ rd_http_error_destroy(herr);
+ rd_usleep(retry_ms * 1000 * (i + 1), &rk->rk_terminate);
}
*rbufp = hreq.hreq_buf;
hreq.hreq_buf = NULL;
- return NULL;
+ if (content_type) {
+ const char *ct = rd_http_req_get_content_type(&hreq);
+ if (ct && *ct)
+ *content_type = rd_strdup(ct);
+ else
+ *content_type = NULL;
+ }
+ if (response_code)
+ *response_code = hreq.hreq_code;
+
+done:
+ RD_IF_FREE(headers, curl_slist_free_all);
+ rd_http_req_destroy(&hreq);
+ return herr;
}
@@ -257,30 +513,6 @@ rd_http_error_t *rd_http_parse_json(rd_http_req_t *hreq, cJSON **jsonp) {
}
-/**
- * @brief Check if the error returned from HTTP(S) is temporary or not.
- *
- * @returns If the \p error_code is temporary, return rd_true,
- * otherwise return rd_false.
- *
- * @locality Any thread.
- */
-static rd_bool_t rd_http_is_failure_temporary(int error_code) {
- switch (error_code) {
- case 408: /**< Request timeout */
- case 425: /**< Too early */
- case 500: /**< Internal server error */
- case 502: /**< Bad gateway */
- case 503: /**< Service unavailable */
- case 504: /**< Gateway timeout */
- return rd_true;
-
- default:
- return rd_false;
- }
-}
-
-
/**
* @brief Perform a blocking HTTP(S) request to \p url with
* HTTP(S) headers and data with \p timeout_s.
@@ -309,7 +541,7 @@ rd_http_error_t *rd_http_post_expect_json(rd_kafka_t *rk,
size_t len;
const char *content_type;
- herr = rd_http_req_init(&hreq, url);
+ herr = rd_http_req_init(rk, &hreq, url);
if (unlikely(herr != NULL))
return herr;
@@ -367,6 +599,39 @@ rd_http_error_t *rd_http_post_expect_json(rd_kafka_t *rk,
return herr;
}
+/**
+ * @brief Append \p params to \p url, taking care of existing query parameters
+ * and hash fragments. \p params must be already URL encoded.
+ *
+ * @returns A newly allocated string with the appended parameters or NULL
+ * on error.
+ */
+char *rd_http_get_params_append(const char *url, const char *params) {
+ char *new_url;
+ CURLU *u = curl_url();
+ CURLUcode rc = curl_url_set(u, CURLUPART_URL, url, 0);
+ if (rc != CURLUE_OK)
+ goto err;
+
+ rc = curl_url_set(u, CURLUPART_QUERY, params, CURLU_APPENDQUERY);
+ if (rc != CURLUE_OK)
+ goto err;
+
+ rc = curl_url_set(u, CURLUPART_FRAGMENT, NULL,
+ 0); // remove hash fragment
+ if (rc != CURLUE_OK)
+ goto err;
+
+ rc = curl_url_get(u, CURLUPART_URL, &new_url, 0);
+ if (rc != CURLUE_OK)
+ goto err;
+
+ curl_url_cleanup(u);
+ return new_url;
+err:
+ curl_url_cleanup(u);
+ return NULL;
+}
/**
* @brief Same as rd_http_get() but requires a JSON response.
@@ -374,66 +639,51 @@ rd_http_error_t *rd_http_post_expect_json(rd_kafka_t *rk,
*
* Same error semantics as rd_http_get().
*/
-rd_http_error_t *rd_http_get_json(const char *url, cJSON **jsonp) {
- rd_http_req_t hreq;
+rd_http_error_t *rd_http_get_json(rd_kafka_t *rk,
+ const char *url,
+ char **headers_array,
+ size_t headers_array_cnt,
+ int timeout_s,
+ int retries,
+ int retry_ms,
+ cJSON **jsonp) {
rd_http_error_t *herr;
- rd_slice_t slice;
- size_t len;
- const char *content_type;
- char *raw_json;
- const char *end;
+ int response_code;
+ char *content_type;
+ rd_buf_t *rbuf;
+ char **headers_array_new =
+ rd_calloc(headers_array_cnt + 1, sizeof(*headers_array_new));
+ rd_http_req_t hreq;
*jsonp = NULL;
- herr = rd_http_req_init(&hreq, url);
- if (unlikely(herr != NULL))
- return herr;
+ memcpy(headers_array_new, headers_array,
+ headers_array_cnt * sizeof(*headers_array_new));
+ headers_array_new[headers_array_cnt++] = "Accept: application/json";
- // FIXME: send Accept: json.. header?
+ herr = rd_http_get(rk, url, headers_array_new, headers_array_cnt,
+ timeout_s, retries, retry_ms, &rbuf, &content_type,
+ &response_code);
+ rd_free(headers_array_new);
- herr = rd_http_req_perform_sync(&hreq);
- len = rd_buf_len(hreq.hreq_buf);
- if (herr && len == 0) {
- rd_http_req_destroy(&hreq);
+ if (unlikely(herr != NULL))
return herr;
- }
-
- if (len == 0) {
- /* Empty response: create empty JSON object */
- *jsonp = cJSON_CreateObject();
- rd_http_req_destroy(&hreq);
- return NULL;
- }
-
- content_type = rd_http_req_get_content_type(&hreq);
if (!content_type || rd_strncasecmp(content_type, "application/json",
strlen("application/json"))) {
- if (!herr)
- herr = rd_http_error_new(
- hreq.hreq_code, "Response is not JSON encoded: %s",
- content_type ? content_type : "(n/a)");
- rd_http_req_destroy(&hreq);
+ herr = rd_http_error_new(response_code,
+ "Response is not JSON encoded: %s",
+ content_type ? content_type : "(n/a)");
+ RD_IF_FREE(rbuf, rd_buf_destroy_free);
+ RD_IF_FREE(content_type, rd_free);
return herr;
}
- /* cJSON requires the entire input to parse in contiguous memory. */
- rd_slice_init_full(&slice, hreq.hreq_buf);
- raw_json = rd_malloc(len + 1);
- rd_slice_read(&slice, raw_json, len);
- raw_json[len] = '\0';
-
- /* Parse JSON */
- end = NULL;
- *jsonp = cJSON_ParseWithOpts(raw_json, &end, 0);
- if (!*jsonp && !herr)
- herr = rd_http_error_new(hreq.hreq_code,
- "Failed to parse JSON response "
- "at %" PRIusz "/%" PRIusz,
- (size_t)(end - raw_json), len);
-
- rd_free(raw_json);
- rd_http_req_destroy(&hreq);
+ hreq.hreq_buf = rbuf;
+ hreq.hreq_code = response_code;
+ herr = rd_http_parse_json(&hreq, jsonp);
+ RD_IF_FREE(rbuf, rd_buf_destroy_free);
+ RD_IF_FREE(content_type, rd_free);
return herr;
}
@@ -444,35 +694,28 @@ void rd_http_global_init(void) {
}
-/**
- * @brief Unittest. Requires a (local) webserver to be set with env var
- * RD_UT_HTTP_URL=http://localhost:1234/some-path
- *
- * This server must return a JSON object or array containing at least one
- * object on the main URL with a 2xx response code,
- * and 4xx response on $RD_UT_HTTP_URL/error (with whatever type of body).
- */
-
-int unittest_http(void) {
+int unittest_http_get(void) {
const char *base_url = rd_getenv("RD_UT_HTTP_URL", NULL);
char *error_url;
size_t error_url_size;
cJSON *json, *jval;
rd_http_error_t *herr;
rd_bool_t empty;
+ rd_kafka_t *rk;
if (!base_url || !*base_url)
RD_UT_SKIP("RD_UT_HTTP_URL environment variable not set");
RD_UT_BEGIN();
+ rk = rd_calloc(1, sizeof(*rk));
error_url_size = strlen(base_url) + strlen("/error") + 1;
error_url = rd_alloca(error_url_size);
rd_snprintf(error_url, error_url_size, "%s/error", base_url);
/* Try the base url first, parse its JSON and extract a key-value. */
json = NULL;
- herr = rd_http_get_json(base_url, &json);
+ herr = rd_http_get_json(rk, base_url, NULL, 0, 5, 1, 1000, &json);
RD_UT_ASSERT(!herr, "Expected get_json(%s) to succeed, got: %s",
base_url, herr->errstr);
@@ -492,7 +735,7 @@ int unittest_http(void) {
/* Try the error URL, verify error code. */
json = NULL;
- herr = rd_http_get_json(error_url, &json);
+ herr = rd_http_get_json(rk, error_url, NULL, 0, 5, 1, 1000, &json);
RD_UT_ASSERT(herr != NULL, "Expected get_json(%s) to fail", error_url);
RD_UT_ASSERT(herr->code >= 400,
"Expected get_json(%s) error code >= "
@@ -506,6 +749,82 @@ int unittest_http(void) {
if (json)
cJSON_Delete(json);
rd_http_error_destroy(herr);
+ rd_free(rk);
RD_UT_PASS();
}
+
+int unittest_http_get_params_append(void) {
+ rd_kafka_t *rk;
+ char *res;
+ RD_UT_BEGIN();
+ char *tests[] = {"http://localhost:1234",
+ "",
+ "http://localhost:1234/",
+
+ "http://localhost:1234/",
+ "a=1",
+ "http://localhost:1234/?a=1",
+
+ "https://localhost:1234/",
+ "a=1&b=2",
+ "https://localhost:1234/?a=1&b=2",
+
+ "http://mydomain.com/?a=1",
+ "c=hi",
+ "http://mydomain.com/?a=1&c=hi",
+
+ "https://mydomain.com/?",
+ "c=hi",
+ "https://mydomain.com/?c=hi",
+
+ "http://localhost:1234/path?a=1&b=2#&c=3",
+ "c=hi",
+ "http://localhost:1234/path?a=1&b=2&c=hi",
+
+ "http://localhost:1234#?c=3",
+ "a=1",
+ "http://localhost:1234/?a=1",
+
+ "https://otherdomain.io/path?a=1c=3",
+ "b=2",
+ "https://otherdomain.io/path?a=1&b=2",
+ NULL};
+
+
+ res = rd_http_get_params_append("", "");
+ RD_UT_ASSERT(!res, "Expected NULL result, got: \"%s\"", res);
+ res = rd_http_get_params_append("", "a=2&b=3");
+ RD_UT_ASSERT(!res, "Expected NULL result, got: \"%s\"", res);
+
+ char **test = tests;
+ rk = rd_calloc(1, sizeof(*rk));
+ while (test[0]) {
+ res = rd_http_get_params_append(test[0], test[1]);
+ RD_UT_ASSERT(!strcmp(res, test[2]),
+ "Expected \"%s\", got: \"%s\"", test[2], res);
+ rd_free(res);
+ test += 3;
+ }
+ rd_free(rk);
+
+ RD_UT_PASS();
+}
+
+/**
+ * @brief Unittest. Requires a (local) webserver to be set with env var
+ * RD_UT_HTTP_URL=http://localhost:1234/some-path
+ *
+ * This server must return a JSON object or array containing at least one
+ * object on the main URL with a 2xx response code,
+ * and 4xx response on $RD_UT_HTTP_URL/error (with whatever type of body).
+ */
+
+int unittest_http(void) {
+ int fails = 0;
+
+ fails += unittest_http_get();
+ fails += unittest_http_get_params_append();
+
+ return fails;
+}
diff --git a/src/rdhttp.h b/src/rdhttp.h
index 9691cc800e..eb93006bf8 100644
--- a/src/rdhttp.h
+++ b/src/rdhttp.h
@@ -33,7 +33,6 @@
#define CJSON_HIDE_SYMBOLS
#include "cJSON.h"
-
typedef struct rd_http_error_s {
int code;
char *errstr;
@@ -42,8 +41,26 @@ typedef struct rd_http_error_s {
void rd_http_error_destroy(rd_http_error_t *herr);
-rd_http_error_t *rd_http_get(const char *url, rd_buf_t **rbufp);
-rd_http_error_t *rd_http_get_json(const char *url, cJSON **jsonp);
+char *rd_http_get_params_append(const char *url, const char *params);
+
+rd_http_error_t *rd_http_get(rd_kafka_t *rk,
+ const char *url,
+ char **headers_array,
+ size_t headers_array_cnt,
+ int timeout_s,
+ int retries,
+ int retry_ms,
+ rd_buf_t **rbufp,
+ char **content_type,
+ int *response_code);
+rd_http_error_t *rd_http_get_json(rd_kafka_t *rk,
+ const char *url,
+ char **headers_array,
+ size_t headers_array_cnt,
+ int timeout_s,
+ int retries,
+ int retry_ms,
+ cJSON **jsonp);
void rd_http_global_init(void);
@@ -62,7 +79,8 @@ typedef struct rd_http_req_s {
* write to. */
} rd_http_req_t;
-rd_http_error_t *rd_http_req_init(rd_http_req_t *hreq, const char *url);
+rd_http_error_t *
+rd_http_req_init(rd_kafka_t *rk, rd_http_req_t *hreq, const char *url);
rd_http_error_t *rd_http_req_perform_sync(rd_http_req_t *hreq);
rd_http_error_t *rd_http_parse_json(rd_http_req_t *hreq, cJSON **jsonp);
rd_http_error_t *rd_http_post_expect_json(rd_kafka_t *rk,
@@ -71,7 +89,7 @@ rd_http_error_t *rd_http_post_expect_json(rd_kafka_t *rk,
const char *data_to_token,
size_t data_to_token_size,
int timeout_s,
- int retry,
+ int retries,
int retry_ms,
cJSON **jsonp);
void rd_http_req_destroy(rd_http_req_t *hreq);
diff --git a/src/rdkafka.c b/src/rdkafka.c
index 901f3117db..c6f89ad469 100644
--- a/src/rdkafka.c
+++ b/src/rdkafka.c
@@ -404,7 +404,9 @@ static const struct rd_kafka_err_desc rd_kafka_err_descs[] = {
_ERR_DESC(RD_KAFKA_RESP_ERR__BAD_MSG, "Local: Bad message format"),
_ERR_DESC(RD_KAFKA_RESP_ERR__BAD_COMPRESSION,
"Local: Invalid compressed data"),
- _ERR_DESC(RD_KAFKA_RESP_ERR__DESTROY, "Local: Broker handle destroyed"),
+ _ERR_DESC(RD_KAFKA_RESP_ERR__DESTROY,
+ "Local: Broker handle destroyed "
+ "for termination"),
_ERR_DESC(
RD_KAFKA_RESP_ERR__FAIL,
"Local: Communication failure with broker"), // FIXME: too specific
@@ -488,7 +490,9 @@ static const struct rd_kafka_err_desc rd_kafka_err_descs[] = {
"Local: Partition log truncation detected"),
_ERR_DESC(RD_KAFKA_RESP_ERR__INVALID_DIFFERENT_RECORD,
"Local: an invalid record in the same batch caused "
- "the failure of this message too."),
+ "the failure of this message too"),
+ _ERR_DESC(RD_KAFKA_RESP_ERR__DESTROY_BROKER,
+ "Local: Broker handle destroyed without termination"),
_ERR_DESC(RD_KAFKA_RESP_ERR_UNKNOWN, "Unknown broker error"),
_ERR_DESC(RD_KAFKA_RESP_ERR_NO_ERROR, "Success"),
@@ -714,6 +718,9 @@ static const struct rd_kafka_err_desc rd_kafka_err_descs[] = {
_ERR_DESC(RD_KAFKA_RESP_ERR_TELEMETRY_TOO_LARGE,
"Broker: Client sent a push telemetry request larger than the "
"maximum size the broker will accept"),
+ _ERR_DESC(RD_KAFKA_RESP_ERR_REBOOTSTRAP_REQUIRED,
+ "Broker: Client metadata is stale, "
+ "client should rebootstrap to obtain new metadata"),
_ERR_DESC(RD_KAFKA_RESP_ERR__END, NULL)};
@@ -925,7 +932,29 @@ rd_kafka_resp_err_t rd_kafka_test_fatal_error(rd_kafka_t *rk,
return RD_KAFKA_RESP_ERR_NO_ERROR;
}
-
+/**
+ * @brief Called when a broker thread is decommissioned.
+ * on the main thread to join the corresponding thread
+ * and remove it from the wait lists.
+ *
+ * @locality main thread
+ */
+void rd_kafka_decommissioned_broker_thread_join(rd_kafka_t *rk,
+ void *rkb_decommissioned) {
+ thrd_t *thrd;
+ int i;
+ RD_LIST_FOREACH(thrd, &rk->wait_decommissioned_thrds, i) {
+ void *rkb = rd_list_elem(&rk->wait_decommissioned_brokers, i);
+ if (rkb == rkb_decommissioned) {
+ rd_list_remove_elem(&rk->wait_decommissioned_thrds, i);
+ rd_list_remove_elem(&rk->wait_decommissioned_brokers,
+ i);
+ thrd_join(*thrd, NULL);
+ rd_free(thrd);
+ i--;
+ }
+ }
+}
/**
* @brief Final destructor for rd_kafka_t, must only be called with refcnt 0.
@@ -964,6 +993,18 @@ void rd_kafka_destroy_final(rd_kafka_t *rk) {
rd_kafka_assignment_destroy(rk);
if (rk->rk_consumer.q)
rd_kafka_q_destroy(rk->rk_consumer.q);
+ rd_avg_destroy(
+ &rk->rk_telemetry.rd_avg_current.rk_avg_poll_idle_ratio);
+ rd_avg_destroy(
+ &rk->rk_telemetry.rd_avg_current.rk_avg_rebalance_latency);
+ rd_avg_destroy(
+ &rk->rk_telemetry.rd_avg_current.rk_avg_commit_latency);
+ rd_avg_destroy(
+ &rk->rk_telemetry.rd_avg_rollover.rk_avg_poll_idle_ratio);
+ rd_avg_destroy(
+ &rk->rk_telemetry.rd_avg_rollover.rk_avg_rebalance_latency);
+ rd_avg_destroy(
+ &rk->rk_telemetry.rd_avg_rollover.rk_avg_commit_latency);
}
/* Purge op-queues */
@@ -1005,8 +1046,7 @@ void rd_kafka_destroy_final(rd_kafka_t *rk) {
cnd_destroy(&rk->rk_init_cnd);
mtx_destroy(&rk->rk_init_lock);
- if (rk->rk_full_metadata)
- rd_kafka_metadata_destroy(&rk->rk_full_metadata->metadata);
+
rd_kafkap_str_destroy(rk->rk_client_id);
rd_kafkap_str_destroy(rk->rk_group_id);
rd_kafkap_str_destroy(rk->rk_eos.transactional_id);
@@ -1156,8 +1196,8 @@ void rd_kafka_destroy_flags(rd_kafka_t *rk, int flags) {
*/
static void rd_kafka_destroy_internal(rd_kafka_t *rk) {
rd_kafka_topic_t *rkt, *rkt_tmp;
- rd_kafka_broker_t *rkb, *rkb_tmp;
- rd_list_t wait_thrds;
+ rd_kafka_broker_t *rkb;
+ rd_list_t wait_thrds, brokers_to_decommission;
thrd_t *thrd;
int i;
@@ -1200,32 +1240,24 @@ static void rd_kafka_destroy_internal(rd_kafka_t *rk) {
}
/* Decommission brokers.
- * Broker thread holds a refcount and detects when broker refcounts
- * reaches 1 and then decommissions itself. */
- TAILQ_FOREACH_SAFE(rkb, &rk->rk_brokers, rkb_link, rkb_tmp) {
- /* Add broker's thread to wait_thrds list for later joining */
- thrd = rd_malloc(sizeof(*thrd));
- *thrd = rkb->rkb_thread;
- rd_list_add(&wait_thrds, thrd);
- rd_kafka_wrunlock(rk);
-
- rd_kafka_dbg(rk, BROKER, "DESTROY", "Sending TERMINATE to %s",
- rd_kafka_broker_name(rkb));
- /* Send op to trigger queue/io wake-up.
- * The op itself is (likely) ignored by the broker thread. */
- rd_kafka_q_enq(rkb->rkb_ops,
- rd_kafka_op_new(RD_KAFKA_OP_TERMINATE));
-
-#ifndef _WIN32
- /* Interrupt IO threads to speed up termination. */
- if (rk->rk_conf.term_sig)
- pthread_kill(rkb->rkb_thread, rk->rk_conf.term_sig);
-#endif
-
- rd_kafka_broker_destroy(rkb);
+ * `rd_kafka_broker_decommission` releases and reacquires
+ * the lock so there could be destroyed brokers in
+ * `rk->rk_brokers` */
+ rd_list_init(&brokers_to_decommission,
+ rd_atomic32_get(&rk->rk_broker_cnt), NULL);
+ TAILQ_FOREACH(rkb, &rk->rk_brokers, rkb_link) {
+ /* Don't try to decommission already decommissioning brokers
+ * otherwise they could be already destroyed when
+ * `rd_kafka_broker_decommission` is called below. */
+ if (rd_list_find(&rk->wait_decommissioned_brokers, rkb,
+ rd_list_cmp_ptr) == NULL)
+ rd_list_add(&brokers_to_decommission, rkb);
+ }
- rd_kafka_wrlock(rk);
+ RD_LIST_FOREACH(rkb, &brokers_to_decommission, i) {
+ rd_kafka_broker_decommission(rk, rkb, &wait_thrds);
}
+ rd_list_destroy(&brokers_to_decommission);
if (rk->rk_clusterid) {
rd_free(rk->rk_clusterid);
@@ -1267,22 +1299,23 @@ static void rd_kafka_destroy_internal(rd_kafka_t *rk) {
/* Loose our special reference to the internal broker. */
mtx_lock(&rk->rk_internal_rkb_lock);
- if ((rkb = rk->rk_internal_rkb)) {
+ if (rk->rk_internal_rkb) {
rd_kafka_dbg(rk, GENERIC, "TERMINATE",
"Decommissioning internal broker");
- /* Send op to trigger queue wake-up. */
- rd_kafka_q_enq(rkb->rkb_ops,
+ thrd = rd_malloc(sizeof(*thrd));
+ *thrd = rk->rk_internal_rkb->rkb_thread;
+
+ /* Send op to trigger queue wake-up.
+ * WARNING: This is last time we can read
+ * from rk_internal_rkb in this thread! */
+ rd_kafka_q_enq(rk->rk_internal_rkb->rkb_ops,
rd_kafka_op_new(RD_KAFKA_OP_TERMINATE));
rk->rk_internal_rkb = NULL;
- thrd = rd_malloc(sizeof(*thrd));
- *thrd = rkb->rkb_thread;
rd_list_add(&wait_thrds, thrd);
}
mtx_unlock(&rk->rk_internal_rkb_lock);
- if (rkb)
- rd_kafka_broker_destroy(rkb);
rd_kafka_dbg(rk, GENERIC, "TERMINATE", "Join %d broker thread(s)",
@@ -1298,6 +1331,17 @@ static void rd_kafka_destroy_internal(rd_kafka_t *rk) {
rd_list_destroy(&wait_thrds);
+ /* Join previously decommissioned broker threads */
+ RD_LIST_FOREACH(thrd, &rk->wait_decommissioned_thrds, i) {
+ int res;
+ if (thrd_join(*thrd, &res) != thrd_success)
+ ;
+ rd_free(thrd);
+ }
+ rd_list_destroy(&rk->additional_brokerlists);
+ rd_list_destroy(&rk->wait_decommissioned_brokers);
+ rd_list_destroy(&rk->wait_decommissioned_thrds);
+
/* Destroy mock cluster */
if (rk->rk_mock.cluster)
rd_kafka_mock_cluster_destroy(rk->rk_mock.cluster);
@@ -1422,9 +1466,7 @@ static RD_INLINE void rd_kafka_stats_emit_toppar(struct _stats_emit *st,
rd_kafka_toppar_lock(rktp);
if (rktp->rktp_broker) {
- rd_kafka_broker_lock(rktp->rktp_broker);
broker_id = rktp->rktp_broker->rkb_nodeid;
- rd_kafka_broker_unlock(rktp->rktp_broker);
}
/* Grab a copy of the latest finalized offset stats */
@@ -1648,6 +1690,7 @@ static void rd_kafka_stats_emit_broker_reqs(struct _stats_emit *st,
[RD_KAFKAP_AlterClientQuotas] = rd_true,
[RD_KAFKAP_DescribeUserScramCredentials] = rd_true,
[RD_KAFKAP_AlterUserScramCredentials] = rd_true,
+ [RD_KAFKAP_ConsumerGroupDescribe] = rd_true,
}};
int i;
int cnt = 0;
@@ -2017,6 +2060,85 @@ static void rd_kafka_1s_tmr_cb(rd_kafka_timers_t *rkts, void *arg) {
rd_kafka_coord_cache_expire(&rk->rk_coord_cache);
}
+/**
+ * @brief Reset broker down reported flag for all brokers.
+ * In case it was set to 1 it will be reset to 0 and
+ * the broker down count will be decremented.
+ *
+ * @locks none
+ * @locks_acquired rd_kafka_rdlock()
+ * @locality any
+ */
+void rd_kafka_reset_any_broker_down_reported(rd_kafka_t *rk) {
+ rd_kafka_broker_t *rkb;
+ rd_kafka_rdlock(rk);
+ TAILQ_FOREACH(rkb, &rk->rk_brokers, rkb_link) {
+ if (rd_atomic32_set(&rkb->rkb_down_reported, 0) == 1)
+ rd_atomic32_sub(&rk->rk_broker_down_cnt, 1);
+ }
+ rd_kafka_rdunlock(rk);
+}
+
+/**
+ * @brief Re-bootstrap timer callback.
+ *
+ * @locality rdkafka main thread
+ * @locks none
+ */
+static void rd_kafka_rebootstrap_tmr_cb(rd_kafka_timers_t *rkts, void *arg) {
+ int i;
+ char *brokerlist;
+ rd_kafka_t *rk = rkts->rkts_rk;
+ rd_list_t additional_brokerlists;
+
+ rd_dassert(thrd_is_current(rk->rk_thread));
+ if (rd_kafka_terminating(rk))
+ /* Avoid re-bootstrapping while terminating */
+ return;
+
+ rd_dassert(rk->rk_conf.metadata_recovery_strategy !=
+ RD_KAFKA_METADATA_RECOVERY_STRATEGY_NONE);
+ if (rk->rk_conf.metadata_recovery_strategy ==
+ RD_KAFKA_METADATA_RECOVERY_STRATEGY_NONE)
+ /* This function should not be called in this case.
+ * this is just a fail-safe. */
+ return;
+
+ rd_kafka_dbg(rk, ALL, "REBOOTSTRAP", "Starting re-bootstrap sequence");
+
+ rd_atomic32_set(&rk->rk_rebootstrap_in_progress, 1);
+ rd_kafka_reset_any_broker_down_reported(rk);
+
+ if (rk->rk_conf.brokerlist) {
+ rd_kafka_brokers_add0(
+ rk,
+ rk->rk_conf.brokerlist, rd_true
+ /* resolve canonical bootstrap server
+ * list names if requested*/);
+ }
+
+ rd_kafka_rdlock(rk);
+ if (rd_list_cnt(&rk->additional_brokerlists) == 0) {
+ rd_kafka_rdunlock(rk);
+ goto done;
+ }
+
+ rd_list_init_copy(&additional_brokerlists, &rk->additional_brokerlists);
+ rd_list_copy_to(&additional_brokerlists, &rk->additional_brokerlists,
+ rd_list_string_copy, NULL);
+ rd_kafka_rdunlock(rk);
+
+ RD_LIST_FOREACH(brokerlist, &additional_brokerlists, i) {
+ rd_kafka_brokers_add0(rk, brokerlist,
+ rd_false
+ /* don't resolve canonical bootstrap server list
+ * names even if requested */);
+ }
+ rd_list_destroy(&additional_brokerlists);
+done:
+ rd_atomic32_set(&rk->rk_rebootstrap_in_progress, 0);
+}
+
static void rd_kafka_stats_emit_tmr_cb(rd_kafka_timers_t *rkts, void *arg) {
rd_kafka_t *rk = rkts->rkts_rk;
rd_kafka_stats_emit_all(rk);
@@ -2070,15 +2192,15 @@ static void rd_kafka_metadata_refresh_cb(rd_kafka_timers_t *rkts, void *arg) {
* @locks none
*/
static int rd_kafka_init_wait(rd_kafka_t *rk, int timeout_ms) {
- struct timespec tspec;
int ret;
+ rd_ts_t abs_timeout;
- rd_timeout_init_timespec(&tspec, timeout_ms);
+ abs_timeout = rd_timeout_init(timeout_ms);
mtx_lock(&rk->rk_init_lock);
while (rk->rk_init_wait_cnt > 0 &&
- cnd_timedwait_abs(&rk->rk_init_cnd, &rk->rk_init_lock, &tspec) ==
- thrd_success)
+ cnd_timedwait_abs(&rk->rk_init_cnd, &rk->rk_init_lock,
+ abs_timeout) == thrd_success)
;
ret = rk->rk_init_wait_cnt;
mtx_unlock(&rk->rk_init_lock);
@@ -2091,10 +2213,8 @@ static int rd_kafka_init_wait(rd_kafka_t *rk, int timeout_ms) {
* Main loop for Kafka handler thread.
*/
static int rd_kafka_thread_main(void *arg) {
- rd_kafka_t *rk = arg;
- rd_kafka_timer_t tmr_1s = RD_ZERO_INIT;
- rd_kafka_timer_t tmr_stats_emit = RD_ZERO_INIT;
- rd_kafka_timer_t tmr_metadata_refresh = RD_ZERO_INIT;
+ rd_kafka_t *rk = arg;
+ rd_kafka_timer_t tmr_stats_emit = RD_ZERO_INIT;
rd_kafka_set_thread_name("main");
rd_kafka_set_thread_sysname("rdk:main");
@@ -2109,14 +2229,14 @@ static int rd_kafka_thread_main(void *arg) {
rd_kafka_wrunlock(rk);
/* 1 second timer for topic scan and connection checking. */
- rd_kafka_timer_start(&rk->rk_timers, &tmr_1s, 1000000,
+ rd_kafka_timer_start(&rk->rk_timers, &rk->one_s_tmr, 1000000,
rd_kafka_1s_tmr_cb, NULL);
if (rk->rk_conf.stats_interval_ms)
rd_kafka_timer_start(&rk->rk_timers, &tmr_stats_emit,
rk->rk_conf.stats_interval_ms * 1000ll,
rd_kafka_stats_emit_tmr_cb, NULL);
if (rk->rk_conf.metadata_refresh_interval_ms > 0)
- rd_kafka_timer_start(&rk->rk_timers, &tmr_metadata_refresh,
+ rd_kafka_timer_start(&rk->rk_timers, &rk->metadata_refresh_tmr,
rk->rk_conf.metadata_refresh_interval_ms *
1000ll,
rd_kafka_metadata_refresh_cb, NULL);
@@ -2156,10 +2276,10 @@ static int rd_kafka_thread_main(void *arg) {
rd_kafka_q_disable(rk->rk_ops);
rd_kafka_q_purge(rk->rk_ops);
- rd_kafka_timer_stop(&rk->rk_timers, &tmr_1s, 1);
+ rd_kafka_timer_stop(&rk->rk_timers, &rk->one_s_tmr, 1);
if (rk->rk_conf.stats_interval_ms)
rd_kafka_timer_stop(&rk->rk_timers, &tmr_stats_emit, 1);
- rd_kafka_timer_stop(&rk->rk_timers, &tmr_metadata_refresh, 1);
+ rd_kafka_timer_stop(&rk->rk_timers, &rk->metadata_refresh_tmr, 1);
/* Synchronise state */
rd_kafka_wrlock(rk);
@@ -2193,7 +2313,6 @@ rd_kafka_t *rd_kafka_new(rd_kafka_type_t type,
rd_kafka_resp_err_t ret_err = RD_KAFKA_RESP_ERR_NO_ERROR;
int ret_errno = 0;
const char *conf_err;
- char *group_remote_assignor_override = NULL;
#ifndef _WIN32
sigset_t newset, oldset;
#endif
@@ -2278,6 +2397,11 @@ rd_kafka_t *rd_kafka_new(rd_kafka_type_t type,
rd_atomic64_init(&rk->rk_ts_last_poll, rk->rk_ts_created);
rd_atomic32_init(&rk->rk_flushing, 0);
+ rd_atomic32_init(&rk->rk_broker_cnt, 0);
+ rd_atomic32_init(&rk->rk_logical_broker_cnt, 0);
+ rd_atomic32_init(&rk->rk_broker_up_cnt, 0);
+ rd_atomic32_init(&rk->rk_broker_down_cnt, 0);
+ rd_atomic32_init(&rk->rk_rebootstrap_in_progress, 0);
rk->rk_rep = rd_kafka_q_new(rk);
rk->rk_ops = rd_kafka_q_new(rk);
@@ -2290,6 +2414,7 @@ rd_kafka_t *rd_kafka_new(rd_kafka_type_t type,
rk->rk_logq->rkq_opaque = rk;
}
+ rd_list_init(&rk->additional_brokerlists, 0, rd_free);
TAILQ_INIT(&rk->rk_brokers);
TAILQ_INIT(&rk->rk_topics);
rd_kafka_timers_init(&rk->rk_timers, rk, rk->rk_ops);
@@ -2297,6 +2422,8 @@ rd_kafka_t *rd_kafka_new(rd_kafka_type_t type,
rd_kafka_coord_cache_init(&rk->rk_coord_cache,
rk->rk_conf.metadata_max_age_ms);
rd_kafka_coord_reqs_init(rk);
+ rd_list_init(&rk->wait_decommissioned_thrds, 0, NULL);
+ rd_list_init(&rk->wait_decommissioned_brokers, 0, NULL);
if (rk->rk_conf.dr_cb || rk->rk_conf.dr_msg_cb)
rk->rk_drmode = RD_KAFKA_DR_MODE_CB;
@@ -2329,11 +2456,30 @@ rd_kafka_t *rd_kafka_new(rd_kafka_type_t type,
#if WITH_OAUTHBEARER_OIDC
if (rk->rk_conf.sasl.oauthbearer.method ==
RD_KAFKA_SASL_OAUTHBEARER_METHOD_OIDC &&
- !rk->rk_conf.sasl.oauthbearer.token_refresh_cb)
- rd_kafka_conf_set_oauthbearer_token_refresh_cb(
- &rk->rk_conf, rd_kafka_oidc_token_refresh_cb);
+ !rk->rk_conf.sasl.oauthbearer.token_refresh_cb) {
+ /* Use JWT bearer */
+ rk->rk_conf.sasl.oauthbearer.builtin_token_refresh_cb = rd_true;
+
+ if (rk->rk_conf.sasl.oauthbearer.metadata_authentication.type ==
+ RD_KAFKA_SASL_OAUTHBEARER_METADATA_AUTHENTICATION_TYPE_AZURE_IMDS) {
+ rd_kafka_conf_set_oauthbearer_token_refresh_cb(
+ &rk->rk_conf,
+ rd_kafka_oidc_token_metadata_azure_imds_refresh_cb);
+ } else if (
+ rk->rk_conf.sasl.oauthbearer.grant_type ==
+ RD_KAFKA_SASL_OAUTHBEARER_GRANT_TYPE_CLIENT_CREDENTIALS) {
+ rd_kafka_conf_set_oauthbearer_token_refresh_cb(
+ &rk->rk_conf,
+ rd_kafka_oidc_token_client_credentials_refresh_cb);
+ } else {
+ rd_kafka_conf_set_oauthbearer_token_refresh_cb(
+ &rk->rk_conf,
+ rd_kafka_oidc_token_jwt_bearer_refresh_cb);
+ }
+ }
#endif
+
rk->rk_controllerid = -1;
/* Admin client defaults */
@@ -2388,64 +2534,6 @@ rd_kafka_t *rd_kafka_new(rd_kafka_type_t type,
goto fail;
}
- if (!rk->rk_conf.group_remote_assignor) {
- rd_kafka_assignor_t *cooperative_assignor;
-
- /* Detect if chosen assignor is cooperative
- * FIXME: remove this compatibility altogether
- * and apply the breaking changes that will be required
- * in next major version. */
-
- cooperative_assignor =
- rd_kafka_assignor_find(rk, "cooperative-sticky");
- rk->rk_conf.partition_assignors_cooperative =
- !rk->rk_conf.partition_assignors.rl_cnt ||
- (cooperative_assignor &&
- cooperative_assignor->rkas_enabled);
-
- if (rk->rk_conf.group_protocol ==
- RD_KAFKA_GROUP_PROTOCOL_CONSUMER) {
- /* Default remote assignor to the chosen local one. */
- if (rk->rk_conf.partition_assignors_cooperative) {
- group_remote_assignor_override =
- rd_strdup("uniform");
- rk->rk_conf.group_remote_assignor =
- group_remote_assignor_override;
- } else {
- rd_kafka_assignor_t *range_assignor =
- rd_kafka_assignor_find(rk, "range");
- if (range_assignor &&
- range_assignor->rkas_enabled) {
- rd_kafka_log(
- rk, LOG_WARNING, "ASSIGNOR",
- "\"range\" assignor is sticky "
- "with group protocol CONSUMER");
- group_remote_assignor_override =
- rd_strdup("range");
- rk->rk_conf.group_remote_assignor =
- group_remote_assignor_override;
- } else {
- rd_kafka_log(
- rk, LOG_WARNING, "ASSIGNOR",
- "roundrobin assignor isn't "
- "available "
- "with group protocol CONSUMER, "
- "using the \"uniform\" one. "
- "It's similar, "
- "but it's also sticky");
- group_remote_assignor_override =
- rd_strdup("uniform");
- rk->rk_conf.group_remote_assignor =
- group_remote_assignor_override;
- }
- }
- }
- } else {
- /* When users starts setting properties of the new protocol,
- * they can only use incremental_assign/unassign. */
- rk->rk_conf.partition_assignors_cooperative = rd_true;
- }
-
/* Create Mock cluster */
rd_atomic32_init(&rk->rk_mock.cluster_cnt, 0);
if (rk->rk_conf.mock.broker_cnt > 0) {
@@ -2534,6 +2622,29 @@ rd_kafka_t *rd_kafka_new(rd_kafka_type_t type,
rk->rk_consumer.q = rd_kafka_q_keep(rk->rk_rep);
}
+ rd_avg_init(
+ &rk->rk_telemetry.rd_avg_rollover.rk_avg_poll_idle_ratio,
+ RD_AVG_GAUGE, 0, 1, 2, rk->rk_conf.enable_metrics_push);
+ rd_avg_init(
+ &rk->rk_telemetry.rd_avg_current.rk_avg_poll_idle_ratio,
+ RD_AVG_GAUGE, 0, 1, 2, rk->rk_conf.enable_metrics_push);
+ rd_avg_init(
+ &rk->rk_telemetry.rd_avg_rollover.rk_avg_rebalance_latency,
+ RD_AVG_GAUGE, 0, 500 * 1000, 2,
+ rk->rk_conf.enable_metrics_push);
+ rd_avg_init(
+ &rk->rk_telemetry.rd_avg_current.rk_avg_rebalance_latency,
+ RD_AVG_GAUGE, 0, 900000 * 1000, 2,
+ rk->rk_conf.enable_metrics_push);
+ rd_avg_init(
+ &rk->rk_telemetry.rd_avg_rollover.rk_avg_commit_latency,
+ RD_AVG_GAUGE, 0, 500 * 1000, 2,
+ rk->rk_conf.enable_metrics_push);
+ rd_avg_init(
+ &rk->rk_telemetry.rd_avg_current.rk_avg_commit_latency,
+ RD_AVG_GAUGE, 0, 500 * 1000, 2,
+ rk->rk_conf.enable_metrics_push);
+
} else if (type == RD_KAFKA_PRODUCER) {
rk->rk_eos.transactional_id =
rd_kafkap_str_new(rk->rk_conf.eos.transactional_id, -1);
@@ -2715,8 +2826,6 @@ rd_kafka_t *rd_kafka_new(rd_kafka_type_t type,
* that belong to rk_conf and thus needs to be cleaned up.
* Legacy APIs, sigh.. */
if (app_conf) {
- if (group_remote_assignor_override)
- rd_free(group_remote_assignor_override);
rd_kafka_assignors_term(rk);
rd_kafka_interceptors_destroy(&rk->rk_conf);
memset(&rk->rk_conf, 0, sizeof(rk->rk_conf));
@@ -2730,7 +2839,65 @@ rd_kafka_t *rd_kafka_new(rd_kafka_type_t type,
return NULL;
}
+/**
+ * Schedules a rebootstrap of the cluster immediately.
+ *
+ * @locks none
+ * @locks_acquired rd_kafka_timers_lock()
+ * @locality any
+ */
+void rd_kafka_rebootstrap(rd_kafka_t *rk) {
+ if (rk->rk_conf.metadata_recovery_strategy ==
+ RD_KAFKA_METADATA_RECOVERY_STRATEGY_NONE)
+ return;
+
+ if (rd_atomic32_set(&rk->rk_rebootstrap_in_progress, 1) == 0) {
+ /* Only when not already in progress 0 -> 1.
+ * After setting down a learned broker it could reconnect and
+ * disconnect again before previous reboostrap completes,
+ * causing a new re-bootstrap. */
+ rd_kafka_timer_start_oneshot(
+ &rk->rk_timers, &rk->rebootstrap_tmr, rd_true /*restart*/,
+ 0, rd_kafka_rebootstrap_tmr_cb, NULL);
+ }
+}
+
+/**
+ * Starts rebootstrap timer with the configured interval. Only if not
+ * started or stopped.
+ *
+ * @locks none
+ * @locks_acquired rd_kafka_timers_lock()
+ * @locality any
+ */
+void rd_kafka_rebootstrap_tmr_start_maybe(rd_kafka_t *rk) {
+ if (rk->rk_conf.metadata_recovery_strategy ==
+ RD_KAFKA_METADATA_RECOVERY_STRATEGY_NONE)
+ return;
+
+ rd_kafka_timer_start_oneshot(
+ &rk->rk_timers, &rk->rebootstrap_tmr, rd_false /*don't restart*/,
+ rk->rk_conf.metadata_recovery_rebootstrap_trigger_ms * 1000LL,
+ rd_kafka_rebootstrap_tmr_cb, NULL);
+}
+
+/**
+ * Stops rebootstrap timer, for example after a successful metadata response.
+ *
+ * @return 1 if the timer was started (before being stopped), else 0.
+ *
+ * @locks none
+ * @locks_acquired rd_kafka_timers_lock()
+ * @locality any
+ */
+int rd_kafka_rebootstrap_tmr_stop(rd_kafka_t *rk) {
+ if (rk->rk_conf.metadata_recovery_strategy ==
+ RD_KAFKA_METADATA_RECOVERY_STRATEGY_NONE)
+ return 0;
+ return rd_kafka_timer_stop(&rk->rk_timers, &rk->rebootstrap_tmr,
+ rd_true /* lock */);
+}
/**
* Counts usage of the legacy/simple consumer (rd_kafka_consume_start() with
@@ -3143,13 +3310,12 @@ static rd_kafka_op_res_t rd_kafka_consume_callback0(
struct consume_ctx ctx = {.consume_cb = consume_cb, .opaque = opaque};
rd_kafka_op_res_t res;
- if (timeout_ms)
- rd_kafka_app_poll_blocking(rkq->rkq_rk);
+ rd_kafka_app_poll_start(rkq->rkq_rk, rkq, 0, timeout_ms);
res = rd_kafka_q_serve(rkq, timeout_ms, max_cnt, RD_KAFKA_Q_CB_RETURN,
rd_kafka_consume_cb, &ctx);
- rd_kafka_app_polled(rkq->rkq_rk);
+ rd_kafka_app_polled(rkq->rkq_rk, rkq);
return res;
}
@@ -3212,16 +3378,15 @@ static rd_kafka_message_t *
rd_kafka_consume0(rd_kafka_t *rk, rd_kafka_q_t *rkq, int timeout_ms) {
rd_kafka_op_t *rko;
rd_kafka_message_t *rkmessage = NULL;
- rd_ts_t abs_timeout = rd_timeout_init(timeout_ms);
+ rd_ts_t now = rd_clock();
+ rd_ts_t abs_timeout = rd_timeout_init0(now, timeout_ms);
- if (timeout_ms)
- rd_kafka_app_poll_blocking(rk);
+ rd_kafka_app_poll_start(rk, rkq, now, timeout_ms);
rd_kafka_yield_thread = 0;
while ((
rko = rd_kafka_q_pop(rkq, rd_timeout_remains_us(abs_timeout), 0))) {
rd_kafka_op_res_t res;
-
res =
rd_kafka_poll_cb(rk, rkq, rko, RD_KAFKA_Q_CB_RETURN, NULL);
@@ -3233,7 +3398,7 @@ rd_kafka_consume0(rd_kafka_t *rk, rd_kafka_q_t *rkq, int timeout_ms) {
/* Callback called rd_kafka_yield(), we must
* stop dispatching the queue and return. */
rd_kafka_set_last_error(RD_KAFKA_RESP_ERR__INTR, EINTR);
- rd_kafka_app_polled(rk);
+ rd_kafka_app_polled(rk, rkq);
return NULL;
}
@@ -3245,7 +3410,7 @@ rd_kafka_consume0(rd_kafka_t *rk, rd_kafka_q_t *rkq, int timeout_ms) {
/* Timeout reached with no op returned. */
rd_kafka_set_last_error(RD_KAFKA_RESP_ERR__TIMED_OUT,
ETIMEDOUT);
- rd_kafka_app_polled(rk);
+ rd_kafka_app_polled(rk, rkq);
return NULL;
}
@@ -3260,7 +3425,7 @@ rd_kafka_consume0(rd_kafka_t *rk, rd_kafka_q_t *rkq, int timeout_ms) {
rd_kafka_set_last_error(0, 0);
- rd_kafka_app_polled(rk);
+ rd_kafka_app_polled(rk, rkq);
return rkmessage;
}
@@ -3648,6 +3813,7 @@ rd_kafka_resp_err_t rd_kafka_query_watermark_offsets(rd_kafka_t *rk,
struct rd_kafka_partition_leader *leader;
rd_list_t leaders;
rd_kafka_resp_err_t err;
+ int tmout;
partitions = rd_kafka_topic_partition_list_new(1);
rktpar =
@@ -3694,9 +3860,13 @@ rd_kafka_resp_err_t rd_kafka_query_watermark_offsets(rd_kafka_t *rk,
/* Wait for reply (or timeout) */
while (state.err == RD_KAFKA_RESP_ERR__IN_PROGRESS) {
- rd_kafka_q_serve(rkq, RD_POLL_INFINITE, 0,
- RD_KAFKA_Q_CB_CALLBACK, rd_kafka_poll_cb,
- NULL);
+ tmout = rd_timeout_remains(ts_end);
+ if (rd_timeout_expired(tmout)) {
+ state.err = RD_KAFKA_RESP_ERR__TIMED_OUT;
+ break;
+ }
+ rd_kafka_q_serve(rkq, tmout, 0, RD_KAFKA_Q_CB_CALLBACK,
+ rd_kafka_poll_cb, NULL);
}
rd_kafka_q_destroy_owner(rkq);
@@ -3766,6 +3936,7 @@ static void rd_kafka_get_offsets_for_times_resp_cb(rd_kafka_t *rk,
rd_kafka_buf_t *request,
void *opaque) {
struct _get_offsets_for_times *state;
+ int actions = 0;
if (err == RD_KAFKA_RESP_ERR__DESTROY) {
/* 'state' has gone out of scope when offsets_for_times()
@@ -3776,10 +3947,22 @@ static void rd_kafka_get_offsets_for_times_resp_cb(rd_kafka_t *rk,
state = opaque;
err = rd_kafka_handle_ListOffsets(rk, rkb, err, rkbuf, request,
- state->results, NULL);
+ state->results, &actions);
if (err == RD_KAFKA_RESP_ERR__IN_PROGRESS)
return; /* Retrying */
+ if (actions & RD_KAFKA_ERR_ACTION_REFRESH) {
+ rd_kafka_topic_partition_t *rktpar;
+ /* Remove its cache in case the topic isn't a known topic. */
+ rd_kafka_wrlock(rk);
+ RD_KAFKA_TPLIST_FOREACH(rktpar, state->results) {
+ if (rktpar->err)
+ rd_kafka_metadata_cache_delete_by_name(
+ rk, rktpar->topic);
+ }
+ rd_kafka_wrunlock(rk);
+ }
+
/* Retry if no broker connection is available yet. */
if (err == RD_KAFKA_RESP_ERR__TRANSPORT && rkb &&
rd_kafka_brokers_wait_state_change(
@@ -3894,9 +4077,10 @@ rd_kafka_op_res_t rd_kafka_poll_cb(rd_kafka_t *rk,
cb_type == RD_KAFKA_Q_CB_FORCE_RETURN)
return RD_KAFKA_OP_RES_PASS; /* Dont handle here */
else {
- struct consume_ctx ctx = {.consume_cb =
- rk->rk_conf.consume_cb,
- .opaque = rk->rk_conf.opaque};
+ rkq->rkq_ts_last_poll_end = rd_clock();
+ struct consume_ctx ctx = {.consume_cb =
+ rk->rk_conf.consume_cb,
+ .opaque = rk->rk_conf.opaque};
return rd_kafka_consume_cb(rk, rkq, rko, cb_type, &ctx);
}
@@ -4046,6 +4230,9 @@ rd_kafka_op_res_t rd_kafka_poll_cb(rd_kafka_t *rk,
case RD_KAFKA_OP_TERMINATE:
/* nop: just a wake-up */
res = RD_KAFKA_OP_RES_YIELD;
+ if (rko->rko_u.terminated.cb) {
+ rko->rko_u.terminated.cb(rk, rko->rko_u.terminated.rkb);
+ }
rd_kafka_op_destroy(rko);
break;
@@ -4123,8 +4310,9 @@ rd_kafka_op_res_t rd_kafka_poll_cb(rd_kafka_t *rk,
int rd_kafka_poll(rd_kafka_t *rk, int timeout_ms) {
int r;
- r = rd_kafka_q_serve(rk->rk_rep, timeout_ms, 0, RD_KAFKA_Q_CB_CALLBACK,
- rd_kafka_poll_cb, NULL);
+ r = rd_kafka_q_serve_maybe_consume(rk->rk_rep, timeout_ms, 0,
+ RD_KAFKA_Q_CB_CALLBACK,
+ rd_kafka_poll_cb, NULL);
return r;
}
@@ -4132,8 +4320,9 @@ int rd_kafka_poll(rd_kafka_t *rk, int timeout_ms) {
rd_kafka_event_t *rd_kafka_queue_poll(rd_kafka_queue_t *rkqu, int timeout_ms) {
rd_kafka_op_t *rko;
- rko = rd_kafka_q_pop_serve(rkqu->rkqu_q, rd_timeout_us(timeout_ms), 0,
- RD_KAFKA_Q_CB_EVENT, rd_kafka_poll_cb, NULL);
+ rko = rd_kafka_q_pop_serve_maybe_consume(
+ rkqu->rkqu_q, rd_timeout_us(timeout_ms), 0, RD_KAFKA_Q_CB_EVENT,
+ rd_kafka_poll_cb, NULL);
if (!rko)
@@ -4145,8 +4334,9 @@ rd_kafka_event_t *rd_kafka_queue_poll(rd_kafka_queue_t *rkqu, int timeout_ms) {
int rd_kafka_queue_poll_callback(rd_kafka_queue_t *rkqu, int timeout_ms) {
int r;
- r = rd_kafka_q_serve(rkqu->rkqu_q, timeout_ms, 0,
- RD_KAFKA_Q_CB_CALLBACK, rd_kafka_poll_cb, NULL);
+ r = rd_kafka_q_serve_maybe_consume(rkqu->rkqu_q, timeout_ms, 0,
+ RD_KAFKA_Q_CB_CALLBACK,
+ rd_kafka_poll_cb, NULL);
return r;
}
@@ -4703,6 +4893,26 @@ rd_kafka_consumer_group_state_code(const char *name) {
return RD_KAFKA_CONSUMER_GROUP_STATE_UNKNOWN;
}
+static const char *rd_kafka_consumer_group_type_names[] = {
+ "Unknown", "Consumer", "Classic"};
+
+const char *
+rd_kafka_consumer_group_type_name(rd_kafka_consumer_group_type_t type) {
+ if (type < 0 || type >= RD_KAFKA_CONSUMER_GROUP_TYPE__CNT)
+ return NULL;
+ return rd_kafka_consumer_group_type_names[type];
+}
+
+rd_kafka_consumer_group_type_t
+rd_kafka_consumer_group_type_code(const char *name) {
+ size_t i;
+ for (i = 0; i < RD_KAFKA_CONSUMER_GROUP_TYPE__CNT; i++) {
+ if (!rd_strcasecmp(rd_kafka_consumer_group_type_names[i], name))
+ return i;
+ }
+ return RD_KAFKA_CONSUMER_GROUP_TYPE_UNKNOWN;
+}
+
static void rd_kafka_DescribeGroups_resp_cb(rd_kafka_t *rk,
rd_kafka_broker_t *rkb,
rd_kafka_resp_err_t err,
@@ -4757,8 +4967,8 @@ static void rd_kafka_DescribeGroups_resp_cb(rd_kafka_t *rk,
goto err;
}
+ gi->broker.id = rkb->rkb_nodeid;
rd_kafka_broker_lock(rkb);
- gi->broker.id = rkb->rkb_nodeid;
gi->broker.host = rd_strdup(rkb->rkb_origname);
gi->broker.port = rkb->rkb_port;
rd_kafka_broker_unlock(rkb);
@@ -4957,6 +5167,9 @@ rd_kafka_list_groups(rd_kafka_t *rk,
/* Query each broker for its list of groups */
rd_kafka_rdlock(rk);
TAILQ_FOREACH(rkb, &rk->rk_brokers, rkb_link) {
+ if (rd_kafka_broker_or_instance_terminating(rkb))
+ continue;
+
rd_kafka_error_t *error;
rd_kafka_broker_lock(rkb);
if (rkb->rkb_nodeid == -1 || RD_KAFKA_BROKER_IS_LOGICAL(rkb)) {
@@ -4968,7 +5181,7 @@ rd_kafka_list_groups(rd_kafka_t *rk,
state.wait_cnt++;
rkb_cnt++;
error = rd_kafka_ListGroupsRequest(
- rkb, 0, NULL, 0, RD_KAFKA_REPLYQ(state.q, 0),
+ rkb, 0, NULL, 0, NULL, 0, RD_KAFKA_REPLYQ(state.q, 0),
rd_kafka_ListGroups_resp_cb, &state);
if (error) {
rd_kafka_ListGroups_resp_cb(rk, rkb,
@@ -5068,13 +5281,8 @@ const char *rd_kafka_get_debug_contexts(void) {
int rd_kafka_path_is_dir(const char *path) {
-#ifdef _WIN32
- struct _stat st;
- return (_stat(path, &st) == 0 && st.st_mode & S_IFDIR);
-#else
- struct stat st;
- return (stat(path, &st) == 0 && S_ISDIR(st.st_mode));
-#endif
+ rd_bool_t is_dir;
+ return rd_file_stat(path, &is_dir) && is_dir;
}
@@ -5226,7 +5434,7 @@ void rd_kafka_Uuid_destroy(rd_kafka_Uuid_t *uuid) {
*
* @remark Must be freed after use.
*/
-const char *rd_kafka_Uuid_str(const rd_kafka_Uuid_t *uuid) {
+char *rd_kafka_Uuid_str(const rd_kafka_Uuid_t *uuid) {
int i, j;
unsigned char bytes[16];
char *ret = rd_calloc(37, sizeof(*ret));
diff --git a/src/rdkafka.h b/src/rdkafka.h
index 7d4ae8112b..3565b1c5a8 100644
--- a/src/rdkafka.h
+++ b/src/rdkafka.h
@@ -167,7 +167,7 @@ typedef SSIZE_T ssize_t;
* @remark This value should only be used during compile time,
* for runtime checks of version use rd_kafka_version()
*/
-#define RD_KAFKA_VERSION 0x020500ff
+#define RD_KAFKA_VERSION 0x020c01ff
/**
* @brief Returns the librdkafka version as integer.
@@ -263,6 +263,8 @@ typedef struct rd_kafka_headers_s rd_kafka_headers_t;
typedef struct rd_kafka_group_result_s rd_kafka_group_result_t;
typedef struct rd_kafka_acl_result_s rd_kafka_acl_result_t;
typedef struct rd_kafka_Uuid_s rd_kafka_Uuid_t;
+typedef struct rd_kafka_topic_partition_result_s
+ rd_kafka_topic_partition_result_t;
/* @endcond */
@@ -286,7 +288,7 @@ typedef enum {
RD_KAFKA_RESP_ERR__BAD_MSG = -199,
/** Bad/unknown compression */
RD_KAFKA_RESP_ERR__BAD_COMPRESSION = -198,
- /** Broker is going away */
+ /** Broker is going away, together with client instance */
RD_KAFKA_RESP_ERR__DESTROY = -197,
/** Generic failure */
RD_KAFKA_RESP_ERR__FAIL = -196,
@@ -410,6 +412,8 @@ typedef enum {
/** A different record in the batch was invalid
* and this message failed persisting. */
RD_KAFKA_RESP_ERR__INVALID_DIFFERENT_RECORD = -138,
+ /** Broker is going away but client isn't terminating */
+ RD_KAFKA_RESP_ERR__DESTROY_BROKER = -137,
/** End internal error codes */
RD_KAFKA_RESP_ERR__END = -100,
@@ -652,6 +656,9 @@ typedef enum {
/** Client sent a push telemetry request larger than the maximum size
* the broker will accept. */
RD_KAFKA_RESP_ERR_TELEMETRY_TOO_LARGE = 118,
+ /** Client metadata is stale,
+ * client should rebootstrap to obtain new metadata. */
+ RD_KAFKA_RESP_ERR_REBOOTSTRAP_REQUIRED = 129,
RD_KAFKA_RESP_ERR_END_ALL,
} rd_kafka_resp_err_t;
@@ -3224,7 +3231,7 @@ void *rd_kafka_topic_opaque(const rd_kafka_topic_t *rkt);
* The unassigned partition is used by the producer API for messages
* that should be partitioned using the configured or default partitioner.
*/
-#define RD_KAFKA_PARTITION_UA ((int32_t)-1)
+#define RD_KAFKA_PARTITION_UA ((int32_t) - 1)
/**
@@ -4590,11 +4597,37 @@ rd_kafka_consumer_group_metadata_new_with_genid(const char *group_id,
const char *member_id,
const char *group_instance_id);
+/**
+ * @brief Get group id of a group metadata.
+ *
+ * @param group_metadata The group metadata.
+ *
+ * @returns The group id contained in the passed \p group_metadata.
+ *
+ * @remark The returned pointer has the same lifetime as \p group_metadata.
+ */
+RD_EXPORT
+const char *rd_kafka_consumer_group_metadata_group_id(
+ const rd_kafka_consumer_group_metadata_t *group_metadata);
+
+/**
+ * @brief Get group instance id of a group metadata.
+ *
+ * @param group_metadata The group metadata.
+ *
+ * @returns The group instance id contained in the passed \p group_metadata
+ * or NULL.
+ *
+ * @remark The returned pointer has the same lifetime as \p group_metadata.
+ */
+RD_EXPORT
+const char *rd_kafka_consumer_group_metadata_group_instance_id(
+ const rd_kafka_consumer_group_metadata_t *group_metadata);
/**
* @brief Get member id of a group metadata.
*
- * @param group_metadata The group metadata
+ * @param group_metadata The group metadata.
*
* @returns The member id contained in the passed \p group_metadata.
*
@@ -4604,6 +4637,18 @@ RD_EXPORT
const char *rd_kafka_consumer_group_metadata_member_id(
const rd_kafka_consumer_group_metadata_t *group_metadata);
+/**
+ * @brief Get the generation id (classic protocol)
+ * or member epoch (consumer protocol) of a group metadata.
+ *
+ * @param group_metadata The group metadata.
+ *
+ * @returns The generation id or member epoch
+ * contained in the passed \p group_metadata.
+ */
+RD_EXPORT
+int32_t rd_kafka_consumer_group_metadata_generation_id(
+ const rd_kafka_consumer_group_metadata_t *group_metadata);
/**
* @brief Frees the consumer group metadata object as returned by
@@ -5158,6 +5203,18 @@ typedef enum {
RD_KAFKA_CONSUMER_GROUP_STATE__CNT
} rd_kafka_consumer_group_state_t;
+/**
+ * @enum rd_kafka_consumer_group_type_t
+ *
+ * @brief Consumer group type.
+ */
+typedef enum {
+ RD_KAFKA_CONSUMER_GROUP_TYPE_UNKNOWN = 0,
+ RD_KAFKA_CONSUMER_GROUP_TYPE_CONSUMER = 1,
+ RD_KAFKA_CONSUMER_GROUP_TYPE_CLASSIC = 2,
+ RD_KAFKA_CONSUMER_GROUP_TYPE__CNT
+} rd_kafka_consumer_group_type_t;
+
/**
* @brief Group information
*/
@@ -5242,6 +5299,30 @@ RD_EXPORT
rd_kafka_consumer_group_state_t
rd_kafka_consumer_group_state_code(const char *name);
+/**
+ * @brief Returns a name for a group type code.
+ *
+ * @param type The group type value.
+ *
+ * @return The group type name corresponding to the provided group type value.
+ */
+RD_EXPORT
+const char *
+rd_kafka_consumer_group_type_name(rd_kafka_consumer_group_type_t type);
+
+/**
+ * @brief Returns a code for a group type name.
+ *
+ * @param name The group type name.
+ *
+ * @return The group type value corresponding to the provided group type name.
+ *
+ * @remark The comparison is case-insensitive.
+ */
+RD_EXPORT
+rd_kafka_consumer_group_type_t
+rd_kafka_consumer_group_type_code(const char *name);
+
/**
* @brief Release list memory
*/
@@ -5295,6 +5376,18 @@ void rd_kafka_group_list_destroy(const struct rd_kafka_group_list *grplist);
RD_EXPORT
int rd_kafka_brokers_add(rd_kafka_t *rk, const char *brokerlist);
+/**
+ * @brief Retrieve and return the learned broker ids.
+ *
+ * @param rk Instance to use.
+ * @param cntp Will be updated to the number of brokers returned.
+ *
+ * @returns a malloc:ed list of int32_t broker ids.
+ *
+ * @remark The returned pointer must be freed.
+ */
+RD_EXPORT
+int32_t *rd_kafka_brokers_learned_ids(rd_kafka_t *rk, size_t *cntp);
/**
@@ -5515,6 +5608,8 @@ typedef int rd_kafka_event_type_t;
#define RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT 0x200000
/** ListOffsets_result_t */
#define RD_KAFKA_EVENT_LISTOFFSETS_RESULT 0x400000
+/** ElectLeaders_result_t */
+#define RD_KAFKA_EVENT_ELECTLEADERS_RESULT 0x800000
/**
* @returns the event type for the given event.
@@ -5673,6 +5768,7 @@ int rd_kafka_event_error_is_fatal(rd_kafka_event_t *rkev);
* - RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT
* - RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT
* - RD_KAFKA_EVENT_LISTOFFSETS_RESULT
+ * - RD_KAFKA_EVENT_ELECTLEADERS_RESULT
*/
RD_EXPORT
void *rd_kafka_event_opaque(rd_kafka_event_t *rkev);
@@ -5796,6 +5892,8 @@ typedef rd_kafka_event_t rd_kafka_DescribeUserScramCredentials_result_t;
typedef rd_kafka_event_t rd_kafka_AlterUserScramCredentials_result_t;
/*! ListOffsets result type */
typedef rd_kafka_event_t rd_kafka_ListOffsets_result_t;
+/*! ElectLeaders result type */
+typedef rd_kafka_event_t rd_kafka_ElectLeaders_result_t;
/**
* @brief Get CreateTopics result.
@@ -6068,6 +6166,21 @@ rd_kafka_event_DescribeUserScramCredentials_result(rd_kafka_event_t *rkev);
RD_EXPORT const rd_kafka_AlterUserScramCredentials_result_t *
rd_kafka_event_AlterUserScramCredentials_result(rd_kafka_event_t *rkev);
+/**
+ * @brief Get ElectLeaders result.
+ *
+ * @returns the result of a ElectLeaders request, or NULL if
+ * event is of different type.
+ *
+ * @remark The lifetime of the returned memory is the same
+ * as the lifetime of the \p rkev object.
+ *
+ * Event types:
+ * RD_KAFKA_EVENT_ELECTLEADERS_RESULT
+ */
+RD_EXPORT const rd_kafka_ElectLeaders_result_t *
+rd_kafka_event_ElectLeaders_result(rd_kafka_event_t *rkev);
+
/**
* @brief Poll a queue for an event for max \p timeout_ms.
*
@@ -6904,6 +7017,30 @@ rd_kafka_group_result_name(const rd_kafka_group_result_t *groupres);
RD_EXPORT const rd_kafka_topic_partition_list_t *
rd_kafka_group_result_partitions(const rd_kafka_group_result_t *groupres);
+/**
+ * @brief Topic Partition Result provides per-topic+partition operation result
+ * Consists of TopicPartition object and error object.
+ */
+
+/**
+ * @returns the topic partition object from the topic partition result object.
+ * @remarks lifetime of the returned string is the same as the \p
+ * partition_result.
+ * The error object is set inside the topic partition object. For the
+ * detailed error information, use
+ * rd_kafka_topic_partition_result_error()
+ */
+RD_EXPORT const rd_kafka_topic_partition_t *
+rd_kafka_topic_partition_result_partition(
+ const rd_kafka_topic_partition_result_t *partition_result);
+
+/**
+ * @returns the error object from the topic partition result object.
+ * @remarks lifetime of the returned string is the same as the \p
+ * partition_result.
+ */
+RD_EXPORT const rd_kafka_error_t *rd_kafka_topic_partition_result_error(
+ const rd_kafka_topic_partition_result_t *partition_result);
/**@}*/
@@ -6980,6 +7117,7 @@ typedef enum rd_kafka_admin_op_t {
RD_KAFKA_ADMIN_OP_DESCRIBETOPICS, /**< DescribeTopics */
RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER, /**< DescribeCluster */
RD_KAFKA_ADMIN_OP_LISTOFFSETS, /**< ListOffsets */
+ RD_KAFKA_ADMIN_OP_ELECTLEADERS, /**< ElectLeaders */
RD_KAFKA_ADMIN_OP__CNT /**< Number of ops defined */
} rd_kafka_admin_op_t;
@@ -7208,6 +7346,24 @@ rd_kafka_error_t *rd_kafka_AdminOptions_set_match_consumer_group_states(
const rd_kafka_consumer_group_state_t *consumer_group_states,
size_t consumer_group_states_cnt);
+/**
+ * @brief Set consumer groups types to query for.
+ *
+ * @param options Admin options.
+ * @param consumer_group_types Array of consumer group types.
+ * @param consumer_group_types_cnt Size of the \p consumer_group_types array.
+ *
+ * @return NULL on success, a new error instance that must be
+ * released with rd_kafka_error_destroy() in case of error.
+ *
+ * @remark This option is valid for ListConsumerGroups.
+ */
+RD_EXPORT
+rd_kafka_error_t *rd_kafka_AdminOptions_set_match_consumer_group_types(
+ rd_kafka_AdminOptions_t *options,
+ const rd_kafka_consumer_group_type_t *consumer_group_types,
+ size_t consumer_group_types_cnt);
+
/**
* @brief Set Isolation Level to an allowed `rd_kafka_IsolationLevel_t` value.
*/
@@ -7631,6 +7787,8 @@ typedef enum rd_kafka_ConfigSource_t {
/** Built-in default configuration for configs that have a
* default value */
RD_KAFKA_CONFIG_SOURCE_DEFAULT_CONFIG = 5,
+ /** Group config that is configured for a specific group */
+ RD_KAFKA_CONFIG_SOURCE_GROUP_CONFIG = 8,
/** Number of source types defined */
RD_KAFKA_CONFIG_SOURCE__CNT,
@@ -7719,12 +7877,13 @@ rd_kafka_ConfigEntry_synonyms(const rd_kafka_ConfigEntry_t *entry,
* @brief Apache Kafka resource types
*/
typedef enum rd_kafka_ResourceType_t {
- RD_KAFKA_RESOURCE_UNKNOWN = 0, /**< Unknown */
- RD_KAFKA_RESOURCE_ANY = 1, /**< Any (used for lookups) */
- RD_KAFKA_RESOURCE_TOPIC = 2, /**< Topic */
- RD_KAFKA_RESOURCE_GROUP = 3, /**< Group */
- RD_KAFKA_RESOURCE_BROKER = 4, /**< Broker */
- RD_KAFKA_RESOURCE__CNT, /**< Number of resource types defined */
+ RD_KAFKA_RESOURCE_UNKNOWN = 0, /**< Unknown */
+ RD_KAFKA_RESOURCE_ANY = 1, /**< Any (used for lookups) */
+ RD_KAFKA_RESOURCE_TOPIC = 2, /**< Topic */
+ RD_KAFKA_RESOURCE_GROUP = 3, /**< Group */
+ RD_KAFKA_RESOURCE_BROKER = 4, /**< Broker */
+ RD_KAFKA_RESOURCE_TRANSACTIONAL_ID = 5, /**< Transactional ID */
+ RD_KAFKA_RESOURCE__CNT, /**< Number of resource types defined */
} rd_kafka_ResourceType_t;
/**
@@ -8532,6 +8691,17 @@ RD_EXPORT
rd_kafka_consumer_group_state_t rd_kafka_ConsumerGroupListing_state(
const rd_kafka_ConsumerGroupListing_t *grplist);
+/**
+ * @brief Gets type for the \p grplist group.
+ *
+ * @param grplist The group listing.
+ *
+ * @return A group type.
+ */
+RD_EXPORT
+rd_kafka_consumer_group_type_t rd_kafka_ConsumerGroupListing_type(
+ const rd_kafka_ConsumerGroupListing_t *grplist);
+
/**
* @brief Get an array of valid list groups from a ListConsumerGroups result.
*
@@ -8730,6 +8900,17 @@ RD_EXPORT
const rd_kafka_Node_t *rd_kafka_ConsumerGroupDescription_coordinator(
const rd_kafka_ConsumerGroupDescription_t *grpdesc);
+/**
+ * @brief Gets type for the \p grpdesc group.
+ *
+ * @param grpdesc The group description.
+ *
+ * @return A group type.
+ */
+RD_EXPORT
+rd_kafka_consumer_group_type_t rd_kafka_ConsumerGroupDescription_type(
+ const rd_kafka_ConsumerGroupDescription_t *grpdesc);
+
/**
* @brief Gets the members count of \p grpdesc group.
*
@@ -8842,6 +9023,21 @@ RD_EXPORT
const rd_kafka_topic_partition_list_t *rd_kafka_MemberAssignment_partitions(
const rd_kafka_MemberAssignment_t *assignment);
+/**
+ * @brief Gets target assignment of \p member.
+ *
+ * @param member The group member.
+ *
+ * @return The target assignment for `consumer` group types.
+ * Returns NULL for the `classic` group types.
+ *
+ * @remark The lifetime of the returned memory is the same
+ * as the lifetime of the \p member object.
+ */
+RD_EXPORT
+const rd_kafka_MemberAssignment_t *rd_kafka_MemberDescription_target_assignment(
+ const rd_kafka_MemberDescription_t *member);
+
/**@}*/
/**
@@ -9850,6 +10046,100 @@ RD_EXPORT void rd_kafka_DeleteAcls(rd_kafka_t *rk,
/**@}*/
+/**
+ * @name Admin API - Elect Leaders
+ * @{
+ *
+ *
+ *
+ */
+
+/**
+ * @brief Represents elect leaders request.
+ */
+typedef struct rd_kafka_ElectLeaders_s rd_kafka_ElectLeaders_t;
+
+/**
+ * @enum rd_kafka_ElectionType_t
+ * @brief Apache Kafka Election Types
+ */
+typedef enum rd_kafka_ElectionType_t {
+ RD_KAFKA_ELECTION_TYPE_PREFERRED = 0, /**< Preferred Replica Election */
+ RD_KAFKA_ELECTION_TYPE_UNCLEAN = 1, /**< Unclean Election */
+} rd_kafka_ElectionType_t;
+
+/**
+ * @brief Create a new rd_kafka_ElectLeaders_t object. This object is later
+ * passed to rd_kafka_ElectLeaders().
+ *
+ * @param election_type The election type that needs to be performed,
+ * preferred or unclean.
+ * @param partitions The topic partitions for which the leader election
+ * needs to be performed.
+ *
+ * @returns a new allocated elect leaders object or returns NULL in case
+ * of invalid election_type.
+ * Use rd_kafka_ElectLeaders_destroy() to free object when done.
+ */
+RD_EXPORT rd_kafka_ElectLeaders_t *
+rd_kafka_ElectLeaders_new(rd_kafka_ElectionType_t election_type,
+ rd_kafka_topic_partition_list_t *partitions);
+
+/**
+ * @brief Destroy and free a rd_kafka_ElectLeaders_t object previously created
+ * with rd_kafka_ElectLeaders_new()
+ *
+ * @param elect_leaders The rd_kafka_ElectLeaders_t object to be destroyed.
+ */
+RD_EXPORT void
+rd_kafka_ElectLeaders_destroy(rd_kafka_ElectLeaders_t *elect_leaders);
+
+/**
+ * @brief Elect Leaders for the provided Topic Partitions
+ * according to the specified election type.
+ *
+ * @param rk Client instance.
+ * @param elect_leaders The elect leaders request containing
+ * election type and partitions information.
+ * @param options Optional admin options, or NULL for defaults.
+ * @param rkqu Queue to emit result on.
+ *
+ * Supported admin options:
+ * - rd_kafka_AdminOptions_set_operation_timeout() - default 60 seconds.
+ * Controls how long the brokers will wait for records to be deleted.
+ * - rd_kafka_AdminOptions_set_request_timeout() - default socket.timeout.ms.
+ * Controls how long \c rdkafka will wait for the request to complete.
+ *
+ * @remark The result event type emitted on the supplied queue is of type
+ * \c RD_KAFKA_EVENT_ELECTLEADERS_RESULT
+ * @remark If we are passing partitions as NULL, then the broker
+ * will attempt leader election for all partitions, but the results
+ * will contain only partitions for which there was an election or
+ * resulted in an error.
+ */
+RD_EXPORT void rd_kafka_ElectLeaders(rd_kafka_t *rk,
+ rd_kafka_ElectLeaders_t *elect_leaders,
+ const rd_kafka_AdminOptions_t *options,
+ rd_kafka_queue_t *rkqu);
+
+/**
+ * @brief Get the array of topic partition result objects from the
+ * elect leaders result event and populates the size of the
+ * array in \p cntp.
+ *
+ * @param result The elect leaders result.
+ * @param cntp The number of elements in the array.
+ *
+ * @returns the array of topic partition result objects from the
+ * elect leaders result event.
+ */
+RD_EXPORT const rd_kafka_topic_partition_result_t **
+rd_kafka_ElectLeaders_result_partitions(
+ const rd_kafka_ElectLeaders_result_t *result,
+ size_t *cntp);
+
+/**@}*/
+
/**
* @name Security APIs
* @{
diff --git a/src/rdkafka_admin.c b/src/rdkafka_admin.c
index 6909814d79..b2671f3c82 100644
--- a/src/rdkafka_admin.c
+++ b/src/rdkafka_admin.c
@@ -231,11 +231,12 @@ static const char *rd_kafka_admin_state_desc[] = {
* @enum Admin request target broker. Must be negative values since the field
* used is broker_id.
*/
-enum { RD_KAFKA_ADMIN_TARGET_CONTROLLER = -1, /**< Cluster controller */
- RD_KAFKA_ADMIN_TARGET_COORDINATOR = -2, /**< (Group) Coordinator */
- RD_KAFKA_ADMIN_TARGET_FANOUT = -3, /**< This rko is a fanout and
- * and has no target broker */
- RD_KAFKA_ADMIN_TARGET_ALL = -4, /**< All available brokers */
+enum {
+ RD_KAFKA_ADMIN_TARGET_CONTROLLER = -1, /**< Cluster controller */
+ RD_KAFKA_ADMIN_TARGET_COORDINATOR = -2, /**< (Group) Coordinator */
+ RD_KAFKA_ADMIN_TARGET_FANOUT = -3, /**< This rko is a fanout and
+ * and has no target broker */
+ RD_KAFKA_ADMIN_TARGET_ALL = -4, /**< All available brokers */
};
/**
@@ -381,6 +382,8 @@ static rd_kafka_op_t *rd_kafka_admin_result_new(rd_kafka_op_t *rko_req) {
rko_result->rko_evtype = rko_req->rko_u.admin_request.reply_event_type;
+ rko_result->rko_u.admin_result.cbs = rko_req->rko_u.admin_request.cbs;
+
return rko_result;
}
@@ -495,8 +498,8 @@ rd_kafka_admin_coord_request(rd_kafka_broker_t *rkb,
rkb, &rko->rko_u.admin_request.args,
&rko->rko_u.admin_request.options, errstr, sizeof(errstr), replyq,
rd_kafka_admin_handle_response, eonce);
+
if (err) {
- rd_kafka_enq_once_del_source(eonce, "coordinator response");
rd_kafka_admin_result_fail(
rko, err, "%s worker failed to send request: %s",
rd_kafka_op2str(rko->rko_type), errstr);
@@ -1511,8 +1514,9 @@ rd_kafka_admin_MetadataRequest(rd_kafka_broker_t *rkb,
rd_false /* No admin operation requires topic creation. */,
include_cluster_authorized_operations,
include_topic_authorized_operations,
- rd_false /* No admin operation should update cgrp. */, force_racks,
- resp_cb, replyq,
+ rd_false /* No admin operation should update cgrp. */,
+ -1 /* No subscription version is used */, force_racks, resp_cb,
+ replyq,
rd_true /* Admin operation metadata requests are always forced. */,
opaque);
}
@@ -1647,6 +1651,60 @@ rd_kafka_error_t *rd_kafka_AdminOptions_set_match_consumer_group_states(
return !err ? NULL : rd_kafka_error_new(err, "%s", errstr);
}
+rd_kafka_error_t *rd_kafka_AdminOptions_set_match_consumer_group_types(
+ rd_kafka_AdminOptions_t *options,
+ const rd_kafka_consumer_group_type_t *consumer_group_types,
+ size_t consumer_group_types_cnt) {
+ size_t i;
+ char errstr[512];
+ rd_kafka_resp_err_t err;
+ rd_list_t *types_list = rd_list_new(0, NULL);
+ uint64_t types_bitmask = 0;
+
+ rd_list_init_int32(types_list, consumer_group_types_cnt);
+
+ if (RD_KAFKA_CONSUMER_GROUP_TYPE__CNT >= 64) {
+ rd_assert("BUG: cannot handle types with a bitmask anymore");
+ }
+
+ for (i = 0; i < consumer_group_types_cnt; i++) {
+ uint64_t type_bit;
+ rd_kafka_consumer_group_type_t type = consumer_group_types[i];
+
+ if (type < RD_KAFKA_CONSUMER_GROUP_TYPE_UNKNOWN ||
+ type >= RD_KAFKA_CONSUMER_GROUP_TYPE__CNT) {
+ rd_list_destroy(types_list);
+ return rd_kafka_error_new(
+ RD_KAFKA_RESP_ERR__INVALID_ARG,
+ "Only a valid type is allowed");
+ } else if (type == RD_KAFKA_CONSUMER_GROUP_TYPE_UNKNOWN) {
+ rd_list_destroy(types_list);
+ return rd_kafka_error_new(
+ RD_KAFKA_RESP_ERR__INVALID_ARG,
+ "UNKNOWN type is not allowed");
+ }
+
+ type_bit = 1 << type;
+ if (types_bitmask & type_bit) {
+ rd_list_destroy(types_list);
+ return rd_kafka_error_new(
+ RD_KAFKA_RESP_ERR__INVALID_ARG,
+ "Duplicate types not allowed");
+ } else {
+ types_bitmask = types_bitmask | type_bit;
+ rd_list_set_int32(types_list, (int32_t)i, type);
+ }
+ }
+
+ err = rd_kafka_confval_set_type(&options->match_consumer_group_types,
+ RD_KAFKA_CONFVAL_PTR, types_list,
+ errstr, sizeof(errstr));
+ if (err) {
+ rd_list_destroy(types_list);
+ }
+ return !err ? NULL : rd_kafka_error_new(err, "%s", errstr);
+}
+
void rd_kafka_AdminOptions_set_opaque(rd_kafka_AdminOptions_t *options,
void *opaque) {
rd_kafka_confval_set_type(&options->opaque, RD_KAFKA_CONFVAL_PTR,
@@ -1668,7 +1726,8 @@ static void rd_kafka_AdminOptions_init(rd_kafka_t *rk,
options->for_api == RD_KAFKA_ADMIN_OP_DELETETOPICS ||
options->for_api == RD_KAFKA_ADMIN_OP_CREATEPARTITIONS ||
options->for_api == RD_KAFKA_ADMIN_OP_DELETERECORDS ||
- options->for_api == RD_KAFKA_ADMIN_OP_LISTOFFSETS)
+ options->for_api == RD_KAFKA_ADMIN_OP_LISTOFFSETS ||
+ options->for_api == RD_KAFKA_ADMIN_OP_ELECTLEADERS)
rd_kafka_confval_init_int(&options->operation_timeout,
"operation_timeout", -1, 3600 * 1000,
rk->rk_conf.admin.request_timeout_ms);
@@ -1715,6 +1774,14 @@ static void rd_kafka_AdminOptions_init(rd_kafka_t *rk,
rd_kafka_confval_disable(&options->match_consumer_group_states,
"match_consumer_group_states");
+ if (options->for_api == RD_KAFKA_ADMIN_OP_ANY ||
+ options->for_api == RD_KAFKA_ADMIN_OP_LISTCONSUMERGROUPS)
+ rd_kafka_confval_init_ptr(&options->match_consumer_group_types,
+ "match_consumer_group_types");
+ else
+ rd_kafka_confval_disable(&options->match_consumer_group_types,
+ "match_consumer_group_types");
+
if (options->for_api == RD_KAFKA_ADMIN_OP_ANY ||
options->for_api == RD_KAFKA_ADMIN_OP_LISTOFFSETS)
rd_kafka_confval_init_int(&options->isolation_level,
@@ -1747,6 +1814,16 @@ static void rd_kafka_AdminOptions_copy_to(rd_kafka_AdminOptions_t *dst,
states_list_copy, errstr, sizeof(errstr));
rd_assert(!err);
}
+ if (src->match_consumer_group_types.u.PTR) {
+ char errstr[512];
+ rd_list_t *types_list_copy = rd_list_copy_preallocated(
+ src->match_consumer_group_types.u.PTR, NULL);
+
+ rd_kafka_resp_err_t err = rd_kafka_confval_set_type(
+ &dst->match_consumer_group_types, RD_KAFKA_CONFVAL_PTR,
+ types_list_copy, errstr, sizeof(errstr));
+ rd_assert(!err);
+ }
}
@@ -1770,6 +1847,9 @@ void rd_kafka_AdminOptions_destroy(rd_kafka_AdminOptions_t *options) {
if (options->match_consumer_group_states.u.PTR) {
rd_list_destroy(options->match_consumer_group_states.u.PTR);
}
+ if (options->match_consumer_group_types.u.PTR) {
+ rd_list_destroy(options->match_consumer_group_types.u.PTR);
+ }
rd_free(options);
}
@@ -2075,7 +2155,7 @@ rd_kafka_CreateTopicsResponse_parse(rd_kafka_op_t *rko_req,
* does not maintain ordering unfortunately. */
skel.topic = terr->topic;
orig_pos = rd_list_index(&rko_result->rko_u.admin_result.args,
- &skel, rd_kafka_NewTopic_cmp);
+ &skel, rd_kafka_NewTopic_cmp);
if (orig_pos == -1) {
rd_kafka_topic_result_destroy(terr);
rd_kafka_buf_parse_fail(
@@ -2284,7 +2364,7 @@ rd_kafka_DeleteTopicsResponse_parse(rd_kafka_op_t *rko_req,
* does not maintain ordering unfortunately. */
skel.topic = terr->topic;
orig_pos = rd_list_index(&rko_result->rko_u.admin_result.args,
- &skel, rd_kafka_DeleteTopic_cmp);
+ &skel, rd_kafka_DeleteTopic_cmp);
if (orig_pos == -1) {
rd_kafka_topic_result_destroy(terr);
rd_kafka_buf_parse_fail(
@@ -2569,7 +2649,7 @@ rd_kafka_CreatePartitionsResponse_parse(rd_kafka_op_t *rko_req,
* does not maintain ordering unfortunately. */
skel.topic = terr->topic;
orig_pos = rd_list_index(&rko_result->rko_u.admin_result.args,
- &skel, rd_kafka_NewPartitions_cmp);
+ &skel, rd_kafka_NewPartitions_cmp);
if (orig_pos == -1) {
rd_kafka_topic_result_destroy(terr);
rd_kafka_buf_parse_fail(
@@ -2788,9 +2868,15 @@ rd_kafka_ConfigEntry_synonyms(const rd_kafka_ConfigEntry_t *entry,
const char *rd_kafka_ConfigSource_name(rd_kafka_ConfigSource_t confsource) {
static const char *names[] = {
- "UNKNOWN_CONFIG", "DYNAMIC_TOPIC_CONFIG",
- "DYNAMIC_BROKER_CONFIG", "DYNAMIC_DEFAULT_BROKER_CONFIG",
- "STATIC_BROKER_CONFIG", "DEFAULT_CONFIG",
+ "UNKNOWN_CONFIG",
+ "DYNAMIC_TOPIC_CONFIG",
+ "DYNAMIC_BROKER_CONFIG",
+ "DYNAMIC_DEFAULT_BROKER_CONFIG",
+ "STATIC_BROKER_CONFIG",
+ "DEFAULT_CONFIG",
+ "DYNAMIC_BROKER_LOGGER_CONFIG",
+ "CLIENT_METRICS_CONFIG",
+ "GROUP_CONFIG",
};
if ((unsigned int)confsource >=
@@ -2825,9 +2911,8 @@ const char *rd_kafka_ResourcePatternType_name(
}
const char *rd_kafka_ResourceType_name(rd_kafka_ResourceType_t restype) {
- static const char *names[] = {
- "UNKNOWN", "ANY", "TOPIC", "GROUP", "BROKER",
- };
+ static const char *names[] = {"UNKNOWN", "ANY", "TOPIC",
+ "GROUP", "BROKER", "TRANSACTIONAL_ID"};
if ((unsigned int)restype >= (unsigned int)RD_KAFKA_RESOURCE__CNT)
return "UNSUPPORTED";
@@ -2836,6 +2921,35 @@ const char *rd_kafka_ResourceType_name(rd_kafka_ResourceType_t restype) {
}
+rd_kafka_ConfigResourceType_t
+rd_kafka_ResourceType_to_ConfigResourceType(rd_kafka_ResourceType_t restype) {
+ switch (restype) {
+ case RD_KAFKA_RESOURCE_TOPIC:
+ return RD_KAFKA_CONFIG_RESOURCE_TOPIC;
+ case RD_KAFKA_RESOURCE_BROKER:
+ return RD_KAFKA_CONFIG_RESOURCE_BROKER;
+ case RD_KAFKA_RESOURCE_GROUP:
+ return RD_KAFKA_CONFIG_RESOURCE_GROUP;
+ default:
+ return RD_KAFKA_CONFIG_RESOURCE_UNKNOWN;
+ }
+}
+
+rd_kafka_ResourceType_t rd_kafka_ConfigResourceType_to_ResourceType(
+ rd_kafka_ConfigResourceType_t config_resource_type) {
+ switch (config_resource_type) {
+ case RD_KAFKA_CONFIG_RESOURCE_TOPIC:
+ return RD_KAFKA_RESOURCE_TOPIC;
+ case RD_KAFKA_CONFIG_RESOURCE_BROKER:
+ return RD_KAFKA_RESOURCE_BROKER;
+ case RD_KAFKA_CONFIG_RESOURCE_GROUP:
+ return RD_KAFKA_RESOURCE_GROUP;
+ default:
+ return RD_KAFKA_RESOURCE_UNKNOWN;
+ }
+}
+
+
rd_kafka_ConfigResource_t *
rd_kafka_ConfigResource_new(rd_kafka_ResourceType_t restype,
const char *resname) {
@@ -3100,6 +3214,7 @@ rd_kafka_AlterConfigsResponse_parse(rd_kafka_op_t *rko_req,
int16_t error_code;
rd_kafkap_str_t error_msg;
int8_t res_type;
+ int8_t config_resource_type;
rd_kafkap_str_t kres_name;
char *res_name;
char *this_errstr = NULL;
@@ -3109,11 +3224,14 @@ rd_kafka_AlterConfigsResponse_parse(rd_kafka_op_t *rko_req,
rd_kafka_buf_read_i16(reply, &error_code);
rd_kafka_buf_read_str(reply, &error_msg);
- rd_kafka_buf_read_i8(reply, &res_type);
+ rd_kafka_buf_read_i8(reply, &config_resource_type);
rd_kafka_buf_read_str(reply, &kres_name);
RD_KAFKAP_STR_DUPA(&res_name, &kres_name);
rd_kafka_buf_skip_tags(reply);
+ res_type = rd_kafka_ConfigResourceType_to_ResourceType(
+ config_resource_type);
+
if (error_code) {
if (RD_KAFKAP_STR_IS_NULL(&error_msg) ||
RD_KAFKAP_STR_LEN(&error_msg) == 0)
@@ -3293,6 +3411,7 @@ rd_kafka_IncrementalAlterConfigsResponse_parse(rd_kafka_op_t *rko_req,
int16_t error_code;
rd_kafkap_str_t error_msg;
int8_t res_type;
+ int8_t config_resource_type;
rd_kafkap_str_t kres_name;
char *res_name;
char *this_errstr = NULL;
@@ -3302,11 +3421,14 @@ rd_kafka_IncrementalAlterConfigsResponse_parse(rd_kafka_op_t *rko_req,
rd_kafka_buf_read_i16(reply, &error_code);
rd_kafka_buf_read_str(reply, &error_msg);
- rd_kafka_buf_read_i8(reply, &res_type);
+ rd_kafka_buf_read_i8(reply, &config_resource_type);
rd_kafka_buf_read_str(reply, &kres_name);
RD_KAFKAP_STR_DUPA(&res_name, &kres_name);
rd_kafka_buf_skip_tags(reply);
+ res_type = rd_kafka_ConfigResourceType_to_ResourceType(
+ config_resource_type);
+
if (error_code) {
if (RD_KAFKAP_STR_IS_NULL(&error_msg) ||
RD_KAFKAP_STR_LEN(&error_msg) == 0)
@@ -3563,6 +3685,7 @@ rd_kafka_DescribeConfigsResponse_parse(rd_kafka_op_t *rko_req,
for (i = 0; i < (int)res_cnt; i++) {
int16_t error_code;
rd_kafkap_str_t error_msg;
+ int8_t config_resource_type;
int8_t res_type;
rd_kafkap_str_t kres_name;
char *res_name;
@@ -3574,10 +3697,13 @@ rd_kafka_DescribeConfigsResponse_parse(rd_kafka_op_t *rko_req,
rd_kafka_buf_read_i16(reply, &error_code);
rd_kafka_buf_read_str(reply, &error_msg);
- rd_kafka_buf_read_i8(reply, &res_type);
+ rd_kafka_buf_read_i8(reply, &config_resource_type);
rd_kafka_buf_read_str(reply, &kres_name);
RD_KAFKAP_STR_DUPA(&res_name, &kres_name);
+ res_type = rd_kafka_ConfigResourceType_to_ResourceType(
+ config_resource_type);
+
if (error_code) {
if (RD_KAFKAP_STR_IS_NULL(&error_msg) ||
RD_KAFKAP_STR_LEN(&error_msg) == 0)
@@ -7131,12 +7257,14 @@ const rd_kafka_group_result_t **rd_kafka_ListConsumerGroupOffsets_result_groups(
static rd_kafka_ConsumerGroupListing_t *
rd_kafka_ConsumerGroupListing_new(const char *group_id,
rd_bool_t is_simple_consumer_group,
- rd_kafka_consumer_group_state_t state) {
+ rd_kafka_consumer_group_state_t state,
+ rd_kafka_consumer_group_type_t type) {
rd_kafka_ConsumerGroupListing_t *grplist;
grplist = rd_calloc(1, sizeof(*grplist));
grplist->group_id = rd_strdup(group_id);
grplist->is_simple_consumer_group = is_simple_consumer_group;
grplist->state = state;
+ grplist->type = type;
return grplist;
}
@@ -7150,7 +7278,7 @@ static rd_kafka_ConsumerGroupListing_t *rd_kafka_ConsumerGroupListing_copy(
const rd_kafka_ConsumerGroupListing_t *grplist) {
return rd_kafka_ConsumerGroupListing_new(
grplist->group_id, grplist->is_simple_consumer_group,
- grplist->state);
+ grplist->state, grplist->type);
}
/**
@@ -7187,6 +7315,11 @@ rd_kafka_consumer_group_state_t rd_kafka_ConsumerGroupListing_state(
return grplist->state;
}
+rd_kafka_consumer_group_type_t rd_kafka_ConsumerGroupListing_type(
+ const rd_kafka_ConsumerGroupListing_t *grplist) {
+ return grplist->type;
+}
+
/**
* @brief Create a new ListConsumerGroupsResult object.
*
@@ -7256,11 +7389,16 @@ rd_kafka_admin_ListConsumerGroupsRequest(rd_kafka_broker_t *rkb,
rd_kafka_resp_err_t err;
rd_kafka_error_t *error;
const char **states_str = NULL;
+ const char **types_str = NULL;
int states_str_cnt = 0;
rd_list_t *states =
rd_kafka_confval_get_ptr(&options->match_consumer_group_states);
+ int types_str_cnt = 0;
+ rd_list_t *types =
+ rd_kafka_confval_get_ptr(&options->match_consumer_group_types);
+
- /* Prepare list_options */
+ /* Prepare list_options for consumer group state */
if (states && rd_list_cnt(states) > 0) {
states_str_cnt = rd_list_cnt(states);
states_str = rd_calloc(states_str_cnt, sizeof(*states_str));
@@ -7270,13 +7408,27 @@ rd_kafka_admin_ListConsumerGroupsRequest(rd_kafka_broker_t *rkb,
}
}
+ /* Prepare list_options for consumer group type */
+ if (types && rd_list_cnt(types) > 0) {
+ types_str_cnt = rd_list_cnt(types);
+ types_str = rd_calloc(types_str_cnt, sizeof(*types_str));
+ for (i = 0; i < types_str_cnt; i++) {
+ types_str[i] = rd_kafka_consumer_group_type_name(
+ rd_list_get_int32(types, i));
+ }
+ }
error = rd_kafka_ListGroupsRequest(rkb, -1, states_str, states_str_cnt,
- replyq, resp_cb, opaque);
+ types_str, types_str_cnt, replyq,
+ resp_cb, opaque);
if (states_str) {
rd_free(states_str);
}
+ if (types_str) {
+ rd_free(types_str);
+ }
+
if (error) {
rd_snprintf(errstr, errstr_size, "%s",
rd_kafka_error_string(error));
@@ -7305,7 +7457,8 @@ rd_kafka_ListConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req,
rd_kafka_broker_t *rkb = reply->rkbuf_rkb;
rd_list_t valid, errors;
rd_kafka_ListConsumerGroupsResult_t *list_result;
- char *group_id = NULL, *group_state = NULL, *proto_type = NULL;
+ char *group_id = NULL, *group_state = NULL, *proto_type = NULL,
+ *group_type_str = NULL;
api_version = rd_kafka_buf_ApiVersion(reply);
if (api_version >= 1) {
@@ -7333,17 +7486,22 @@ rd_kafka_ListConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req,
for (i = 0; i < cnt; i++) {
rd_kafkap_str_t GroupId, ProtocolType,
- GroupState = RD_ZERO_INIT;
+ GroupState = RD_ZERO_INIT, GroupType = RD_ZERO_INIT;
rd_kafka_ConsumerGroupListing_t *group_listing;
rd_bool_t is_simple_consumer_group, is_consumer_protocol_type;
rd_kafka_consumer_group_state_t state =
RD_KAFKA_CONSUMER_GROUP_STATE_UNKNOWN;
+ rd_kafka_consumer_group_type_t type =
+ RD_KAFKA_CONSUMER_GROUP_TYPE_UNKNOWN;
rd_kafka_buf_read_str(reply, &GroupId);
rd_kafka_buf_read_str(reply, &ProtocolType);
if (api_version >= 4) {
rd_kafka_buf_read_str(reply, &GroupState);
}
+ if (api_version >= 5) {
+ rd_kafka_buf_read_str(reply, &GroupType);
+ }
rd_kafka_buf_skip_tags(reply);
group_id = RD_KAFKAP_STR_DUP(&GroupId);
@@ -7353,21 +7511,29 @@ rd_kafka_ListConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req,
state = rd_kafka_consumer_group_state_code(group_state);
}
+ if (api_version >= 5) {
+ group_type_str = RD_KAFKAP_STR_DUP(&GroupType);
+ type =
+ rd_kafka_consumer_group_type_code(group_type_str);
+ }
+
is_simple_consumer_group = *proto_type == '\0';
is_consumer_protocol_type =
!strcmp(proto_type, CONSUMER_PROTOCOL_TYPE);
if (is_simple_consumer_group || is_consumer_protocol_type) {
group_listing = rd_kafka_ConsumerGroupListing_new(
- group_id, is_simple_consumer_group, state);
+ group_id, is_simple_consumer_group, state, type);
rd_list_add(&valid, group_listing);
}
rd_free(group_id);
rd_free(group_state);
rd_free(proto_type);
- group_id = NULL;
- group_state = NULL;
- proto_type = NULL;
+ rd_free(group_type_str);
+ group_id = NULL;
+ group_state = NULL;
+ proto_type = NULL;
+ group_type_str = NULL;
}
rd_kafka_buf_skip_tags(reply);
@@ -7378,14 +7544,16 @@ rd_kafka_ListConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req,
rd_free(group_state);
if (proto_type)
rd_free(proto_type);
+ if (group_type_str)
+ rd_free(group_type_str);
if (reply->rkbuf_err) {
error_code = reply->rkbuf_err;
error = rd_kafka_error_new(
error_code,
"Broker [%d"
- "] "
- "ListConsumerGroups response protocol parse failure: %s",
+ "] "
+ "ListConsumerGroups response protocol parse failure: %s",
rd_kafka_broker_id(rkb), rd_kafka_err2str(error_code));
rd_list_add(&errors, error);
}
@@ -7631,7 +7799,8 @@ static rd_kafka_MemberDescription_t *rd_kafka_MemberDescription_new(
const char *consumer_id,
const char *group_instance_id,
const char *host,
- const rd_kafka_topic_partition_list_t *assignment) {
+ const rd_kafka_topic_partition_list_t *assignment,
+ const rd_kafka_topic_partition_list_t *target_assignment) {
rd_kafka_MemberDescription_t *member;
member = rd_calloc(1, sizeof(*member));
member->client_id = rd_strdup(client_id);
@@ -7645,6 +7814,12 @@ static rd_kafka_MemberDescription_t *rd_kafka_MemberDescription_new(
else
member->assignment.partitions =
rd_kafka_topic_partition_list_new(0);
+ if (target_assignment) {
+ member->target_assignment =
+ rd_calloc(1, sizeof(rd_kafka_MemberAssignment_t));
+ member->target_assignment->partitions =
+ rd_kafka_topic_partition_list_copy(target_assignment);
+ }
return member;
}
@@ -7658,9 +7833,10 @@ static rd_kafka_MemberDescription_t *rd_kafka_MemberDescription_new(
*/
static rd_kafka_MemberDescription_t *
rd_kafka_MemberDescription_copy(const rd_kafka_MemberDescription_t *src) {
- return rd_kafka_MemberDescription_new(src->client_id, src->consumer_id,
- src->group_instance_id, src->host,
- src->assignment.partitions);
+ return rd_kafka_MemberDescription_new(
+ src->client_id, src->consumer_id, src->group_instance_id, src->host,
+ src->assignment.partitions,
+ src->target_assignment ? src->target_assignment->partitions : NULL);
}
/**
@@ -7679,11 +7855,14 @@ rd_kafka_MemberDescription_destroy(rd_kafka_MemberDescription_t *member) {
rd_free(member->client_id);
rd_free(member->consumer_id);
rd_free(member->host);
- if (member->group_instance_id != NULL)
- rd_free(member->group_instance_id);
- if (member->assignment.partitions)
- rd_kafka_topic_partition_list_destroy(
- member->assignment.partitions);
+ RD_IF_FREE(member->group_instance_id, rd_free);
+ RD_IF_FREE(member->assignment.partitions,
+ rd_kafka_topic_partition_list_destroy);
+ if (member->target_assignment) {
+ RD_IF_FREE(member->target_assignment->partitions,
+ rd_kafka_topic_partition_list_destroy);
+ rd_free(member->target_assignment);
+ }
rd_free(member);
}
@@ -7721,6 +7900,11 @@ const rd_kafka_topic_partition_list_t *rd_kafka_MemberAssignment_partitions(
return assignment->partitions;
}
+const rd_kafka_MemberAssignment_t *rd_kafka_MemberDescription_target_assignment(
+ const rd_kafka_MemberDescription_t *member) {
+ return member->target_assignment;
+}
+
/**
* @brief Create a new ConsumerGroupDescription object.
@@ -7746,6 +7930,7 @@ rd_kafka_ConsumerGroupDescription_new(
const rd_kafka_AclOperation_t *authorized_operations,
int authorized_operations_cnt,
rd_kafka_consumer_group_state_t state,
+ rd_kafka_consumer_group_type_t type,
const rd_kafka_Node_t *coordinator,
rd_kafka_error_t *error) {
rd_kafka_ConsumerGroupDescription_t *grpdesc;
@@ -7769,6 +7954,7 @@ rd_kafka_ConsumerGroupDescription_new(
authorized_operations, authorized_operations_cnt);
grpdesc->state = state;
+ grpdesc->type = type;
if (coordinator != NULL)
grpdesc->coordinator = rd_kafka_Node_copy(coordinator);
grpdesc->error =
@@ -7791,7 +7977,8 @@ rd_kafka_ConsumerGroupDescription_new_error(const char *group_id,
rd_kafka_error_t *error) {
return rd_kafka_ConsumerGroupDescription_new(
group_id, rd_false, NULL, NULL, NULL, 0,
- RD_KAFKA_CONSUMER_GROUP_STATE_UNKNOWN, NULL, error);
+ RD_KAFKA_CONSUMER_GROUP_STATE_UNKNOWN,
+ RD_KAFKA_CONSUMER_GROUP_TYPE_UNKNOWN, NULL, error);
}
/**
@@ -7807,7 +7994,8 @@ rd_kafka_ConsumerGroupDescription_copy(
grpdesc->group_id, grpdesc->is_simple_consumer_group,
&grpdesc->members, grpdesc->partition_assignor,
grpdesc->authorized_operations, grpdesc->authorized_operations_cnt,
- grpdesc->state, grpdesc->coordinator, grpdesc->error);
+ grpdesc->state, grpdesc->type, grpdesc->coordinator,
+ grpdesc->error);
}
/**
@@ -7879,6 +8067,11 @@ const rd_kafka_Node_t *rd_kafka_ConsumerGroupDescription_coordinator(
return grpdesc->coordinator;
}
+rd_kafka_consumer_group_type_t rd_kafka_ConsumerGroupDescription_type(
+ const rd_kafka_ConsumerGroupDescription_t *grpdesc) {
+ return grpdesc->type;
+}
+
size_t rd_kafka_ConsumerGroupDescription_member_count(
const rd_kafka_ConsumerGroupDescription_t *grpdesc) {
return rd_list_cnt(&grpdesc->members);
@@ -7898,51 +8091,6 @@ static int rd_kafka_DescribeConsumerGroups_cmp(const void *a, const void *b) {
return strcmp(a, b);
}
-/** @brief Merge the DescribeConsumerGroups response from a single broker
- * into the user response list.
- */
-static void rd_kafka_DescribeConsumerGroups_response_merge(
- rd_kafka_op_t *rko_fanout,
- const rd_kafka_op_t *rko_partial) {
- rd_kafka_ConsumerGroupDescription_t *groupres = NULL;
- rd_kafka_ConsumerGroupDescription_t *newgroupres;
- const char *grp = rko_partial->rko_u.admin_result.opaque;
- int orig_pos;
-
- rd_assert(rko_partial->rko_evtype ==
- RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT);
-
- if (!rko_partial->rko_err) {
- /* Proper results.
- * We only send one group per request, make sure it matches */
- groupres =
- rd_list_elem(&rko_partial->rko_u.admin_result.results, 0);
- rd_assert(groupres);
- rd_assert(!strcmp(groupres->group_id, grp));
- newgroupres = rd_kafka_ConsumerGroupDescription_copy(groupres);
- } else {
- /* Op errored, e.g. timeout */
- rd_kafka_error_t *error =
- rd_kafka_error_new(rko_partial->rko_err, NULL);
- newgroupres =
- rd_kafka_ConsumerGroupDescription_new_error(grp, error);
- rd_kafka_error_destroy(error);
- }
-
- /* As a convenience to the application we insert group result
- * in the same order as they were requested. */
- orig_pos = rd_list_index(&rko_fanout->rko_u.admin_request.args, grp,
- rd_kafka_DescribeConsumerGroups_cmp);
- rd_assert(orig_pos != -1);
-
- /* Make sure result is not already set */
- rd_assert(rd_list_elem(&rko_fanout->rko_u.admin_request.fanout.results,
- orig_pos) == NULL);
-
- rd_list_set(&rko_fanout->rko_u.admin_request.fanout.results, orig_pos,
- newgroupres);
-}
-
/**
* @brief Construct and send DescribeConsumerGroupsRequest to \p rkb
@@ -7995,6 +8143,50 @@ static rd_kafka_resp_err_t rd_kafka_admin_DescribeConsumerGroupsRequest(
return RD_KAFKA_RESP_ERR_NO_ERROR;
}
+/**
+ * @brief Construct and send ConsumerGroupDescribeRequest to \p rkb
+ * with the groups (char *) in \p groups, using
+ * \p options.
+ *
+ * The response (unparsed) will be enqueued on \p replyq
+ * for handling by \p resp_cb (with \p opaque passed).
+ *
+ * @returns RD_KAFKA_RESP_ERR_NO_ERROR if the request was enqueued for
+ * transmission, otherwise an error code and errstr will be
+ * updated with a human readable error string.
+ */
+static rd_kafka_resp_err_t
+rd_kafka_admin_ConsumerGroupDescribeRequest(rd_kafka_broker_t *rkb,
+ const rd_list_t *groups /*(char*)*/,
+ rd_kafka_AdminOptions_t *options,
+ char *errstr,
+ size_t errstr_size,
+ rd_kafka_replyq_t replyq,
+ rd_kafka_resp_cb_t *resp_cb,
+ void *opaque) {
+
+ int include_authorized_operations;
+ rd_kafka_resp_err_t err;
+ int groups_cnt = rd_list_cnt(groups);
+ rd_kafka_error_t *error = NULL;
+
+ include_authorized_operations =
+ rd_kafka_confval_get_int(&options->include_authorized_operations);
+
+ error = rd_kafka_ConsumerGroupDescribeRequest(
+ rkb, (char **)groups->rl_elems, groups_cnt,
+ include_authorized_operations, replyq, resp_cb, opaque);
+
+ if (error) {
+ rd_snprintf(errstr, errstr_size, "%s",
+ rd_kafka_error_string(error));
+ err = rd_kafka_error_code(error);
+ rd_kafka_error_destroy(error);
+ return err;
+ }
+
+ return RD_KAFKA_RESP_ERR_NO_ERROR;
+}
/**
* @brief Parse DescribeConsumerGroupsResponse and create ADMIN_RESULT op.
*/
@@ -8029,10 +8221,10 @@ rd_kafka_DescribeConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req,
rd_list_init(&rko_result->rko_u.admin_result.results, cnt,
rd_kafka_ConsumerGroupDescription_free);
- rd_kafka_broker_lock(rkb);
nodeid = rkb->rkb_nodeid;
- host = rd_strdup(rkb->rkb_origname);
- port = rkb->rkb_port;
+ rd_kafka_broker_lock(rkb);
+ host = rd_strdup(rkb->rkb_origname);
+ port = rkb->rkb_port;
rd_kafka_broker_unlock(rkb);
node = rd_kafka_Node_new(nodeid, host, port, NULL);
@@ -8131,9 +8323,13 @@ rd_kafka_DescribeConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req,
client_id = RD_KAFKAP_STR_DUP(&ClientId);
client_host = RD_KAFKAP_STR_DUP(&ClientHost);
+ /* Target Assignment is `NULL` for the `classic`
+ * protocol as there is no concept of Target Assignment
+ * there. */
member = rd_kafka_MemberDescription_new(
client_id, member_id, group_instance_id,
- client_host, partitions);
+ client_host, partitions,
+ NULL /* target assignment */);
if (partitions)
rd_kafka_topic_partition_list_destroy(
partitions);
@@ -8162,7 +8358,7 @@ rd_kafka_DescribeConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req,
group_id, is_simple_consumer_group, &members, proto,
operations, operation_cnt,
rd_kafka_consumer_group_state_code(group_state),
- node, error);
+ RD_KAFKA_CONSUMER_GROUP_TYPE_CLASSIC, node, error);
} else
grpdesc = rd_kafka_ConsumerGroupDescription_new_error(
group_id, error);
@@ -8219,6 +8415,325 @@ rd_kafka_DescribeConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req,
return reply->rkbuf_err;
}
+/**
+ * @brief Parse ConsumerGroupDescriberesponse and create ADMIN_RESULT op.
+ */
+static rd_kafka_resp_err_t
+rd_kafka_ConsumerGroupDescribeResponse_parse(rd_kafka_op_t *rko_req,
+ rd_kafka_op_t **rko_resultp,
+ rd_kafka_buf_t *reply,
+ char *errstr,
+ size_t errstr_size) {
+ const int log_decode_errors = LOG_ERR;
+ int32_t groups_cnt;
+ rd_kafka_op_t *rko_result = NULL;
+ rd_kafka_broker_t *rkb = reply->rkbuf_rkb;
+ rd_kafka_error_t *error = NULL;
+ char *group_id = NULL, *group_state = NULL, *assignor_name = NULL,
+ *host = NULL;
+ rd_kafka_AclOperation_t *operations = NULL;
+ rd_kafka_Node_t *node = NULL;
+ rd_kafka_topic_partition_list_t *assignment = NULL,
+ *target_assignment = NULL;
+ int32_t nodeid;
+ uint16_t port;
+ int operation_cnt = -1;
+ int32_t i;
+
+ rd_kafka_buf_read_throttle_time(reply);
+
+ rd_kafka_buf_read_arraycnt(reply, &groups_cnt, RD_KAFKAP_GROUPS_MAX);
+
+ nodeid = rkb->rkb_nodeid;
+ rd_kafka_broker_lock(rkb);
+ host = rd_strdup(rkb->rkb_origname);
+ port = rkb->rkb_port;
+ rd_kafka_broker_unlock(rkb);
+
+ node = rd_kafka_Node_new(nodeid, host, port, NULL);
+
+ rko_result = rd_kafka_admin_result_new(rko_req);
+ rd_list_init(&rko_result->rko_u.admin_result.results, groups_cnt,
+ rd_kafka_ConsumerGroupDescription_free);
+
+ for (i = 0; i < groups_cnt; i++) {
+ int16_t ErrorCode;
+ int32_t authorized_operations = -1;
+ int32_t MemberCnt, j;
+ int32_t GroupEpoch, AssignmentEpoch;
+ rd_kafkap_str_t GroupId, GroupState, AssignorName, ErrorString;
+ rd_list_t members;
+ rd_kafka_ConsumerGroupDescription_t *grpdesc = NULL;
+
+ rd_kafka_buf_read_i16(reply, &ErrorCode);
+ rd_kafka_buf_read_str(reply, &ErrorString);
+ rd_kafka_buf_read_str(reply, &GroupId);
+ rd_kafka_buf_read_str(reply, &GroupState);
+ rd_kafka_buf_read_i32(reply, &GroupEpoch);
+ rd_kafka_buf_read_i32(reply, &AssignmentEpoch);
+ rd_kafka_buf_read_str(reply, &AssignorName);
+ rd_kafka_buf_read_arraycnt(reply, &MemberCnt, 100000);
+
+ group_id = RD_KAFKAP_STR_DUP(&GroupId);
+ group_state = RD_KAFKAP_STR_DUP(&GroupState);
+ assignor_name = RD_KAFKAP_STR_DUP(&AssignorName);
+
+ if (ErrorCode) {
+ error = rd_kafka_error_new(
+ ErrorCode, "ConsumerGroupDescribe: %.*s",
+ RD_KAFKAP_STR_PR(&ErrorString));
+ }
+
+ rd_list_init(&members, MemberCnt,
+ rd_kafka_MemberDescription_free);
+
+ for (j = 0; j < MemberCnt; j++) {
+ char *member_id = NULL, *instance_id = NULL,
+ *client_id = NULL, *client_host = NULL;
+ rd_kafkap_str_t MemberId, InstanceId, RackId, ClientId,
+ ClientHost, SubscribedTopicRegex;
+ int32_t MemberEpoch, idx;
+ rd_kafka_MemberDescription_t *member;
+ int32_t SubscribedTopicNamesArrayCnt;
+
+ rd_kafka_buf_read_str(reply, &MemberId);
+ rd_kafka_buf_read_str(reply, &InstanceId);
+ rd_kafka_buf_read_str(reply, &RackId);
+ rd_kafka_buf_read_i32(reply, &MemberEpoch);
+ rd_kafka_buf_read_str(reply, &ClientId);
+ rd_kafka_buf_read_str(reply, &ClientHost);
+ rd_kafka_buf_read_arraycnt(
+ reply, &SubscribedTopicNamesArrayCnt, 100000);
+
+ for (idx = 0; idx < SubscribedTopicNamesArrayCnt;
+ idx++) {
+ rd_kafkap_str_t SubscribedTopicName;
+ rd_kafka_buf_read_str(reply,
+ &SubscribedTopicName);
+ }
+ rd_kafka_buf_read_str(reply, &SubscribedTopicRegex);
+ const rd_kafka_topic_partition_field_t fields[] = {
+ RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION,
+ RD_KAFKA_TOPIC_PARTITION_FIELD_END};
+
+ assignment = rd_kafka_buf_read_topic_partitions(
+ reply, rd_true /* use topic_id */,
+ rd_true /* use topic name*/, 0, fields);
+
+ /* Assignment tags */
+ rd_kafka_buf_skip_tags(reply);
+
+ target_assignment = rd_kafka_buf_read_topic_partitions(
+ reply, rd_true /* use topic_id */,
+ rd_true /* use topic name*/, 0, fields);
+
+ /* TargetAssignment tags */
+ rd_kafka_buf_skip_tags(reply);
+
+ /* Member tags */
+ rd_kafka_buf_skip_tags(reply);
+
+ member_id = RD_KAFKAP_STR_DUP(&MemberId);
+ if (!RD_KAFKAP_STR_IS_NULL(&InstanceId)) {
+ instance_id = RD_KAFKAP_STR_DUP(&InstanceId);
+ }
+ client_id = RD_KAFKAP_STR_DUP(&ClientId);
+ client_host = RD_KAFKAP_STR_DUP(&ClientHost);
+
+ member = rd_kafka_MemberDescription_new(
+ client_id, member_id, instance_id, client_host,
+ assignment, target_assignment);
+
+
+ rd_list_add(&members, member);
+
+ RD_IF_FREE(assignment,
+ rd_kafka_topic_partition_list_destroy);
+ RD_IF_FREE(target_assignment,
+ rd_kafka_topic_partition_list_destroy);
+
+ RD_IF_FREE(member_id, rd_free);
+ RD_IF_FREE(instance_id, rd_free);
+ RD_IF_FREE(client_id, rd_free);
+ RD_IF_FREE(client_host, rd_free);
+ member_id = NULL;
+ instance_id = NULL;
+ client_id = NULL;
+ client_host = NULL;
+ }
+ rd_kafka_buf_read_i32(reply, &authorized_operations);
+ operations = rd_kafka_AuthorizedOperations_parse(
+ authorized_operations, &operation_cnt);
+ rd_kafka_buf_skip_tags(reply);
+
+ /* If the error code is Group ID Not Found or Unsupported
+ Version, we will set the ConsumerGroupType to Consumer to
+ identify it for further processing with the old protocol and
+ eventually in rd_kafka_DescribeConsumerGroupsResponse_parse
+ we will set the ConsumerGroupType to Unknown */
+ if (!error) {
+ grpdesc = rd_kafka_ConsumerGroupDescription_new(
+ group_id, rd_false, &members, assignor_name,
+ operations, operation_cnt,
+ rd_kafka_consumer_group_state_code(group_state),
+ RD_KAFKA_CONSUMER_GROUP_TYPE_CONSUMER, node, error);
+ } else {
+ grpdesc = rd_kafka_ConsumerGroupDescription_new_error(
+ group_id, error);
+ }
+
+ rd_list_add(&rko_result->rko_u.admin_result.results, grpdesc);
+
+ rd_list_destroy(&members);
+ rd_free(group_id);
+ rd_free(group_state);
+ rd_free(assignor_name);
+ RD_IF_FREE(error, rd_kafka_error_destroy);
+ RD_IF_FREE(operations, rd_free);
+
+ error = NULL;
+ group_id = NULL;
+ group_state = NULL;
+ assignor_name = NULL;
+ operations = NULL;
+ }
+ rd_kafka_buf_skip_tags(reply);
+ RD_IF_FREE(host, rd_free);
+ RD_IF_FREE(node, rd_kafka_Node_destroy);
+ *rko_resultp = rko_result;
+ return RD_KAFKA_RESP_ERR_NO_ERROR;
+err_parse:
+ RD_IF_FREE(group_id, rd_free);
+ RD_IF_FREE(group_state, rd_free);
+ RD_IF_FREE(assignor_name, rd_free);
+ RD_IF_FREE(host, rd_free);
+ RD_IF_FREE(node, rd_kafka_Node_destroy);
+ RD_IF_FREE(error, rd_kafka_error_destroy);
+ RD_IF_FREE(operations, rd_free);
+ RD_IF_FREE(assignment, rd_kafka_topic_partition_list_destroy);
+ RD_IF_FREE(target_assignment, rd_kafka_topic_partition_list_destroy);
+ RD_IF_FREE(rko_result, rd_kafka_op_destroy);
+
+ rd_snprintf(
+ errstr, errstr_size,
+ "DescribeConsumerGroups response protocol parse failure: %s",
+ rd_kafka_err2str(reply->rkbuf_err));
+ return reply->rkbuf_err;
+}
+
+/**
+ * @brief In case if we get an Unsupported Feature error or if it is a consumer
+ group and we get errors GROUP_ID_NOT_FOUND(69) or
+ UNSUPPORTED_VERSION(35) we need to send a request to the old
+ protocol.
+ */
+static rd_bool_t rd_kafka_admin_describe_consumer_group_do_fallback_to_classic(
+ rd_kafka_ConsumerGroupDescription_t *groupres) {
+ return groupres->error &&
+ (groupres->error->code == RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND ||
+ groupres->error->code ==
+ RD_KAFKA_RESP_ERR_UNSUPPORTED_VERSION ||
+ groupres->error->code ==
+ RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE);
+}
+
+static void rd_kafka_admin_describe_consumer_group_request(
+ rd_kafka_op_t *rko_fanout,
+ rd_kafka_t *rk,
+ const char *group_id,
+ const struct rd_kafka_admin_worker_cbs *cbs,
+ const rd_kafka_AdminOptions_t *options,
+ rd_kafka_q_t *rkq) {
+ rd_kafka_op_t *rko = rd_kafka_admin_request_op_new(
+ rk, RD_KAFKA_OP_DESCRIBECONSUMERGROUPS,
+ RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT, cbs, options, rkq);
+
+ rko->rko_u.admin_request.fanout_parent = rko_fanout;
+ rko->rko_u.admin_request.broker_id = RD_KAFKA_ADMIN_TARGET_COORDINATOR;
+ rko->rko_u.admin_request.coordtype = RD_KAFKA_COORD_GROUP;
+ rko->rko_u.admin_request.coordkey = rd_strdup(group_id);
+
+ /* Set the group name as the opaque so the fanout worker use it
+ * to fill in errors.
+ * References rko_fanout's memory, which will always outlive
+ * the fanned out op. */
+ rd_kafka_AdminOptions_set_opaque(&rko->rko_u.admin_request.options,
+ (void *)group_id);
+
+ rd_list_init(&rko->rko_u.admin_request.args, 1, rd_free);
+ rd_list_add(&rko->rko_u.admin_request.args, rd_strdup(group_id));
+
+ rd_kafka_q_enq(rko_fanout->rko_rk->rk_ops, rko);
+}
+
+/** @brief Merge the DescribeConsumerGroups response from a single broker
+ * into the user response list.
+ */
+static void rd_kafka_DescribeConsumerGroups_response_merge(
+ rd_kafka_op_t *rko_fanout,
+ const rd_kafka_op_t *rko_partial) {
+ rd_kafka_ConsumerGroupDescription_t *groupres = NULL;
+ rd_kafka_ConsumerGroupDescription_t *newgroupres;
+ const char *grp = rko_partial->rko_u.admin_result.opaque;
+ int orig_pos;
+
+ rd_assert(rko_partial->rko_evtype ==
+ RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT);
+
+ if (!rko_partial->rko_err) {
+ /* Proper results.
+ * We only send one group per request, make sure it matches */
+ groupres =
+ rd_list_elem(&rko_partial->rko_u.admin_result.results, 0);
+ rd_assert(groupres);
+ rd_assert(!strcmp(groupres->group_id, grp));
+ newgroupres = rd_kafka_ConsumerGroupDescription_copy(groupres);
+ } else {
+ /* Op errored, e.g. timeout */
+ rd_kafka_error_t *error =
+ rd_kafka_error_new(rko_partial->rko_err, NULL);
+ newgroupres =
+ rd_kafka_ConsumerGroupDescription_new_error(grp, error);
+ rd_kafka_error_destroy(error);
+ }
+
+ rd_bool_t is_consumer_group_response =
+ rko_partial->rko_u.admin_result.cbs->request ==
+ rd_kafka_admin_ConsumerGroupDescribeRequest;
+
+ if (is_consumer_group_response &&
+ rd_kafka_admin_describe_consumer_group_do_fallback_to_classic(
+ newgroupres)) {
+ /* We need to send a request to the old protocol */
+ rko_fanout->rko_u.admin_request.fanout.outstanding++;
+ static const struct rd_kafka_admin_worker_cbs cbs = {
+ rd_kafka_admin_DescribeConsumerGroupsRequest,
+ rd_kafka_DescribeConsumerGroupsResponse_parse,
+ };
+ rd_kafka_admin_describe_consumer_group_request(
+ rko_fanout, rko_fanout->rko_rk, grp, &cbs,
+ &rko_fanout->rko_u.admin_request.options,
+ rko_fanout->rko_rk->rk_ops);
+
+ rd_kafka_ConsumerGroupDescription_destroy(newgroupres);
+ } else {
+ /* As a convenience to the application we insert group result
+ * in the same order as they were requested. */
+ orig_pos =
+ rd_list_index(&rko_fanout->rko_u.admin_request.args, grp,
+ rd_kafka_DescribeConsumerGroups_cmp);
+ rd_assert(orig_pos != -1);
+
+ /* Make sure result is not already set */
+ rd_assert(rd_list_elem(
+ &rko_fanout->rko_u.admin_request.fanout.results,
+ orig_pos) == NULL);
+
+ rd_list_set(&rko_fanout->rko_u.admin_request.fanout.results,
+ orig_pos, newgroupres);
+ }
+}
+
void rd_kafka_DescribeConsumerGroups(rd_kafka_t *rk,
const char **groups,
size_t groups_cnt,
@@ -8288,34 +8803,13 @@ void rd_kafka_DescribeConsumerGroups(rd_kafka_t *rk,
* coordinator into one op. */
for (i = 0; i < groups_cnt; i++) {
static const struct rd_kafka_admin_worker_cbs cbs = {
- rd_kafka_admin_DescribeConsumerGroupsRequest,
- rd_kafka_DescribeConsumerGroupsResponse_parse,
+ rd_kafka_admin_ConsumerGroupDescribeRequest,
+ rd_kafka_ConsumerGroupDescribeResponse_parse,
};
char *grp =
rd_list_elem(&rko_fanout->rko_u.admin_request.args, (int)i);
- rd_kafka_op_t *rko = rd_kafka_admin_request_op_new(
- rk, RD_KAFKA_OP_DESCRIBECONSUMERGROUPS,
- RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT, &cbs, options,
- rk->rk_ops);
-
- rko->rko_u.admin_request.fanout_parent = rko_fanout;
- rko->rko_u.admin_request.broker_id =
- RD_KAFKA_ADMIN_TARGET_COORDINATOR;
- rko->rko_u.admin_request.coordtype = RD_KAFKA_COORD_GROUP;
- rko->rko_u.admin_request.coordkey = rd_strdup(grp);
-
- /* Set the group name as the opaque so the fanout worker use it
- * to fill in errors.
- * References rko_fanout's memory, which will always outlive
- * the fanned out op. */
- rd_kafka_AdminOptions_set_opaque(
- &rko->rko_u.admin_request.options, grp);
-
- rd_list_init(&rko->rko_u.admin_request.args, 1, rd_free);
- rd_list_add(&rko->rko_u.admin_request.args,
- rd_strdup(groups[i]));
-
- rd_kafka_q_enq(rk->rk_ops, rko);
+ rd_kafka_admin_describe_consumer_group_request(
+ rko_fanout, rk, grp, &cbs, options, rk->rk_ops);
}
}
@@ -9009,3 +9503,292 @@ void rd_kafka_DescribeCluster(rd_kafka_t *rk,
}
/**@}*/
+
+/**
+ * @name ElectLeaders
+ * @{
+ *
+ *
+ *
+ *
+ */
+
+/**
+ * @brief Creates a new rd_kafka_ElectLeaders_t object with the given
+ * \p election_type and \p partitions.
+ */
+rd_kafka_ElectLeaders_t *
+rd_kafka_ElectLeaders_new(rd_kafka_ElectionType_t election_type,
+ rd_kafka_topic_partition_list_t *partitions) {
+
+ rd_kafka_ElectLeaders_t *elect_leaders;
+
+ elect_leaders = rd_calloc(1, sizeof(*elect_leaders));
+ if (partitions)
+ elect_leaders->partitions =
+ rd_kafka_topic_partition_list_copy(partitions);
+ elect_leaders->election_type = election_type;
+
+ return elect_leaders;
+}
+
+rd_kafka_ElectLeaders_t *
+rd_kafka_ElectLeaders_copy(const rd_kafka_ElectLeaders_t *elect_leaders) {
+ return rd_kafka_ElectLeaders_new(elect_leaders->election_type,
+ elect_leaders->partitions);
+}
+
+void rd_kafka_ElectLeaders_destroy(rd_kafka_ElectLeaders_t *elect_leaders) {
+ if (elect_leaders->partitions)
+ rd_kafka_topic_partition_list_destroy(
+ elect_leaders->partitions);
+ rd_free(elect_leaders);
+}
+
+static void rd_kafka_ElectLeaders_free(void *ptr) {
+ rd_kafka_ElectLeaders_destroy(ptr);
+}
+
+/**
+ * @brief Creates a new rd_kafka_ElectLeadersResult_t object with the given
+ * \p error and \p partitions.
+ */
+static rd_kafka_ElectLeadersResult_t *
+rd_kafka_ElectLeadersResult_new(rd_list_t *partitions) {
+
+ rd_kafka_ElectLeadersResult_t *result;
+ result = rd_calloc(1, sizeof(*result));
+ rd_list_init_copy(&result->partitions, partitions);
+ rd_list_copy_to(&result->partitions, partitions,
+ rd_kafka_topic_partition_result_copy_opaque, NULL);
+ return result;
+}
+
+static const rd_kafka_topic_partition_result_t **
+rd_kafka_ElectLeadersResult_partitions(
+ const rd_kafka_ElectLeadersResult_t *result,
+ size_t *cntp) {
+ *cntp = rd_list_cnt(&result->partitions);
+ return (const rd_kafka_topic_partition_result_t **)
+ result->partitions.rl_elems;
+}
+
+static void
+rd_kafka_ElectLeadersResult_destroy(rd_kafka_ElectLeadersResult_t *result) {
+ rd_list_destroy(&result->partitions);
+ rd_free(result);
+}
+
+static void rd_kafka_ElectLeadersResult_free(void *ptr) {
+ rd_kafka_ElectLeadersResult_destroy(ptr);
+}
+
+static const rd_kafka_ElectLeadersResult_t *rd_kafka_ElectLeaders_result_result(
+ const rd_kafka_ElectLeaders_result_t *result) {
+ return (const rd_kafka_ElectLeadersResult_t *)rd_list_elem(
+ &result->rko_u.admin_result.results, 0);
+}
+
+const rd_kafka_topic_partition_result_t **
+rd_kafka_ElectLeaders_result_partitions(
+ const rd_kafka_ElectLeaders_result_t *result,
+ size_t *cntp) {
+ return rd_kafka_ElectLeadersResult_partitions(
+ rd_kafka_ElectLeaders_result_result(result), cntp);
+}
+
+/**
+ * @brief Parse ElectLeadersResponse and create ADMIN_RESULT op.
+ */
+static rd_kafka_resp_err_t
+rd_kafka_ElectLeadersResponse_parse(rd_kafka_op_t *rko_req,
+ rd_kafka_op_t **rko_resultp,
+ rd_kafka_buf_t *reply,
+ char *errstr,
+ size_t errstr_size) {
+ const int log_decode_errors = LOG_ERR;
+ rd_kafka_op_t *rko_result = NULL;
+ rd_kafka_ElectLeadersResult_t *result = NULL;
+ int16_t top_level_error_code = 0;
+ int32_t TopicArrayCnt;
+ int partition_cnt;
+ rd_list_t partitions_arr;
+ rd_kafka_ElectLeaders_t *request =
+ rko_req->rko_u.admin_request.args.rl_elems[0];
+ int i;
+ int j;
+
+ rd_kafka_buf_read_throttle_time(reply);
+
+ if (rd_kafka_buf_ApiVersion(reply) >= 1) {
+ rd_kafka_buf_read_i16(reply, &top_level_error_code);
+ }
+
+ if (top_level_error_code) {
+ rd_kafka_admin_result_fail(
+ rko_req, top_level_error_code,
+ "ElectLeaders request failed: %s",
+ rd_kafka_err2str(top_level_error_code));
+ return top_level_error_code;
+ }
+
+ /* #partitions */
+ rd_kafka_buf_read_arraycnt(reply, &TopicArrayCnt, RD_KAFKAP_TOPICS_MAX);
+
+ if (request->partitions)
+ partition_cnt = request->partitions->cnt;
+ else
+ partition_cnt = 1;
+ rd_list_init(&partitions_arr, partition_cnt,
+ rd_kafka_topic_partition_result_free);
+ memset(partitions_arr.rl_elems, 0,
+ sizeof(*partitions_arr.rl_elems) * partition_cnt);
+
+ for (i = 0; i < TopicArrayCnt; i++) {
+ rd_kafka_topic_partition_result_t *partition_result;
+ rd_kafkap_str_t ktopic;
+ char *topic;
+ int32_t PartArrayCnt;
+
+ rd_kafka_buf_read_str(reply, &ktopic);
+ RD_KAFKAP_STR_DUPA(&topic, &ktopic);
+
+ rd_kafka_buf_read_arraycnt(reply, &PartArrayCnt,
+ RD_KAFKAP_PARTITIONS_MAX);
+
+ for (j = 0; j < PartArrayCnt; j++) {
+ int32_t partition;
+ int16_t partition_error_code;
+ rd_kafkap_str_t partition_error_msg;
+ char *partition_errstr;
+ int orig_pos;
+
+ rd_kafka_buf_read_i32(reply, &partition);
+ rd_kafka_buf_read_i16(reply, &partition_error_code);
+ rd_kafka_buf_read_str(reply, &partition_error_msg);
+
+ rd_kafka_buf_skip_tags(reply);
+
+ if (RD_KAFKAP_STR_IS_NULL(&partition_error_msg) ||
+ RD_KAFKAP_STR_LEN(&partition_error_msg) == 0)
+ partition_errstr = (char *)rd_kafka_err2str(
+ partition_error_code);
+ else
+ RD_KAFKAP_STR_DUPA(&partition_errstr,
+ &partition_error_msg);
+
+ partition_result = rd_kafka_topic_partition_result_new(
+ topic, partition, partition_error_code,
+ partition_errstr);
+
+ if (request->partitions) {
+ orig_pos =
+ rd_kafka_topic_partition_list_find_idx(
+ request->partitions, topic, partition);
+
+ if (orig_pos == -1) {
+ rd_kafka_buf_parse_fail(
+ reply,
+ "Broker returned partition %s "
+ "[%" PRId32
+ "] that was not "
+ "included in the original request",
+ topic, partition);
+ }
+
+ if (rd_list_elem(&partitions_arr, orig_pos) !=
+ NULL) {
+ rd_kafka_buf_parse_fail(
+ reply,
+ "Broker returned partition %s "
+ "[%" PRId32 "] multiple times",
+ topic, partition);
+ }
+
+ rd_list_set(&partitions_arr, orig_pos,
+ partition_result);
+ } else {
+ rd_list_add(&partitions_arr, partition_result);
+ }
+ }
+ rd_kafka_buf_skip_tags(reply);
+ }
+
+ rd_kafka_buf_skip_tags(reply);
+
+ result = rd_kafka_ElectLeadersResult_new(&partitions_arr);
+
+ rko_result = rd_kafka_admin_result_new(rko_req);
+
+ rd_list_init(&rko_result->rko_u.admin_result.results, 1,
+ rd_kafka_ElectLeadersResult_free);
+
+ rd_list_add(&rko_result->rko_u.admin_result.results, result);
+
+ *rko_resultp = rko_result;
+
+ rd_list_destroy(&partitions_arr);
+
+ return RD_KAFKA_RESP_ERR_NO_ERROR;
+err_parse:
+
+ rd_list_destroy(&partitions_arr);
+
+ if (rko_result)
+ rd_kafka_op_destroy(rko_result);
+
+ rd_snprintf(errstr, errstr_size,
+ "ElectLeaders response protocol parse failure: %s",
+ rd_kafka_err2str(reply->rkbuf_err));
+
+ return reply->rkbuf_err;
+}
+
+void rd_kafka_ElectLeaders(rd_kafka_t *rk,
+ rd_kafka_ElectLeaders_t *elect_leaders,
+ const rd_kafka_AdminOptions_t *options,
+ rd_kafka_queue_t *rkqu) {
+ rd_kafka_op_t *rko;
+ rd_kafka_topic_partition_list_t *copied_partitions = NULL;
+
+ static const struct rd_kafka_admin_worker_cbs cbs = {
+ rd_kafka_ElectLeadersRequest,
+ rd_kafka_ElectLeadersResponse_parse,
+ };
+
+ rd_assert(rkqu);
+
+ rko = rd_kafka_admin_request_op_new(rk, RD_KAFKA_OP_ELECTLEADERS,
+ RD_KAFKA_EVENT_ELECTLEADERS_RESULT,
+ &cbs, options, rkqu->rkqu_q);
+
+ if (elect_leaders->partitions) {
+ /* Duplicate topic partitions should not be present in the list
+ */
+ copied_partitions = rd_kafka_topic_partition_list_copy(
+ elect_leaders->partitions);
+ if (rd_kafka_topic_partition_list_has_duplicates(
+ copied_partitions, rd_false /* check partition*/)) {
+ rd_kafka_admin_result_fail(
+ rko, RD_KAFKA_RESP_ERR__INVALID_ARG,
+ "Duplicate partitions specified");
+ rd_kafka_admin_common_worker_destroy(
+ rk, rko, rd_true /*destroy*/);
+ rd_kafka_topic_partition_list_destroy(
+ copied_partitions);
+ return;
+ }
+ }
+
+ rd_list_init(&rko->rko_u.admin_request.args, 1,
+ rd_kafka_ElectLeaders_free);
+
+ rd_list_add(&rko->rko_u.admin_request.args,
+ rd_kafka_ElectLeaders_copy(elect_leaders));
+
+ rd_kafka_q_enq(rk->rk_ops, rko);
+ if (copied_partitions)
+ rd_kafka_topic_partition_list_destroy(copied_partitions);
+}
+
+/**@}*/
diff --git a/src/rdkafka_admin.h b/src/rdkafka_admin.h
index 62b2e7244c..c84849ea66 100644
--- a/src/rdkafka_admin.h
+++ b/src/rdkafka_admin.h
@@ -35,6 +35,11 @@
#include "rdmap.h"
#include "rdkafka_error.h"
#include "rdkafka_confval.h"
+
+#if WITH_SSL && OPENSSL_VERSION_NUMBER >= 0x10101000L
+#include
+#endif
+
#if WITH_SSL
typedef struct rd_kafka_broker_s rd_kafka_broker_t;
extern int rd_kafka_ssl_hmac(rd_kafka_broker_t *rkb,
@@ -109,6 +114,12 @@ struct rd_kafka_AdminOptions_s {
* Valid for: ListConsumerGroups.
*/
+ rd_kafka_confval_t
+ match_consumer_group_types; /**< PTR: list of consumer group types
+ * to query for.
+ * Valid for: ListConsumerGroups.
+ */
+
rd_kafka_confval_t
isolation_level; /**< INT:Isolation Level needed for list Offset
* to query for.
@@ -272,6 +283,38 @@ struct rd_kafka_ConfigResource_result_s {
* but with response error values. */
};
+/**
+ * @brief Resource type specific to config apis.
+ */
+typedef enum rd_kafka_ConfigResourceType_t {
+ RD_KAFKA_CONFIG_RESOURCE_UNKNOWN = 0,
+ RD_KAFKA_CONFIG_RESOURCE_TOPIC = 2,
+ RD_KAFKA_CONFIG_RESOURCE_BROKER = 4,
+ RD_KAFKA_CONFIG_RESOURCE_GROUP = 32,
+} rd_kafka_ConfigResourceType_t;
+
+/**
+ * @brief Maps `rd_kafka_ResourceType_t` to `rd_kafka_ConfigResourceType_t`
+ * for Config Apis. We are incorrectly using `rd_kafka_ResourceType_t` in
+ * both Config Apis and ACL Apis. So, we need this function to map the
+ * resource type internally to `rd_kafka_ConfigResourceType_t`. Like the
+ * enum value for `GROUP` is 32 in Config Apis, but it is 3 for ACL Apis.
+ */
+rd_kafka_ConfigResourceType_t
+rd_kafka_ResourceType_to_ConfigResourceType(rd_kafka_ResourceType_t restype);
+
+/**
+ * @brief Maps `rd_kafka_ConfigResourceType_t` to `rd_kafka_ResourceType_t`
+ * for Config Apis. We are incorrectly using `rd_kafka_ResourceType_t` in
+ * both Config Apis and ACL Apis. So, we need this function to map the
+ * `rd_kafka_ConfigResourceType_t` internally to
+ * `rd_kafka_ResourceType_t`. Like the enum value for `GROUP` is 32 in
+ * Config Apis, but it is 3 for ACL Apis.
+ */
+rd_kafka_ResourceType_t rd_kafka_ConfigResourceType_to_ResourceType(
+ rd_kafka_ConfigResourceType_t config_resource_type);
+
+
/**@}*/
@@ -445,6 +488,7 @@ struct rd_kafka_ConsumerGroupListing_s {
/** Is it a simple consumer group? That means empty protocol_type. */
rd_bool_t is_simple_consumer_group;
rd_kafka_consumer_group_state_t state; /**< Consumer group state. */
+ rd_kafka_consumer_group_type_t type; /**< Consumer group type. */
};
@@ -483,6 +527,9 @@ struct rd_kafka_MemberDescription_s {
char *group_instance_id; /**< Group instance id */
char *host; /**< Group member host */
rd_kafka_MemberAssignment_t assignment; /**< Member assignment */
+ rd_kafka_MemberAssignment_t
+ *target_assignment; /**< Target assignment. `NULL` for `classic`
+ protocol */
};
/**
@@ -502,6 +549,8 @@ struct rd_kafka_ConsumerGroupDescription_s {
char *partition_assignor;
/** Consumer group state. */
rd_kafka_consumer_group_state_t state;
+ /** Consumer group type. */
+ rd_kafka_consumer_group_type_t type;
/** Consumer group coordinator. */
rd_kafka_Node_t *coordinator;
/** Count of operations allowed for topic. -1 indicates operations not
@@ -586,4 +635,27 @@ typedef struct rd_kafka_ClusterDescription_s {
/**@}*/
+/**
+ * @name ElectLeaders
+ * @{
+ */
+
+/**
+ * @struct ElectLeaders request object
+ */
+struct rd_kafka_ElectLeaders_s {
+ rd_kafka_ElectionType_t election_type; /*Election Type*/
+ rd_kafka_topic_partition_list_t
+ *partitions; /*TopicPartitions for election*/
+};
+
+/**
+ * @struct ElectLeaders result object
+ */
+typedef struct rd_kafka_ElectLeadersResult_s {
+ rd_list_t partitions; /**< Type (rd_kafka_topic_partition_result_t *) */
+} rd_kafka_ElectLeadersResult_t;
+
+/**@}*/
+
#endif /* _RDKAFKA_ADMIN_H_ */
diff --git a/src/rdkafka_aux.c b/src/rdkafka_aux.c
index d327b6c8b0..7d5ccb5b2f 100644
--- a/src/rdkafka_aux.c
+++ b/src/rdkafka_aux.c
@@ -332,3 +332,78 @@ uint16_t rd_kafka_Node_port(const rd_kafka_Node_t *node) {
const char *rd_kafka_Node_rack(const rd_kafka_Node_t *node) {
return node->rack;
}
+
+/**
+ * @brief Creates a new rd_kafka_topic_partition_result_t object.
+ */
+
+rd_kafka_topic_partition_result_t *
+rd_kafka_topic_partition_result_new(const char *topic,
+ int32_t partition,
+ rd_kafka_resp_err_t err,
+ const char *errstr) {
+
+ rd_kafka_topic_partition_result_t *new_result;
+
+ new_result = rd_calloc(1, sizeof(*new_result));
+ new_result->topic_partition =
+ rd_kafka_topic_partition_new(topic, partition);
+ new_result->topic_partition->err = err;
+ new_result->error = rd_kafka_error_new(err, "%s", errstr);
+
+ return new_result;
+}
+
+const rd_kafka_topic_partition_t *rd_kafka_topic_partition_result_partition(
+ const rd_kafka_topic_partition_result_t *partition_result) {
+ return partition_result->topic_partition;
+}
+
+const rd_kafka_error_t *rd_kafka_topic_partition_result_error(
+ const rd_kafka_topic_partition_result_t *partition_result) {
+ return partition_result->error;
+}
+
+/**
+ * @brief Destroys the rd_kafka_topic_partition_result_t object.
+ */
+void rd_kafka_topic_partition_result_destroy(
+ rd_kafka_topic_partition_result_t *partition_result) {
+ rd_kafka_topic_partition_destroy(partition_result->topic_partition);
+ rd_kafka_error_destroy(partition_result->error);
+ rd_free(partition_result);
+}
+
+/**
+ * @brief Destroys the array of rd_kafka_topic_partition_result_t objects.
+ */
+void rd_kafka_topic_partition_result_destroy_array(
+ rd_kafka_topic_partition_result_t **partition_results,
+ int32_t partition_results_cnt) {
+ int32_t i;
+ for (i = 0; i < partition_results_cnt; i++) {
+ rd_kafka_topic_partition_result_destroy(partition_results[i]);
+ }
+}
+
+rd_kafka_topic_partition_result_t *rd_kafka_topic_partition_result_copy(
+ const rd_kafka_topic_partition_result_t *src) {
+ return rd_kafka_topic_partition_result_new(
+ src->topic_partition->topic, src->topic_partition->partition,
+ src->topic_partition->err, src->error->errstr);
+}
+
+void *rd_kafka_topic_partition_result_copy_opaque(const void *src,
+ void *opaque) {
+ return rd_kafka_topic_partition_result_copy(
+ (const rd_kafka_topic_partition_result_t *)src);
+}
+
+/**
+ * @brief Frees the memory allocated for a
+ * topic partition result object by calling
+ * its destroy function.
+ */
+void rd_kafka_topic_partition_result_free(void *ptr) {
+ rd_kafka_topic_partition_result_destroy(ptr);
+}
diff --git a/src/rdkafka_aux.h b/src/rdkafka_aux.h
index fec88cb2ad..340fcf708d 100644
--- a/src/rdkafka_aux.h
+++ b/src/rdkafka_aux.h
@@ -128,4 +128,47 @@ void rd_kafka_Node_destroy(rd_kafka_Node_t *node);
void rd_kafka_Node_free(void *node);
+/**
+ * @brief Represents a topic partition result.
+ *
+ * @remark Public Type
+ */
+struct rd_kafka_topic_partition_result_s {
+ rd_kafka_topic_partition_t *topic_partition;
+ rd_kafka_error_t *error;
+};
+
+/**
+ * @brief Create a new rd_kafka_topic_partition_result_t object.
+ *
+ * @param topic The topic name.
+ * @param partition The partition number.
+ * @param err The error code.
+ * @param errstr The error string.
+ *
+ * @returns a newly allocated rd_kafka_topic_partition_result_t object.
+ * Use rd_kafka_topic_partition_result_destroy() to free object when
+ * done.
+ */
+rd_kafka_topic_partition_result_t *
+rd_kafka_topic_partition_result_new(const char *topic,
+ int32_t partition,
+ rd_kafka_resp_err_t err,
+ const char *errstr);
+
+rd_kafka_topic_partition_result_t *rd_kafka_topic_partition_result_copy(
+ const rd_kafka_topic_partition_result_t *src);
+
+void *rd_kafka_topic_partition_result_copy_opaque(const void *src,
+ void *opaque);
+
+void rd_kafka_topic_partition_result_destroy(
+ rd_kafka_topic_partition_result_t *partition_result);
+
+void rd_kafka_topic_partition_result_destroy_array(
+ rd_kafka_topic_partition_result_t **partition_results,
+ int32_t partition_results_cnt);
+
+void rd_kafka_topic_partition_result_free(void *ptr);
+
#endif /* _RDKAFKA_AUX_H_ */
diff --git a/src/rdkafka_broker.c b/src/rdkafka_broker.c
index 1beeece2e8..eb8e849240 100644
--- a/src/rdkafka_broker.c
+++ b/src/rdkafka_broker.c
@@ -82,9 +82,9 @@
static const int rd_kafka_max_block_ms = 1000;
const char *rd_kafka_broker_state_names[] = {
- "INIT", "DOWN", "TRY_CONNECT", "CONNECT", "SSL_HANDSHAKE",
- "AUTH_LEGACY", "UP", "UPDATE", "APIVERSION_QUERY", "AUTH_HANDSHAKE",
- "AUTH_REQ", "REAUTH"};
+ "INIT", "DOWN", "TRY_CONNECT", "CONNECT",
+ "SSL_HANDSHAKE", "AUTH_LEGACY", "UP", "APIVERSION_QUERY",
+ "AUTH_HANDSHAKE", "AUTH_REQ", "REAUTH"};
const char *rd_kafka_secproto_names[] = {
[RD_KAFKA_PROTO_PLAINTEXT] = "plaintext",
@@ -111,7 +111,6 @@ rd_kafka_broker_needs_persistent_connection(rd_kafka_broker_t *rkb) {
rd_atomic32_get(&rkb->rkb_persistconn.coord);
}
-
/**
* @returns > 0 if a connection to this broker is needed, else 0.
* @locality broker thread
@@ -119,7 +118,7 @@ rd_kafka_broker_needs_persistent_connection(rd_kafka_broker_t *rkb) {
*/
static RD_INLINE int rd_kafka_broker_needs_connection(rd_kafka_broker_t *rkb) {
return rkb->rkb_state == RD_KAFKA_BROKER_STATE_INIT &&
- !rd_kafka_terminating(rkb->rkb_rk) &&
+ !rd_kafka_broker_or_instance_terminating(rkb) &&
!rd_kafka_fatal_error_code(rkb->rkb_rk) &&
(!rkb->rkb_rk->rk_conf.sparse_connections ||
rd_kafka_broker_needs_persistent_connection(rkb));
@@ -311,6 +310,32 @@ int16_t rd_kafka_broker_ApiVersion_supported(rd_kafka_broker_t *rkb,
rkb, ApiKey, minver, maxver, featuresp, rd_true /* do_lock */);
}
+/**
+ * @brief Check that at least a ApiVersion greater or equal to
+ * \p minver exists for \p ApiKey.
+ *
+ * @returns `rd_true` if the broker supports \p ApiKey with
+ * a version greater than or equal to \p minver, else `rd_false`.
+ * @locks none
+ * @locks_acquired rd_kafka_broker_lock()
+ * @locality any
+ */
+rd_bool_t rd_kafka_broker_ApiVersion_at_least(rd_kafka_broker_t *rkb,
+ int16_t ApiKey,
+ int16_t minver) {
+ return rd_kafka_broker_ApiVersion_supported0(
+ rkb, ApiKey, minver, RD_KAFKAP_RPC_VERSION_MAX, NULL,
+ rd_true /* do_lock */) != -1;
+}
+
+rd_bool_t rd_kafka_broker_ApiVersion_at_least_no_lock(rd_kafka_broker_t *rkb,
+ int16_t ApiKey,
+ int16_t minver) {
+ return rd_kafka_broker_ApiVersion_supported0(
+ rkb, ApiKey, minver, RD_KAFKAP_RPC_VERSION_MAX, NULL,
+ rd_false /* don't lock rkb */) != -1;
+}
+
/**
* @brief Set broker state.
*
@@ -321,7 +346,9 @@ int16_t rd_kafka_broker_ApiVersion_supported(rd_kafka_broker_t *rkb,
* @locality broker thread
*/
void rd_kafka_broker_set_state(rd_kafka_broker_t *rkb, int state) {
- rd_bool_t trigger_monitors = rd_false;
+ rd_bool_t trigger_monitors = rd_false,
+ reset_any_broker_down_reported = rd_false,
+ skip_broker_down = rkb->rkb_c.skip_broker_down;
if ((int)rkb->rkb_state == state)
return;
@@ -331,57 +358,70 @@ void rd_kafka_broker_set_state(rd_kafka_broker_t *rkb, int state) {
rd_kafka_broker_state_names[rkb->rkb_state],
rd_kafka_broker_state_names[state]);
- if (rkb->rkb_source == RD_KAFKA_INTERNAL) {
+ if (rd_kafka_broker_state_is_down(state) && skip_broker_down)
+ /* Reset the flag so if it disconnects again it won't be
+ * treated as a planned disconnection. */
+ rkb->rkb_c.skip_broker_down = rd_false;
+
+ if (rkb->rkb_source == RD_KAFKA_INTERNAL ||
+ RD_KAFKA_BROKER_IS_LOGICAL(rkb)) {
/* no-op */
- } else if (state == RD_KAFKA_BROKER_STATE_DOWN &&
- !rkb->rkb_down_reported) {
+ } else if (rd_kafka_broker_state_is_down(state) && !skip_broker_down &&
+ /* Only 0 -> 1 */
+ rd_atomic32_set(&rkb->rkb_down_reported, 1) == 0) {
+
/* Propagate ALL_BROKERS_DOWN event if all brokers are
- * now down, unless we're terminating.
- * Only trigger for brokers that has an address set,
- * e.g., not logical brokers that lost their address. */
+ * now down, unless we're terminating. */
if (rd_atomic32_add(&rkb->rkb_rk->rk_broker_down_cnt, 1) ==
rd_atomic32_get(&rkb->rkb_rk->rk_broker_cnt) -
rd_atomic32_get(
- &rkb->rkb_rk->rk_broker_addrless_cnt) &&
- !rd_kafka_broker_is_addrless(rkb) &&
- !rd_kafka_terminating(rkb->rkb_rk))
+ &rkb->rkb_rk->rk_logical_broker_cnt) &&
+ !rd_kafka_terminating(rkb->rkb_rk)) {
+ rd_kafka_rebootstrap(rkb->rkb_rk);
rd_kafka_op_err(
rkb->rkb_rk, RD_KAFKA_RESP_ERR__ALL_BROKERS_DOWN,
"%i/%i brokers are down",
rd_atomic32_get(&rkb->rkb_rk->rk_broker_down_cnt),
rd_atomic32_get(&rkb->rkb_rk->rk_broker_cnt) -
rd_atomic32_get(
- &rkb->rkb_rk->rk_broker_addrless_cnt));
- rkb->rkb_down_reported = 1;
+ &rkb->rkb_rk->rk_logical_broker_cnt));
+ }
} else if (rd_kafka_broker_state_is_up(state) &&
- rkb->rkb_down_reported) {
+ /* Only 1 -> 0 */
+ rd_atomic32_set(&rkb->rkb_down_reported, 0) == 1) {
rd_atomic32_sub(&rkb->rkb_rk->rk_broker_down_cnt, 1);
- rkb->rkb_down_reported = 0;
}
if (rkb->rkb_source != RD_KAFKA_INTERNAL) {
if (rd_kafka_broker_state_is_up(state) &&
!rd_kafka_broker_state_is_up(rkb->rkb_state)) {
- /* Up -> Down */
- rd_atomic32_add(&rkb->rkb_rk->rk_broker_up_cnt, 1);
+ /* ~Up -> Up */
+ if (!RD_KAFKA_BROKER_IS_LOGICAL(rkb)) {
+ rd_atomic32_add(&rkb->rkb_rk->rk_broker_up_cnt,
+ 1);
+
+ /* If at least one broker connects we reset
+ * the down counter to try again with rest of
+ * brokers. Otherwise, a single broker
+ * connection can cause `ALL_BROKERS_DOWN` to
+ * get triggered even if the connection to rest
+ * of brokers was restored but they didn't
+ * attempt a re-connection because of sparse
+ * broker connections. */
+ reset_any_broker_down_reported = rd_true;
+ }
trigger_monitors = rd_true;
- if (RD_KAFKA_BROKER_IS_LOGICAL(rkb))
- rd_atomic32_add(
- &rkb->rkb_rk->rk_logical_broker_up_cnt, 1);
-
} else if (rd_kafka_broker_state_is_up(rkb->rkb_state) &&
!rd_kafka_broker_state_is_up(state)) {
- /* ~Down(!Up) -> Up */
- rd_atomic32_sub(&rkb->rkb_rk->rk_broker_up_cnt, 1);
+ /* Up -> Down */
+ if (!RD_KAFKA_BROKER_IS_LOGICAL(rkb))
+ rd_atomic32_sub(&rkb->rkb_rk->rk_broker_up_cnt,
+ 1);
trigger_monitors = rd_true;
-
- if (RD_KAFKA_BROKER_IS_LOGICAL(rkb))
- rd_atomic32_sub(
- &rkb->rkb_rk->rk_logical_broker_up_cnt, 1);
}
/* If the connection or connection attempt failed and there
@@ -397,6 +437,14 @@ void rd_kafka_broker_set_state(rd_kafka_broker_t *rkb, int state) {
rkb->rkb_state = state;
rkb->rkb_ts_state = rd_clock();
+ if (reset_any_broker_down_reported) {
+ rd_kafka_broker_unlock(rkb);
+ /* Releases rkb->rkb_lock to respect
+ * lock ordering and avoid deadlocks */
+ rd_kafka_reset_any_broker_down_reported(rkb->rkb_rk);
+ rd_kafka_broker_lock(rkb);
+ }
+
if (trigger_monitors)
rd_kafka_broker_trigger_monitors(rkb);
@@ -456,34 +504,31 @@ static void rd_kafka_broker_set_error(rd_kafka_broker_t *rkb,
/* Provide more meaningful error messages in certain cases */
if (err == RD_KAFKA_RESP_ERR__TRANSPORT &&
- !strcmp(errstr, "Disconnected")) {
+ rd_kafka_transport_error_disconnected(fmt)) {
if (rkb->rkb_state == RD_KAFKA_BROKER_STATE_APIVERSION_QUERY) {
/* A disconnect while requesting ApiVersion typically
* means we're connecting to a SSL-listener as
* PLAINTEXT, but may also be caused by connecting to
* a broker that does not support ApiVersion (<0.10). */
-
if (rkb->rkb_proto != RD_KAFKA_PROTO_SSL &&
- rkb->rkb_proto != RD_KAFKA_PROTO_SASL_SSL)
- rd_kafka_broker_set_error(
- rkb, level, err,
- "Disconnected while requesting "
+ rkb->rkb_proto != RD_KAFKA_PROTO_SASL_SSL) {
+ ofe = rd_snprintf(
+ errstr + of, sizeof(errstr) - of, "%s",
+ ": requesting "
"ApiVersion: "
"might be caused by incorrect "
"security.protocol configuration "
"(connecting to a SSL listener?) or "
"broker version is < 0.10 "
- "(see api.version.request)",
- ap /*ignored*/);
- else
- rd_kafka_broker_set_error(
- rkb, level, err,
- "Disconnected while requesting "
+ "(see api.version.request)");
+ } else {
+ ofe = rd_snprintf(
+ errstr + of, sizeof(errstr) - of, "%s",
+ ": requesting "
"ApiVersion: "
"might be caused by broker version "
- "< 0.10 (see api.version.request)",
- ap /*ignored*/);
- return;
+ "< 0.10 (see api.version.request)");
+ }
} else if (rkb->rkb_state == RD_KAFKA_BROKER_STATE_UP &&
state_duration_ms < 2000 /*2s*/ &&
@@ -491,18 +536,20 @@ static void rd_kafka_broker_set_error(rd_kafka_broker_t *rkb,
RD_KAFKA_PROTO_SASL_SSL &&
rkb->rkb_rk->rk_conf.security_protocol !=
RD_KAFKA_PROTO_SASL_PLAINTEXT) {
+
/* If disconnected shortly after transitioning to UP
* state it typically means the broker listener is
* configured for SASL authentication but the client
* is not. */
- rd_kafka_broker_set_error(
- rkb, level, err,
- "Disconnected: verify that security.protocol "
- "is correctly configured, broker might "
- "require SASL authentication",
- ap /*ignored*/);
- return;
+ ofe =
+ rd_snprintf(errstr + of, sizeof(errstr) - of, "%s",
+ ": verify that security.protocol "
+ "is correctly configured, broker might "
+ "require SASL authentication");
}
+ if (ofe > sizeof(errstr) - of)
+ ofe = sizeof(errstr) - of;
+ of += ofe;
}
/* Check if error is identical to last error (prior to appending
@@ -554,13 +601,14 @@ static void rd_kafka_broker_set_error(rd_kafka_broker_t *rkb,
identical ? ": identical to last error" : "",
suppress ? ": error log suppressed" : "");
- if (level != LOG_DEBUG && (level <= LOG_CRIT || !suppress)) {
+ if (level <= LOG_CRIT || (level <= LOG_INFO && !suppress)) {
rd_kafka_log(rkb->rkb_rk, level, "FAIL", "%s: %s",
rkb->rkb_name, errstr);
- /* Send ERR op to application for processing. */
- rd_kafka_q_op_err(rkb->rkb_rk->rk_rep, err, "%s: %s",
- rkb->rkb_name, errstr);
+ if (level <= LOG_ERR)
+ /* Send ERR op to application for processing. */
+ rd_kafka_q_op_err(rkb->rkb_rk->rk_rep, err, "%s: %s",
+ rkb->rkb_name, errstr);
}
}
@@ -692,6 +740,13 @@ void rd_kafka_broker_fail(rd_kafka_broker_t *rkb,
if (rktp->rktp_leader_id != rktp->rktp_broker_id) {
rd_kafka_toppar_delegate_to_leader(rktp);
+ } else if (rd_kafka_broker_termination_in_progress(rkb)) {
+ /* Remove `rktp_broker` and `rktp_leader`
+ * references in `rktp`, even if this broker
+ * is still the leader, to allow to
+ * decommission it. */
+ rd_kafka_toppar_undelegate(rktp);
+ rd_kafka_toppar_forget_leader(rktp);
}
}
@@ -710,12 +765,18 @@ void rd_kafka_broker_fail(rd_kafka_broker_t *rkb,
/* Query for topic leaders to quickly pick up on failover. */
if (err != RD_KAFKA_RESP_ERR__DESTROY &&
+ err != RD_KAFKA_RESP_ERR__DESTROY_BROKER &&
old_state >= RD_KAFKA_BROKER_STATE_UP)
rd_kafka_metadata_refresh_known_topics(
rkb->rkb_rk, NULL, rd_true /*force*/, "broker down");
}
+#define rd_kafka_broker_planned_fail(RKB, ERR, FMT, ...) \
+ do { \
+ (RKB)->rkb_c.skip_broker_down = rd_true; \
+ rd_kafka_broker_fail(RKB, LOG_DEBUG, ERR, FMT, __VA_ARGS__); \
+ } while (0)
/**
* @brief Handle broker connection close.
@@ -999,11 +1060,11 @@ static void rd_kafka_broker_timeout_scan(rd_kafka_broker_t *rkb, rd_ts_t now) {
1000.0f));
else
rttinfo[0] = 0;
- rd_kafka_broker_fail(rkb, LOG_ERR,
- RD_KAFKA_RESP_ERR__TIMED_OUT,
- "%i request(s) timed out: "
- "disconnect%s",
- rkb->rkb_req_timeouts, rttinfo);
+ rd_kafka_broker_planned_fail(
+ rkb, RD_KAFKA_RESP_ERR__TIMED_OUT,
+ "%i request(s) timed out: "
+ "disconnect%s",
+ rkb->rkb_req_timeouts, rttinfo);
}
}
}
@@ -1042,8 +1103,9 @@ static int rd_kafka_broker_resolve(rd_kafka_broker_t *rkb,
int save_idx = 0;
if (!*nodename && rkb->rkb_source == RD_KAFKA_LOGICAL) {
- rd_kafka_broker_fail(rkb, LOG_DEBUG, RD_KAFKA_RESP_ERR__RESOLVE,
- "Logical broker has no address yet");
+ rd_kafka_broker_planned_fail(
+ rkb, RD_KAFKA_RESP_ERR__RESOLVE, "%s",
+ "Logical broker has no address yet");
return -1;
}
@@ -1403,12 +1465,13 @@ rd_kafka_broker_t *rd_kafka_broker_random0(const char *func,
int fcnt = 0;
TAILQ_FOREACH(rkb, &rk->rk_brokers, rkb_link) {
- if (RD_KAFKA_BROKER_IS_LOGICAL(rkb))
+ if (rd_kafka_broker_or_instance_terminating(rkb) ||
+ RD_KAFKA_BROKER_IS_LOGICAL(rkb))
continue;
rd_kafka_broker_lock(rkb);
if ((is_up && rd_kafka_broker_state_is_up(rkb->rkb_state)) ||
- (!is_up && (int)rkb->rkb_state == state)) {
+ (!is_up && (state == -1 || (int)rkb->rkb_state == state))) {
if (filter && filter(rkb, opaque)) {
/* Filtered out */
fcnt++;
@@ -1459,6 +1522,8 @@ rd_kafka_broker_weighted(rd_kafka_t *rk,
TAILQ_FOREACH(rkb, &rk->rk_brokers, rkb_link) {
int weight;
+ if (rd_kafka_broker_or_instance_terminating(rkb))
+ continue;
rd_kafka_broker_lock(rkb);
if (features && (rkb->rkb_features & features) != features)
@@ -1512,12 +1577,11 @@ rd_kafka_broker_weighted(rd_kafka_t *rk,
static int rd_kafka_broker_weight_usable(rd_kafka_broker_t *rkb) {
int weight = 0;
- if (!rd_kafka_broker_state_is_up(rkb->rkb_state))
+ if (!rd_kafka_broker_state_is_up(rkb->rkb_state) ||
+ RD_KAFKA_BROKER_IS_LOGICAL(rkb))
return 0;
- weight +=
- 2000 * (rkb->rkb_nodeid != -1 && !RD_KAFKA_BROKER_IS_LOGICAL(rkb));
- weight += 10 * !RD_KAFKA_BROKER_IS_LOGICAL(rkb);
+ weight += 2000;
if (likely(!rd_atomic32_get(&rkb->rkb_blocking_request_cnt))) {
rd_ts_t tx_last = rd_atomic64_get(&rkb->rkb_c.ts_send);
@@ -1526,6 +1590,9 @@ static int rd_kafka_broker_weight_usable(rd_kafka_broker_t *rkb) {
1000000);
weight += 1; /* is not blocking */
+ if (rkb->rkb_source == RD_KAFKA_LEARNED)
+ /* Prefer learned brokers */
+ weight += 1000;
/* Prefer least idle broker (based on last 10 minutes use) */
if (idle < 0)
@@ -1740,6 +1807,9 @@ rd_list_t *rd_kafka_brokers_get_nodeids_async(rd_kafka_t *rk,
}
i = 0;
TAILQ_FOREACH(rkb, &rk->rk_brokers, rkb_link) {
+ if (rd_kafka_broker_or_instance_terminating(rkb))
+ continue;
+
rd_kafka_broker_lock(rkb);
if (rkb->rkb_nodeid != -1 &&
!RD_KAFKA_BROKER_IS_LOGICAL(rkb)) {
@@ -1874,6 +1944,30 @@ static rd_kafka_buf_t *rd_kafka_waitresp_find(rd_kafka_broker_t *rkb,
rd_avg_add(&rkb->rkb_telemetry.rd_avg_current.rkb_avg_rtt,
rkbuf->rkbuf_ts_sent);
+ switch (rkbuf->rkbuf_reqhdr.ApiKey) {
+ case RD_KAFKAP_Fetch:
+ if (rkb->rkb_rk->rk_type == RD_KAFKA_CONSUMER)
+ rd_avg_add(&rkb->rkb_telemetry.rd_avg_current
+ .rkb_avg_fetch_latency,
+ rkbuf->rkbuf_ts_sent);
+ break;
+ case RD_KAFKAP_OffsetCommit:
+ if (rkb->rkb_rk->rk_type == RD_KAFKA_CONSUMER)
+ rd_avg_add(
+ &rkb->rkb_rk->rk_telemetry.rd_avg_current
+ .rk_avg_commit_latency,
+ rkbuf->rkbuf_ts_sent);
+ break;
+ case RD_KAFKAP_Produce:
+ if (rkb->rkb_rk->rk_type == RD_KAFKA_PRODUCER)
+ rd_avg_add(&rkb->rkb_telemetry.rd_avg_current
+ .rkb_avg_produce_latency,
+ rkbuf->rkbuf_ts_sent);
+ break;
+ default:
+ break;
+ }
+
if (rkbuf->rkbuf_flags & RD_KAFKA_OP_F_BLOCKING &&
rd_atomic32_sub(&rkb->rkb_blocking_request_cnt, 1) == 1)
rd_kafka_brokers_broadcast_state_change(rkb->rkb_rk);
@@ -2074,7 +2168,7 @@ int rd_kafka_recv(rd_kafka_broker_t *rkb) {
err_parse:
err = rkbuf->rkbuf_err;
err:
- if (!strcmp(errstr, "Disconnected"))
+ if (rd_kafka_transport_error_disconnected(errstr))
rd_kafka_broker_conn_closed(rkb, err, errstr);
else
rd_kafka_broker_fail(rkb, LOG_ERR, err, "Receive failed: %s",
@@ -2177,53 +2271,6 @@ rd_kafka_broker_reconnect_backoff(const rd_kafka_broker_t *rkb, rd_ts_t now) {
return (int)(remains / 1000);
}
-
-/**
- * @brief Unittest for reconnect.backoff.ms
- */
-static int rd_ut_reconnect_backoff(void) {
- rd_kafka_broker_t rkb = RD_ZERO_INIT;
- rd_kafka_conf_t conf = {.reconnect_backoff_ms = 10,
- .reconnect_backoff_max_ms = 90};
- rd_ts_t now = 1000000;
- int backoff;
-
- rkb.rkb_reconnect_backoff_ms = conf.reconnect_backoff_ms;
-
- /* broker's backoff is the initial reconnect.backoff.ms=10 */
- rd_kafka_broker_update_reconnect_backoff(&rkb, &conf, now);
- backoff = rd_kafka_broker_reconnect_backoff(&rkb, now);
- RD_UT_ASSERT_RANGE(backoff, 7, 15, "%d");
-
- /* .. 20 */
- rd_kafka_broker_update_reconnect_backoff(&rkb, &conf, now);
- backoff = rd_kafka_broker_reconnect_backoff(&rkb, now);
- RD_UT_ASSERT_RANGE(backoff, 15, 30, "%d");
-
- /* .. 40 */
- rd_kafka_broker_update_reconnect_backoff(&rkb, &conf, now);
- backoff = rd_kafka_broker_reconnect_backoff(&rkb, now);
- RD_UT_ASSERT_RANGE(backoff, 30, 60, "%d");
-
- /* .. 80, the jitter is capped at reconnect.backoff.max.ms=90 */
- rd_kafka_broker_update_reconnect_backoff(&rkb, &conf, now);
- backoff = rd_kafka_broker_reconnect_backoff(&rkb, now);
- RD_UT_ASSERT_RANGE(backoff, 60, conf.reconnect_backoff_max_ms, "%d");
-
- /* .. 90, capped by reconnect.backoff.max.ms */
- rd_kafka_broker_update_reconnect_backoff(&rkb, &conf, now);
- backoff = rd_kafka_broker_reconnect_backoff(&rkb, now);
- RD_UT_ASSERT_RANGE(backoff, 67, conf.reconnect_backoff_max_ms, "%d");
-
- /* .. 90, should remain at capped value. */
- rd_kafka_broker_update_reconnect_backoff(&rkb, &conf, now);
- backoff = rd_kafka_broker_reconnect_backoff(&rkb, now);
- RD_UT_ASSERT_RANGE(backoff, 67, conf.reconnect_backoff_max_ms, "%d");
-
- RD_UT_PASS();
-}
-
-
/**
* @brief Initiate asynchronous connection attempt to the next address
* in the broker's address list.
@@ -2313,6 +2360,7 @@ void rd_kafka_broker_connect_up(rd_kafka_broker_t *rkb) {
if (rd_kafka_broker_ApiVersion_supported(
rkb, RD_KAFKAP_GetTelemetrySubscriptions, 0, 0, &features) !=
-1 &&
+ rkb->rkb_source == RD_KAFKA_LEARNED &&
rkb->rkb_rk->rk_conf.enable_metrics_push) {
rd_kafka_t *rk = rkb->rkb_rk;
rd_kafka_op_t *rko =
@@ -2346,7 +2394,7 @@ static void rd_kafka_broker_handle_SaslHandshake(rd_kafka_t *rk,
char *mechs = "(n/a)";
size_t msz, mof = 0;
- if (err == RD_KAFKA_RESP_ERR__DESTROY)
+ if (rd_kafka_broker_is_any_err_destroy(err))
return;
if (err)
@@ -2525,7 +2573,7 @@ static void rd_kafka_broker_handle_ApiVersion(rd_kafka_t *rk,
size_t api_cnt = 0;
int16_t retry_ApiVersion = -1;
- if (err == RD_KAFKA_RESP_ERR__DESTROY)
+ if (rd_kafka_broker_is_any_err_destroy(err))
return;
err = rd_kafka_handle_ApiVersion(rk, rkb, err, rkbuf, request, &apis,
@@ -2918,10 +2966,18 @@ void rd_kafka_broker_buf_retry(rd_kafka_broker_t *rkb, rd_kafka_buf_t *rkbuf) {
rd_atomic64_add(&rkb->rkb_c.tx_retries, 1);
/* In some cases, failed Produce requests do not increment the retry
* count, see rd_kafka_handle_Produce_error. */
- if (rkbuf->rkbuf_retries > 0)
- backoff = (1 << (rkbuf->rkbuf_retries - 1)) *
+ if (rkbuf->rkbuf_retries > 0) {
+ int shift = rkbuf->rkbuf_retries - 1;
+
+ /* Cap shift at 34 to prevent overflow in final calculation.
+ * Accounts for multiplication by retry_backoff_ms (max 300000)
+ * and jitter (max 1200). */
+ if (shift > 34)
+ shift = 34;
+
+ backoff = ((int64_t)1 << shift) *
(rkb->rkb_rk->rk_conf.retry_backoff_ms);
- else
+ } else
backoff = rkb->rkb_rk->rk_conf.retry_backoff_ms;
/* We are multiplying by 10 as (backoff_ms * percent * 1000)/100 ->
@@ -3077,53 +3133,6 @@ void rd_kafka_dr_implicit_ack(rd_kafka_broker_t *rkb,
rd_kafka_dr_msgq(rktp->rktp_rkt, &acked, RD_KAFKA_RESP_ERR_NO_ERROR);
}
-
-
-/**
- * @brief Map existing partitions to this broker using the
- * toppar's leader_id. Only undelegated partitions
- * matching this broker are mapped.
- *
- * @locks none
- * @locality any
- */
-static void rd_kafka_broker_map_partitions(rd_kafka_broker_t *rkb) {
- rd_kafka_t *rk = rkb->rkb_rk;
- rd_kafka_topic_t *rkt;
- int cnt = 0;
-
- if (rkb->rkb_nodeid == -1 || RD_KAFKA_BROKER_IS_LOGICAL(rkb))
- return;
-
- rd_kafka_rdlock(rk);
- TAILQ_FOREACH(rkt, &rk->rk_topics, rkt_link) {
- int i;
-
- rd_kafka_topic_wrlock(rkt);
- for (i = 0; i < rkt->rkt_partition_cnt; i++) {
- rd_kafka_toppar_t *rktp = rkt->rkt_p[i];
-
- /* Only map undelegated partitions matching this
- * broker*/
- rd_kafka_toppar_lock(rktp);
- if (rktp->rktp_leader_id == rkb->rkb_nodeid &&
- !(rktp->rktp_broker && rktp->rktp_next_broker)) {
- rd_kafka_toppar_broker_update(
- rktp, rktp->rktp_leader_id, rkb,
- "broker node information updated");
- cnt++;
- }
- rd_kafka_toppar_unlock(rktp);
- }
- rd_kafka_topic_wrunlock(rkt);
- }
- rd_kafka_rdunlock(rk);
-
- rd_rkb_dbg(rkb, TOPIC | RD_KAFKA_DBG_BROKER, "LEADER",
- "Mapped %d partition(s) to broker", cnt);
-}
-
-
/**
* @brief Broker id comparator
*/
@@ -3164,6 +3173,10 @@ static void rd_kafka_broker_prepare_destroy(rd_kafka_broker_t *rkb) {
rd_kafka_broker_monitor_del(&rkb->rkb_coord_monitor);
}
+static rd_kafka_resp_err_t rd_kafka_broker_destroy_error(rd_kafka_t *rk) {
+ return rd_kafka_terminating(rk) ? RD_KAFKA_RESP_ERR__DESTROY
+ : RD_KAFKA_RESP_ERR__DESTROY_BROKER;
+}
/**
* @brief Serve a broker op (an op posted by another thread to be handled by
@@ -3183,7 +3196,7 @@ rd_kafka_broker_op_serve(rd_kafka_broker_t *rkb, rd_kafka_op_t *rko) {
switch (rko->rko_type) {
case RD_KAFKA_OP_NODE_UPDATE: {
- enum { _UPD_NAME = 0x1, _UPD_ID = 0x2 } updated = 0;
+ rd_bool_t updated = rd_false;
char brokername[RD_KAFKA_NODENAME_SIZE];
/* Need kafka_wrlock for updating rk_broker_by_id */
@@ -3197,31 +3210,7 @@ rd_kafka_broker_op_serve(rd_kafka_broker_t *rkb, rd_kafka_op_t *rko) {
rd_strlcpy(rkb->rkb_nodename, rko->rko_u.node.nodename,
sizeof(rkb->rkb_nodename));
rkb->rkb_nodename_epoch++;
- updated |= _UPD_NAME;
- }
-
- if (rko->rko_u.node.nodeid != -1 &&
- !RD_KAFKA_BROKER_IS_LOGICAL(rkb) &&
- rko->rko_u.node.nodeid != rkb->rkb_nodeid) {
- int32_t old_nodeid = rkb->rkb_nodeid;
- rd_rkb_dbg(rkb, BROKER, "UPDATE",
- "NodeId changed from %" PRId32
- " to %" PRId32,
- rkb->rkb_nodeid, rko->rko_u.node.nodeid);
-
- rkb->rkb_nodeid = rko->rko_u.node.nodeid;
-
- /* Update system thread name */
- rd_kafka_set_thread_sysname("rdk:broker%" PRId32,
- rkb->rkb_nodeid);
-
- /* Update broker_by_id sorted list */
- if (old_nodeid == -1)
- rd_list_add(&rkb->rkb_rk->rk_broker_by_id, rkb);
- rd_list_sort(&rkb->rkb_rk->rk_broker_by_id,
- rd_kafka_broker_cmp_by_id);
-
- updated |= _UPD_ID;
+ updated = rd_true;
}
rd_kafka_mk_brokername(brokername, sizeof(brokername),
@@ -3240,22 +3229,10 @@ rd_kafka_broker_op_serve(rd_kafka_broker_t *rkb, rd_kafka_op_t *rko) {
rd_kafka_broker_unlock(rkb);
rd_kafka_wrunlock(rkb->rkb_rk);
- if (updated & _UPD_NAME)
- rd_kafka_broker_fail(rkb, LOG_DEBUG,
- RD_KAFKA_RESP_ERR__TRANSPORT,
- "Broker hostname updated");
- else if (updated & _UPD_ID) {
- /* Map existing partitions to this broker. */
- rd_kafka_broker_map_partitions(rkb);
-
- /* If broker is currently in state up we need
- * to trigger a state change so it exits its
- * state&type based .._serve() loop. */
- rd_kafka_broker_lock(rkb);
- if (rkb->rkb_state == RD_KAFKA_BROKER_STATE_UP)
- rd_kafka_broker_set_state(
- rkb, RD_KAFKA_BROKER_STATE_UPDATE);
- rd_kafka_broker_unlock(rkb);
+ if (updated) {
+ rd_kafka_broker_planned_fail(
+ rkb, RD_KAFKA_RESP_ERR__TRANSPORT, "%s",
+ "Broker hostname updated");
}
rd_kafka_brokers_broadcast_state_change(rkb->rkb_rk);
@@ -3284,17 +3261,20 @@ rd_kafka_broker_op_serve(rd_kafka_broker_t *rkb, rd_kafka_op_t *rko) {
rd_kafka_toppar_lock(rktp);
/* Abort join if instance is terminating */
- if (rd_kafka_terminating(rkb->rkb_rk) ||
+ if (rd_kafka_broker_or_instance_terminating(rkb) ||
(rktp->rktp_flags & RD_KAFKA_TOPPAR_F_REMOVE)) {
- rd_rkb_dbg(rkb, BROKER | RD_KAFKA_DBG_TOPIC, "TOPBRK",
- "Topic %s [%" PRId32
- "]: not joining broker: "
- "%s",
- rktp->rktp_rkt->rkt_topic->str,
- rktp->rktp_partition,
- rd_kafka_terminating(rkb->rkb_rk)
- ? "instance is terminating"
- : "partition removed");
+ rd_rkb_dbg(
+ rkb, BROKER | RD_KAFKA_DBG_TOPIC, "TOPBRK",
+ "Topic %s [%" PRId32
+ "]: not joining broker: "
+ "%s",
+ rktp->rktp_rkt->rkt_topic->str,
+ rktp->rktp_partition,
+ rd_kafka_terminating(rkb->rkb_rk)
+ ? "instance is terminating"
+ : rd_kafka_broker_termination_in_progress(rkb)
+ ? "broker is terminating"
+ : "partition removed");
rd_kafka_broker_destroy(rktp->rktp_next_broker);
rktp->rktp_next_broker = NULL;
@@ -3362,6 +3342,8 @@ rd_kafka_broker_op_serve(rd_kafka_broker_t *rkb, rd_kafka_op_t *rko) {
"finish before producing to "
"new leader");
}
+ } else if (rkb->rkb_rk->rk_type == RD_KAFKA_CONSUMER) {
+ rktp->rktp_ts_fetch_backoff = 0;
}
rd_kafka_broker_destroy(rktp->rktp_next_broker);
@@ -3472,6 +3454,11 @@ rd_kafka_broker_op_serve(rd_kafka_broker_t *rkb, rd_kafka_op_t *rko) {
: (topic_err
? topic_err
: RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION));
+
+ if (rkb->rkb_rk->rk_type == RD_KAFKA_CONSUMER) {
+ rd_kafka_toppar_purge_internal_fetch_queue_maybe(
+ rktp);
+ }
}
rd_kafka_toppar_unlock(rktp);
@@ -3496,10 +3483,13 @@ rd_kafka_broker_op_serve(rd_kafka_broker_t *rkb, rd_kafka_op_t *rko) {
* and trigger a state change.
* This makes sure any eonce dependent on state changes
* are triggered. */
- rd_kafka_broker_fail(rkb, LOG_DEBUG, RD_KAFKA_RESP_ERR__DESTROY,
- "Client is terminating");
+ rd_kafka_broker_fail(rkb, LOG_DEBUG,
+ rd_kafka_broker_destroy_error(rkb->rkb_rk),
+ "Decommissioning this broker");
rd_kafka_broker_prepare_destroy(rkb);
+ /* Release main thread reference here */
+ rd_kafka_broker_destroy(rkb);
wakeup = rd_true;
break;
@@ -3537,9 +3527,8 @@ rd_kafka_broker_op_serve(rd_kafka_broker_t *rkb, rd_kafka_op_t *rko) {
rd_kafka_broker_unlock(rkb);
if (do_disconnect)
- rd_kafka_broker_fail(
- rkb, LOG_DEBUG,
- RD_KAFKA_RESP_ERR__TRANSPORT,
+ rd_kafka_broker_planned_fail(
+ rkb, RD_KAFKA_RESP_ERR__TRANSPORT, "%s",
"Closing connection due to "
"nodename change");
}
@@ -3617,7 +3606,7 @@ rd_kafka_broker_ops_io_serve(rd_kafka_broker_t *rkb, rd_ts_t abs_timeout) {
rd_ts_t now;
rd_bool_t wakeup;
- if (unlikely(rd_kafka_terminating(rkb->rkb_rk)))
+ if (unlikely(rd_kafka_broker_or_instance_terminating(rkb)))
abs_timeout = rd_clock() + 1000;
else if (unlikely(rd_kafka_broker_needs_connection(rkb)))
abs_timeout = RD_POLL_NOWAIT;
@@ -4366,6 +4355,8 @@ static RD_INLINE void rd_kafka_broker_idle_check(rd_kafka_broker_t *rkb) {
rd_ts_t ts_recv = rd_atomic64_get(&rkb->rkb_c.ts_recv);
rd_ts_t ts_last_activity = RD_MAX(ts_send, ts_recv);
int idle_ms;
+ const int max_jitter_ms = 2000;
+ rd_dassert(rkb->rkb_rk->rk_conf.connections_max_idle_ms > 0);
/* If nothing has been sent yet, use the connection time as
* last activity. */
@@ -4374,13 +4365,22 @@ static RD_INLINE void rd_kafka_broker_idle_check(rd_kafka_broker_t *rkb) {
idle_ms = (int)((rd_clock() - ts_last_activity) / 1000);
- if (likely(idle_ms < rkb->rkb_rk->rk_conf.connections_max_idle_ms))
+ if (unlikely(rkb->rkb_c.connections_max_idle_ms == -1)) {
+ /* Add a different jitter for each broker. */
+ rkb->rkb_c.connections_max_idle_ms =
+ rkb->rkb_rk->rk_conf.connections_max_idle_ms;
+ if (rkb->rkb_c.connections_max_idle_ms >= 2 * max_jitter_ms)
+ rkb->rkb_c.connections_max_idle_ms -=
+ rd_jitter(0, max_jitter_ms);
+ }
+
+ if (likely(idle_ms < rkb->rkb_c.connections_max_idle_ms))
return;
- rd_kafka_broker_fail(rkb, LOG_DEBUG, RD_KAFKA_RESP_ERR__TRANSPORT,
- "Connection max idle time exceeded "
- "(%dms since last activity)",
- idle_ms);
+ rd_kafka_broker_planned_fail(rkb, RD_KAFKA_RESP_ERR__TRANSPORT,
+ "Connection max idle time exceeded "
+ "(%dms since last activity)",
+ idle_ms);
}
@@ -4456,7 +4456,7 @@ static RD_INLINE void rd_kafka_broker_idle_check(rd_kafka_broker_t *rkb) {
static void rd_kafka_broker_serve(rd_kafka_broker_t *rkb, int timeout_ms) {
rd_ts_t abs_timeout;
- if (unlikely(rd_kafka_terminating(rkb->rkb_rk) ||
+ if (unlikely(rd_kafka_broker_or_instance_terminating(rkb) ||
timeout_ms == RD_POLL_NOWAIT))
timeout_ms = 1;
else if (timeout_ms == RD_POLL_INFINITE)
@@ -4508,6 +4508,7 @@ rd_kafka_broker_addresses_exhausted(const rd_kafka_broker_t *rkb) {
static int rd_kafka_broker_thread_main(void *arg) {
rd_kafka_broker_t *rkb = arg;
rd_kafka_t *rk = rkb->rkb_rk;
+ rd_kafka_op_t *terminate_op;
rd_kafka_set_thread_name("%s", rkb->rkb_name);
rd_kafka_set_thread_sysname("rdk:broker%" PRId32, rkb->rkb_nodeid);
@@ -4574,8 +4575,11 @@ static int rd_kafka_broker_thread_main(void *arg) {
break;
}
- if (unlikely(rd_kafka_terminating(rkb->rkb_rk)))
+ if (unlikely(
+ rd_kafka_broker_or_instance_terminating(rkb))) {
rd_kafka_broker_serve(rkb, 1000);
+ break;
+ }
if (!rd_kafka_sasl_ready(rkb->rkb_rk)) {
/* SASL provider not yet ready. */
@@ -4680,30 +4684,21 @@ static int rd_kafka_broker_thread_main(void *arg) {
rd_kafka_broker_connect_auth(rkb);
break;
- case RD_KAFKA_BROKER_STATE_UPDATE:
- /* FALLTHRU */
case RD_KAFKA_BROKER_STATE_UP:
rd_kafka_broker_serve(rkb, rd_kafka_max_block_ms);
-
- if (rkb->rkb_state == RD_KAFKA_BROKER_STATE_UPDATE) {
- rd_kafka_broker_lock(rkb);
- rd_kafka_broker_set_state(
- rkb, RD_KAFKA_BROKER_STATE_UP);
- rd_kafka_broker_unlock(rkb);
- }
break;
}
- if (rd_kafka_terminating(rkb->rkb_rk)) {
+ if (rd_kafka_broker_or_instance_terminating(rkb)) {
/* Handle is terminating: fail the send+retry queue
* to speed up termination, otherwise we'll
* need to wait for request timeouts. */
r = rd_kafka_broker_bufq_timeout_scan(
rkb, 0, &rkb->rkb_outbufs, NULL, -1,
- RD_KAFKA_RESP_ERR__DESTROY, 0, NULL, 0);
+ rd_kafka_broker_destroy_error(rk), 0, NULL, 0);
r += rd_kafka_broker_bufq_timeout_scan(
rkb, 0, &rkb->rkb_retrybufs, NULL, -1,
- RD_KAFKA_RESP_ERR__DESTROY, 0, NULL, 0);
+ rd_kafka_broker_destroy_error(rk), 0, NULL, 0);
rd_rkb_dbg(
rkb, BROKER, "TERMINATE",
"Handle is terminating in state %s: "
@@ -4720,28 +4715,46 @@ static int rd_kafka_broker_thread_main(void *arg) {
}
}
- if (rkb->rkb_source != RD_KAFKA_INTERNAL) {
- rd_kafka_wrlock(rkb->rkb_rk);
- TAILQ_REMOVE(&rkb->rkb_rk->rk_brokers, rkb, rkb_link);
- if (rkb->rkb_nodeid != -1 && !RD_KAFKA_BROKER_IS_LOGICAL(rkb))
- rd_list_remove(&rkb->rkb_rk->rk_broker_by_id, rkb);
- (void)rd_atomic32_sub(&rkb->rkb_rk->rk_broker_cnt, 1);
- rd_kafka_wrunlock(rkb->rkb_rk);
- }
-
- rd_kafka_broker_fail(rkb, LOG_DEBUG, RD_KAFKA_RESP_ERR__DESTROY,
- "Broker handle is terminating");
-
/* Disable and drain ops queue.
* Simply purging the ops queue risks leaving dangling references
* for ops such as PARTITION_JOIN/PARTITION_LEAVE where the broker
* reference is not maintained in the rko (but in rktp_next_leader).
- * #1596 */
+ * #1596.
+ * Do this before failing the broker to make sure no buffers
+ * are enqueued after that. */
rd_kafka_q_disable(rkb->rkb_ops);
while (rd_kafka_broker_ops_serve(rkb, RD_POLL_NOWAIT))
;
- rd_kafka_broker_destroy(rkb);
+ rd_kafka_broker_fail(rkb, LOG_DEBUG, rd_kafka_broker_destroy_error(rk),
+ "Broker handle is terminating");
+
+ rd_rkb_dbg(rkb, BROKER, "TERMINATE",
+ "Handle terminates in state %s: "
+ "%d refcnts (%p), %d toppar(s), "
+ "%d active toppar(s), "
+ "%d outbufs, %d waitresps, %d retrybufs",
+ rd_kafka_broker_state_names[rkb->rkb_state],
+ rd_refcnt_get(&rkb->rkb_refcnt), &rkb->rkb_refcnt,
+ rkb->rkb_toppar_cnt, rkb->rkb_active_toppar_cnt,
+ (int)rd_kafka_bufq_cnt(&rkb->rkb_outbufs),
+ (int)rd_kafka_bufq_cnt(&rkb->rkb_waitresps),
+ (int)rd_kafka_bufq_cnt(&rkb->rkb_retrybufs));
+
+ rd_dassert(rkb->rkb_state == RD_KAFKA_BROKER_STATE_DOWN);
+ if (rkb->rkb_source != RD_KAFKA_INTERNAL) {
+ rd_kafka_wrlock(rkb->rkb_rk);
+ TAILQ_REMOVE(&rkb->rkb_rk->rk_brokers, rkb, rkb_link);
+
+ if (RD_KAFKA_BROKER_IS_LOGICAL(rkb)) {
+ rd_atomic32_sub(&rkb->rkb_rk->rk_logical_broker_cnt, 1);
+ } else if (rd_atomic32_set(&rkb->rkb_down_reported, 0) == 1) {
+ /* Only 1 -> 0 */
+ rd_atomic32_sub(&rkb->rkb_rk->rk_broker_down_cnt, 1);
+ }
+ rd_atomic32_sub(&rkb->rkb_rk->rk_broker_cnt, 1);
+ rd_kafka_wrunlock(rkb->rkb_rk);
+ }
#if WITH_SSL
/* Remove OpenSSL per-thread error state to avoid memory leaks */
@@ -4757,6 +4770,15 @@ static int rd_kafka_broker_thread_main(void *arg) {
rd_atomic32_sub(&rd_kafka_thread_cnt_curr, 1);
+ terminate_op = rd_kafka_op_new(RD_KAFKA_OP_TERMINATE);
+ terminate_op->rko_u.terminated.rkb = rkb;
+ terminate_op->rko_u.terminated.cb =
+ rd_kafka_decommissioned_broker_thread_join;
+ rd_kafka_q_enq(rk->rk_ops, terminate_op);
+
+ /* Release broker thread reference here and call destroy final. */
+ rd_kafka_broker_destroy(rkb);
+
return 0;
}
@@ -4810,6 +4832,19 @@ void rd_kafka_broker_destroy_final(rd_kafka_broker_t *rkb) {
rd_avg_destroy(
&rkb->rkb_telemetry.rd_avg_current.rkb_avg_outbuf_latency);
+ if (rkb->rkb_rk->rk_type == RD_KAFKA_CONSUMER) {
+ rd_avg_destroy(
+ &rkb->rkb_telemetry.rd_avg_rollover.rkb_avg_fetch_latency);
+ rd_avg_destroy(
+ &rkb->rkb_telemetry.rd_avg_current.rkb_avg_fetch_latency);
+ } else if (rkb->rkb_rk->rk_type == RD_KAFKA_PRODUCER) {
+ rd_avg_destroy(
+ &rkb->rkb_telemetry.rd_avg_current.rkb_avg_produce_latency);
+ rd_avg_destroy(&rkb->rkb_telemetry.rd_avg_rollover
+ .rkb_avg_produce_latency);
+ }
+
+
mtx_lock(&rkb->rkb_logname_lock);
rd_free(rkb->rkb_logname);
rkb->rkb_logname = NULL;
@@ -4877,13 +4912,14 @@ rd_kafka_broker_t *rd_kafka_broker_add(rd_kafka_t *rk,
rd_snprintf(rkb->rkb_name, sizeof(rkb->rkb_name), "%s", name);
}
- rkb->rkb_source = source;
- rkb->rkb_rk = rk;
- rkb->rkb_ts_state = rd_clock();
- rkb->rkb_nodeid = nodeid;
- rkb->rkb_proto = proto;
- rkb->rkb_port = port;
- rkb->rkb_origname = rd_strdup(name);
+ rkb->rkb_source = source;
+ rkb->rkb_rk = rk;
+ rkb->rkb_ts_state = rd_clock();
+ rkb->rkb_nodeid = nodeid;
+ rkb->rkb_proto = proto;
+ rkb->rkb_port = port;
+ rkb->rkb_origname = rd_strdup(name);
+ rkb->rkb_c.connections_max_idle_ms = -1;
mtx_init(&rkb->rkb_lock, mtx_plain);
mtx_init(&rkb->rkb_logname_lock, mtx_plain);
@@ -4922,11 +4958,31 @@ rd_kafka_broker_t *rd_kafka_broker_add(rd_kafka_t *rk,
RD_AVG_GAUGE, 0, 100 * 1000, 2,
rk->rk_conf.enable_metrics_push);
+ if (rk->rk_type == RD_KAFKA_CONSUMER) {
+ rd_avg_init(
+ &rkb->rkb_telemetry.rd_avg_rollover.rkb_avg_fetch_latency,
+ RD_AVG_GAUGE, 0, 500 * 1000, 2,
+ rk->rk_conf.enable_metrics_push);
+ rd_avg_init(
+ &rkb->rkb_telemetry.rd_avg_current.rkb_avg_fetch_latency,
+ RD_AVG_GAUGE, 0, 500 * 1000, 2,
+ rk->rk_conf.enable_metrics_push);
+ } else if (rk->rk_type == RD_KAFKA_PRODUCER) {
+ rd_avg_init(
+ &rkb->rkb_telemetry.rd_avg_current.rkb_avg_produce_latency,
+ RD_AVG_GAUGE, 0, 500 * 1000, 2, rd_true);
+ rd_avg_init(
+ &rkb->rkb_telemetry.rd_avg_rollover.rkb_avg_produce_latency,
+ RD_AVG_GAUGE, 0, 500 * 1000, 2, rd_true);
+ }
+
rd_refcnt_init(&rkb->rkb_refcnt, 0);
rd_kafka_broker_keep(rkb); /* rk_broker's refcount */
rkb->rkb_reconnect_backoff_ms = rk->rk_conf.reconnect_backoff_ms;
rd_atomic32_init(&rkb->rkb_persistconn.coord, 0);
+ rd_atomic32_init(&rkb->termination_in_progress, 0);
+ rd_atomic32_init(&rkb->rkb_down_reported, 0);
rd_atomic64_init(&rkb->rkb_c.ts_send, 0);
rd_atomic64_init(&rkb->rkb_c.ts_recv, 0);
@@ -5095,7 +5151,7 @@ rd_kafka_broker_t *rd_kafka_broker_add_logical(rd_kafka_t *rk,
rd_assert(rkb && *"failed to create broker thread");
rd_kafka_wrunlock(rk);
- rd_atomic32_add(&rk->rk_broker_addrless_cnt, 1);
+ rd_atomic32_add(&rk->rk_logical_broker_cnt, 1);
rd_dassert(RD_KAFKA_BROKER_IS_LOGICAL(rkb));
rd_kafka_broker_keep(rkb);
@@ -5154,14 +5210,6 @@ void rd_kafka_broker_set_nodename(rd_kafka_broker_t *rkb,
rkb->rkb_nodename_epoch++;
changed = rd_true;
}
-
- if (rkb->rkb_nodeid != nodeid) {
- rd_rkb_dbg(rkb, BROKER, "NODEID",
- "Broker nodeid changed from %" PRId32 " to %" PRId32,
- rkb->rkb_nodeid, nodeid);
- rkb->rkb_nodeid = nodeid;
- }
-
rd_kafka_broker_unlock(rkb);
/* Update the log name to include (or exclude) the nodeid.
@@ -5174,11 +5222,6 @@ void rd_kafka_broker_set_nodename(rd_kafka_broker_t *rkb,
if (!changed)
return;
- if (!rd_kafka_broker_is_addrless(rkb))
- rd_atomic32_sub(&rkb->rkb_rk->rk_broker_addrless_cnt, 1);
- else
- rd_atomic32_add(&rkb->rkb_rk->rk_broker_addrless_cnt, 1);
-
/* Trigger a disconnect & reconnect */
rd_kafka_broker_schedule_connection(rkb);
}
@@ -5245,11 +5288,12 @@ static rd_kafka_broker_t *rd_kafka_broker_find(rd_kafka_t *rk,
rd_kafka_mk_nodename(nodename, sizeof(nodename), name, port);
TAILQ_FOREACH(rkb, &rk->rk_brokers, rkb_link) {
- if (RD_KAFKA_BROKER_IS_LOGICAL(rkb))
+ if (rd_kafka_broker_or_instance_terminating(rkb) ||
+ RD_KAFKA_BROKER_IS_LOGICAL(rkb))
continue;
rd_kafka_broker_lock(rkb);
- if (!rd_kafka_terminating(rk) && rkb->rkb_proto == proto &&
+ if (rkb->rkb_proto == proto &&
!strcmp(rkb->rkb_nodename, nodename)) {
rd_kafka_broker_keep(rkb);
rd_kafka_broker_unlock(rkb);
@@ -5492,6 +5536,9 @@ int rd_kafka_brokers_add0(rd_kafka_t *rk,
int rd_kafka_brokers_add(rd_kafka_t *rk, const char *brokerlist) {
+ rd_kafka_wrlock(rk);
+ rd_list_add(&rk->additional_brokerlists, rd_strdup(brokerlist));
+ rd_kafka_wrunlock(rk);
return rd_kafka_brokers_add0(rk, brokerlist, rd_false);
}
@@ -5530,12 +5577,6 @@ void rd_kafka_broker_update(rd_kafka_t *rk,
* the hostname. */
if (strcmp(rkb->rkb_nodename, nodename))
needs_update = 1;
- } else if ((rkb = rd_kafka_broker_find(rk, proto, mdb->host,
- mdb->port))) {
- /* Broker matched by hostname (but not by nodeid),
- * update the nodeid. */
- needs_update = 1;
-
} else if ((rkb = rd_kafka_broker_add(rk, RD_KAFKA_LEARNED, proto,
mdb->host, mdb->port, mdb->id))) {
rd_kafka_broker_keep(rkb);
@@ -5550,7 +5591,6 @@ void rd_kafka_broker_update(rd_kafka_t *rk,
rko = rd_kafka_op_new(RD_KAFKA_OP_NODE_UPDATE);
rd_strlcpy(rko->rko_u.node.nodename, nodename,
sizeof(rko->rko_u.node.nodename));
- rko->rko_u.node.nodeid = mdb->id;
/* Perform a blocking op request so that all
* broker-related state, such as the rk broker list,
* is up to date by the time this call returns.
@@ -5571,24 +5611,12 @@ void rd_kafka_broker_update(rd_kafka_t *rk,
* @returns the broker id, or RD_KAFKA_NODEID_UA if \p rkb is NULL.
*
* @locality any
- * @locks_required none
- * @locks_acquired rkb_lock
*/
int32_t rd_kafka_broker_id(rd_kafka_broker_t *rkb) {
- int32_t broker_id;
-
if (unlikely(!rkb))
return RD_KAFKA_NODEID_UA;
- /* Avoid locking if already on the broker thread */
- if (thrd_is_current(rkb->rkb_thread))
- return rkb->rkb_nodeid;
-
- rd_kafka_broker_lock(rkb);
- broker_id = rkb->rkb_nodeid;
- rd_kafka_broker_unlock(rkb);
-
- return broker_id;
+ return rkb->rkb_nodeid;
}
@@ -5673,6 +5701,17 @@ static int rd_kafka_broker_filter_never_connected(rd_kafka_broker_t *rkb,
return rd_atomic32_get(&rkb->rkb_c.connects);
}
+static void rd_kafka_connect_any_timer_cb(rd_kafka_timers_t *rkts, void *arg) {
+ const char *reason = (const char *)arg;
+ rd_kafka_t *rk = rkts->rkts_rk;
+ if (rd_kafka_terminating(rk))
+ return;
+
+ /* Acquire the read lock for `rd_kafka_connect_any` */
+ rd_kafka_rdlock(rk);
+ rd_kafka_connect_any(rk, reason);
+ rd_kafka_rdunlock(rk);
+}
/**
* @brief Sparse connections:
@@ -5692,15 +5731,14 @@ void rd_kafka_connect_any(rd_kafka_t *rk, const char *reason) {
* a specific purpose (group coordinator) and their connections
* should not be reused for other purposes.
* rd_kafka_broker_random() will not return LOGICAL brokers. */
- if (rd_atomic32_get(&rk->rk_broker_up_cnt) -
- rd_atomic32_get(&rk->rk_logical_broker_up_cnt) >
- 0 ||
+ if (rd_atomic32_get(&rk->rk_broker_up_cnt) > 0 ||
rd_atomic32_get(&rk->rk_broker_cnt) -
- rd_atomic32_get(&rk->rk_broker_addrless_cnt) ==
+ rd_atomic32_get(&rk->rk_logical_broker_cnt) ==
0)
return;
mtx_lock(&rk->rk_suppress.sparse_connect_lock);
+
suppr = rd_interval(&rk->rk_suppress.sparse_connect_random,
rk->rk_conf.sparse_connect_intvl * 1000, 0);
mtx_unlock(&rk->rk_suppress.sparse_connect_lock);
@@ -5710,6 +5748,11 @@ void rd_kafka_connect_any(rd_kafka_t *rk, const char *reason) {
"Not selecting any broker for cluster connection: "
"still suppressed for %" PRId64 "ms: %s",
-suppr / 1000, reason);
+ /* Retry after interval + 1ms has passed */
+ rd_kafka_timer_start_oneshot(
+ &rk->rk_timers, &rk->rk_suppress.sparse_connect_random_tmr,
+ rd_false /* don't restart */, 1000LL - suppr,
+ rd_kafka_connect_any_timer_cb, (void *)reason);
return;
}
@@ -5726,7 +5769,7 @@ void rd_kafka_connect_any(rd_kafka_t *rk, const char *reason) {
if (!rkb) {
/* No brokers matched:
- * this happens if there are brokers in > INIT state,
+ * this happens if all brokers are in > INIT state,
* in which case they're already connecting. */
rd_kafka_dbg(rk, BROKER | RD_KAFKA_DBG_GENERIC, "CONNECT",
@@ -5808,8 +5851,8 @@ static void rd_kafka_broker_handle_purge_queues(rd_kafka_broker_t *rkb,
* the protocol stream, so we need to disconnect from the broker
* to get a clean protocol socket. */
if (partial_cnt)
- rd_kafka_broker_fail(
- rkb, LOG_DEBUG, RD_KAFKA_RESP_ERR__PURGE_QUEUE,
+ rd_kafka_broker_planned_fail(
+ rkb, RD_KAFKA_RESP_ERR__PURGE_QUEUE,
"Purged %d partially sent request: "
"forcing disconnect",
partial_cnt);
@@ -5932,7 +5975,6 @@ void rd_kafka_broker_active_toppar_del(rd_kafka_broker_t *rkb,
*/
void rd_kafka_broker_schedule_connection(rd_kafka_broker_t *rkb) {
rd_kafka_op_t *rko;
-
rko = rd_kafka_op_new(RD_KAFKA_OP_CONNECT);
rd_kafka_op_set_prio(rko, RD_KAFKA_PRIO_FLASH);
rd_kafka_q_enq(rkb->rkb_ops, rko);
@@ -6110,6 +6152,174 @@ void rd_kafka_broker_start_reauth_cb(rd_kafka_timers_t *rkts, void *_rkb) {
rd_kafka_q_enq(rkb->rkb_ops, rko);
}
+int32_t *rd_kafka_brokers_learned_ids(rd_kafka_t *rk, size_t *cntp) {
+ rd_kafka_broker_t *rkb;
+ int32_t *ids, *p;
+ int32_t i;
+
+ *cntp = 0;
+ rd_kafka_rdlock(rk);
+ ids = malloc(sizeof(*ids) * rd_list_cnt(&rk->rk_broker_by_id));
+ p = ids;
+ RD_LIST_FOREACH(rkb, &rk->rk_broker_by_id, i) {
+ *p++ = rkb->rkb_nodeid;
+ (*cntp)++;
+ }
+ rd_kafka_rdunlock(rk);
+
+ return ids;
+}
+
+/**
+ * @brief Decommission a broker.
+ *
+ * @param rk Client instance.
+ * @param rkb Broker to decommission.
+ * @param wait_thrds Add the broker's thread to this list if not NULL.
+ *
+ * @locks rd_kafka_wrlock() is dropped and reacquired.
+ *
+ * Broker threads hold a refcount and detect when it reaches 1 and then
+ * decommissions itself. Callers can wait for this to happen by calling
+ * thrd_join() on elements of \p wait_thrds. Callers are responsible for
+ * managing the creation and destruction of \p wait_thrds which can be NULL.
+ */
+void rd_kafka_broker_decommission(rd_kafka_t *rk,
+ rd_kafka_broker_t *rkb,
+ rd_list_t *wait_thrds) {
+
+ if (rd_atomic32_get(&rkb->termination_in_progress) > 0)
+ return;
+
+ rd_atomic32_add(&rkb->termination_in_progress, 1);
+
+ /* Add broker's thread to wait_thrds list for later joining */
+ if (wait_thrds) {
+ thrd_t *thrd = rd_malloc(sizeof(*thrd));
+ *thrd = rkb->rkb_thread;
+
+ rd_list_add(wait_thrds, thrd);
+ }
+
+ rd_list_remove(&rk->rk_broker_by_id, rkb);
+ rd_kafka_wrunlock(rk);
+
+ rd_kafka_dbg(rk, BROKER, "DESTROY", "Sending TERMINATE to %s",
+ rd_kafka_broker_name(rkb));
+
+#ifndef _WIN32
+ /* Interrupt IO threads to speed up termination. */
+ if (rk->rk_conf.term_sig)
+ pthread_kill(rkb->rkb_thread, rk->rk_conf.term_sig);
+#endif
+
+ if (rk->rk_cgrp && rk->rk_cgrp->rkcg_curr_coord &&
+ rk->rk_cgrp->rkcg_curr_coord == rkb)
+ /* If we're decommissioning current coordinator handle,
+ * mark it as dead and decrease its reference count. */
+ rd_kafka_cgrp_coord_dead(rk->rk_cgrp,
+ RD_KAFKA_RESP_ERR__DESTROY_BROKER,
+ "Group coordinator decommissioned");
+ /* Send op to trigger queue/io wake-up.
+ * Broker thread will destroy this thread reference.
+ * WARNING: This is last time we can read from rkb in this thread! */
+ rd_kafka_q_enq(rkb->rkb_ops, rd_kafka_op_new(RD_KAFKA_OP_TERMINATE));
+
+ rd_kafka_wrlock(rk);
+}
+
+/**
+ * @brief Unittest for reconnect.backoff.ms
+ */
+static int rd_ut_reconnect_backoff(void) {
+ rd_kafka_broker_t rkb = RD_ZERO_INIT;
+ rd_kafka_conf_t conf = {.reconnect_backoff_ms = 10,
+ .reconnect_backoff_max_ms = 90};
+ rd_ts_t now = 1000000;
+ int backoff;
+
+ rkb.rkb_reconnect_backoff_ms = conf.reconnect_backoff_ms;
+
+ /* broker's backoff is the initial reconnect.backoff.ms=10 */
+ rd_kafka_broker_update_reconnect_backoff(&rkb, &conf, now);
+ backoff = rd_kafka_broker_reconnect_backoff(&rkb, now);
+ RD_UT_ASSERT_RANGE(backoff, 7, 15, "%d");
+
+ /* .. 20 */
+ rd_kafka_broker_update_reconnect_backoff(&rkb, &conf, now);
+ backoff = rd_kafka_broker_reconnect_backoff(&rkb, now);
+ RD_UT_ASSERT_RANGE(backoff, 15, 30, "%d");
+
+ /* .. 40 */
+ rd_kafka_broker_update_reconnect_backoff(&rkb, &conf, now);
+ backoff = rd_kafka_broker_reconnect_backoff(&rkb, now);
+ RD_UT_ASSERT_RANGE(backoff, 30, 60, "%d");
+
+ /* .. 80, the jitter is capped at reconnect.backoff.max.ms=90 */
+ rd_kafka_broker_update_reconnect_backoff(&rkb, &conf, now);
+ backoff = rd_kafka_broker_reconnect_backoff(&rkb, now);
+ RD_UT_ASSERT_RANGE(backoff, 60, conf.reconnect_backoff_max_ms, "%d");
+
+ /* .. 90, capped by reconnect.backoff.max.ms */
+ rd_kafka_broker_update_reconnect_backoff(&rkb, &conf, now);
+ backoff = rd_kafka_broker_reconnect_backoff(&rkb, now);
+ RD_UT_ASSERT_RANGE(backoff, 67, conf.reconnect_backoff_max_ms, "%d");
+
+ /* .. 90, should remain at capped value. */
+ rd_kafka_broker_update_reconnect_backoff(&rkb, &conf, now);
+ backoff = rd_kafka_broker_reconnect_backoff(&rkb, now);
+ RD_UT_ASSERT_RANGE(backoff, 67, conf.reconnect_backoff_max_ms, "%d");
+
+ RD_UT_PASS();
+}
+
+/**
+ * @brief Unittest for reconnect.backoff.ms
+ */
+static int rd_ut_ApiVersion_at_least(void) {
+ rd_kafka_broker_t rkb = RD_ZERO_INIT;
+ mtx_init(&rkb.rkb_lock, mtx_plain);
+
+ struct rd_kafka_ApiVersion av = {
+ .ApiKey = RD_KAFKAP_Metadata,
+ .MinVer = 5,
+ .MaxVer = 10,
+ };
+
+ rkb.rkb_ApiVersions_cnt = 1;
+ rkb.rkb_ApiVersions = &av;
+
+ RD_UT_ASSERT(
+ rd_kafka_broker_ApiVersion_at_least(&rkb, RD_KAFKAP_Metadata, 0),
+ "Metadata API version should be at least 0");
+ RD_UT_ASSERT(
+ rd_kafka_broker_ApiVersion_at_least(&rkb, RD_KAFKAP_Metadata, 3),
+ "Metadata API version should be at least 3");
+ RD_UT_ASSERT(
+ rd_kafka_broker_ApiVersion_at_least(&rkb, RD_KAFKAP_Metadata, 5),
+ "Metadata API version should be at least 5");
+ RD_UT_ASSERT(
+ rd_kafka_broker_ApiVersion_at_least(&rkb, RD_KAFKAP_Metadata, 7),
+ "Metadata API version should be at least 7");
+ RD_UT_ASSERT(
+ rd_kafka_broker_ApiVersion_at_least(&rkb, RD_KAFKAP_Metadata, 10),
+ "Metadata API version should be at least 10");
+ RD_UT_ASSERT(
+ !rd_kafka_broker_ApiVersion_at_least(&rkb, RD_KAFKAP_Metadata, 11),
+ "Metadata API version shouldn't be at least 11");
+
+ rkb.rkb_ApiVersions_cnt = 0;
+ RD_UT_ASSERT(
+ !rd_kafka_broker_ApiVersion_at_least(&rkb, RD_KAFKAP_Metadata, 0),
+ "Metadata API version shouldn't be at least 0");
+ RD_UT_ASSERT(
+ !rd_kafka_broker_ApiVersion_at_least(&rkb, RD_KAFKAP_Metadata, 3),
+ "Metadata API version shouldn't be at least 3");
+
+ mtx_destroy(&rkb.rkb_lock);
+ RD_UT_PASS();
+}
+
/**
* @name Unit tests
* @{
@@ -6119,6 +6329,7 @@ int unittest_broker(void) {
int fails = 0;
fails += rd_ut_reconnect_backoff();
+ fails += rd_ut_ApiVersion_at_least();
return fails;
}
diff --git a/src/rdkafka_broker.h b/src/rdkafka_broker.h
index 643e51edcd..a649b7445e 100644
--- a/src/rdkafka_broker.h
+++ b/src/rdkafka_broker.h
@@ -51,7 +51,6 @@ typedef enum {
/* Any state >= STATE_UP means the Kafka protocol layer
* is operational (to some degree). */
RD_KAFKA_BROKER_STATE_UP,
- RD_KAFKA_BROKER_STATE_UPDATE,
RD_KAFKA_BROKER_STATE_APIVERSION_QUERY,
RD_KAFKA_BROKER_STATE_AUTH_HANDSHAKE,
RD_KAFKA_BROKER_STATE_AUTH_REQ,
@@ -82,8 +81,7 @@ typedef struct rd_kafka_broker_monitor_s {
struct rd_kafka_broker_s { /* rd_kafka_broker_t */
TAILQ_ENTRY(rd_kafka_broker_s) rkb_link;
- int32_t rkb_nodeid; /**< Broker Node Id.
- * @locks rkb_lock */
+ int32_t rkb_nodeid; /**< Broker Node Id, read only. */
#define RD_KAFKA_NODEID_UA -1
rd_sockaddr_list_t *rkb_rsal;
@@ -189,8 +187,27 @@ struct rd_kafka_broker_s { /* rd_kafka_broker_t */
rd_atomic64_t reqtype[RD_KAFKAP__NUM]; /**< Per request-type
* counter */
- rd_atomic64_t ts_send; /**< Timestamp of last send */
- rd_atomic64_t ts_recv; /**< Timestamp of last receive */
+ rd_atomic64_t ts_send; /**< Timestamp of last send */
+ rd_atomic64_t ts_recv; /**< Timestamp of last receive */
+ rd_bool_t skip_broker_down; /**< Avoid reporting the
+ * broker down on next
+ * state change.
+ * Useful for a planned
+ * disconnection to avoid
+ * reaching the all
+ * brokers down state. */
+ int connections_max_idle_ms; /**< Maximum idle time
+ * for this broker connections.
+ * jitter is different for
+ * each broker to avoid the
+ * ALL_BROKERS_DOWN error.
+ * Initial value is `-1` as
+ * `connections.max.idle.ms=0`
+ * means disabled, but must
+ * never be used when
+ * `connections.max.idle.ms=0`
+ * for the same reason
+ * (an assert is present). */
} rkb_c;
struct {
@@ -198,21 +215,32 @@ struct rd_kafka_broker_s { /* rd_kafka_broker_t */
int32_t connects; /**< Connection attempts,
* successful or not. */
} rkb_historic_c;
+
struct {
rd_avg_t rkb_avg_rtt; /* Current RTT avg */
rd_avg_t rkb_avg_throttle; /* Current throttle avg */
rd_avg_t
- rkb_avg_outbuf_latency; /**< Current latency
- * between buf_enq0
- * and writing to socket
- */
+ rkb_avg_outbuf_latency; /**< Current latency
+ * between buf_enq0
+ * and writing to socket
+ */
+ rd_avg_t rkb_avg_fetch_latency; /**< Current fetch
+ * latency avg */
+ rd_avg_t rkb_avg_produce_latency; /**< Current produce
+ * latency avg */
} rd_avg_current;
+
struct {
rd_avg_t rkb_avg_rtt; /**< Rolled over RTT avg */
rd_avg_t
rkb_avg_throttle; /**< Rolled over throttle avg */
rd_avg_t rkb_avg_outbuf_latency; /**< Rolled over outbuf
* latency avg */
+ rd_avg_t rkb_avg_fetch_latency; /**< Rolled over fetch
+ * latency avg */
+ rd_avg_t
+ rkb_avg_produce_latency; /**< Rolled over produce
+ * latency avg */
} rd_avg_rollover;
} rkb_telemetry;
@@ -322,7 +350,9 @@ struct rd_kafka_broker_s { /* rd_kafka_broker_t */
rd_kafka_secproto_t rkb_proto;
- int rkb_down_reported; /* Down event reported */
+ /** Down event was reported for this broker
+ * after last connection to any broker. */
+ rd_atomic32_t rkb_down_reported;
#if WITH_SASL_CYRUS
rd_kafka_timer_t rkb_sasl_kinit_refresh_tmr;
#endif
@@ -354,6 +384,9 @@ struct rd_kafka_broker_s { /* rd_kafka_broker_t */
rd_kafka_timer_t rkb_sasl_reauth_tmr;
+
+ /** > 0 if this broker thread is terminating */
+ rd_atomic32_t termination_in_progress;
};
#define rd_kafka_broker_keep(rkb) rd_refcnt_add(&(rkb)->rkb_refcnt)
@@ -381,12 +414,28 @@ rd_kafka_broker_get_state(rd_kafka_broker_t *rkb) {
/**
- * @returns true if the broker state is UP or UPDATE
+ * @returns true if the broker state is UP
*/
-#define rd_kafka_broker_state_is_up(state) \
- ((state) == RD_KAFKA_BROKER_STATE_UP || \
- (state) == RD_KAFKA_BROKER_STATE_UPDATE)
+#define rd_kafka_broker_state_is_up(state) ((state) == RD_KAFKA_BROKER_STATE_UP)
+/**
+ * @returns true if the broker state is DOWN
+ */
+#define rd_kafka_broker_state_is_down(state) \
+ ((state) == RD_KAFKA_BROKER_STATE_DOWN)
+
+/**
+ * @returns true if the error is a broker destroy error, because of
+ * termination or because of decommissioning.
+ */
+#define rd_kafka_broker_is_any_err_destroy(err) \
+ ((err) == RD_KAFKA_RESP_ERR__DESTROY || \
+ (err) == RD_KAFKA_RESP_ERR__DESTROY_BROKER)
+
+
+#define rd_kafka_broker_or_instance_terminating(rkb) \
+ (rd_kafka_broker_termination_in_progress(rkb) || \
+ rd_kafka_terminating((rkb)->rkb_rk))
/**
* @returns true if the broker connection is up, else false.
@@ -399,6 +448,14 @@ rd_kafka_broker_is_up(rd_kafka_broker_t *rkb) {
return rd_kafka_broker_state_is_up(state);
}
+/**
+ * @returns true if the broker needs a persistent connection
+ * @locality any
+ */
+static RD_UNUSED RD_INLINE rd_bool_t
+rd_kafka_broker_termination_in_progress(rd_kafka_broker_t *rkb) {
+ return rd_atomic32_get(&rkb->termination_in_progress) > 0;
+}
/**
* @brief Broker comparator
@@ -441,6 +498,14 @@ int16_t rd_kafka_broker_ApiVersion_supported0(rd_kafka_broker_t *rkb,
int *featuresp,
rd_bool_t do_lock);
+rd_bool_t rd_kafka_broker_ApiVersion_at_least(rd_kafka_broker_t *rkb,
+ int16_t ApiKey,
+ int16_t minver);
+
+rd_bool_t rd_kafka_broker_ApiVersion_at_least_no_lock(rd_kafka_broker_t *rkb,
+ int16_t ApiKey,
+ int16_t minver);
+
rd_kafka_broker_t *rd_kafka_broker_find_by_nodeid0_fl(const char *func,
int line,
rd_kafka_t *rk,
@@ -670,6 +735,10 @@ void rd_kafka_broker_start_reauth_timer(rd_kafka_broker_t *rkb,
void rd_kafka_broker_start_reauth_cb(rd_kafka_timers_t *rkts, void *rkb);
+void rd_kafka_broker_decommission(rd_kafka_t *rk,
+ rd_kafka_broker_t *rkb,
+ rd_list_t *wait_thrds);
+
int unittest_broker(void);
#endif /* _RDKAFKA_BROKER_H_ */
diff --git a/src/rdkafka_buf.c b/src/rdkafka_buf.c
index 292c21819c..012835de08 100644
--- a/src/rdkafka_buf.c
+++ b/src/rdkafka_buf.c
@@ -42,9 +42,6 @@ void rd_kafka_buf_destroy_final(rd_kafka_buf_t *rkbuf) {
rd_list_destroy(rkbuf->rkbuf_u.Metadata.topic_ids);
if (rkbuf->rkbuf_u.Metadata.reason)
rd_free(rkbuf->rkbuf_u.Metadata.reason);
- if (rkbuf->rkbuf_u.Metadata.rko)
- rd_kafka_op_reply(rkbuf->rkbuf_u.Metadata.rko,
- RD_KAFKA_RESP_ERR__DESTROY);
if (rkbuf->rkbuf_u.Metadata.decr) {
/* Decrease metadata cache's full_.._sent state. */
mtx_lock(rkbuf->rkbuf_u.Metadata.decr_lock);
@@ -244,6 +241,12 @@ void rd_kafka_bufq_init(rd_kafka_bufq_t *rkbufq) {
rd_atomic32_init(&rkbufq->rkbq_msg_cnt, 0);
}
+static void rd_kafka_bufq_reset(rd_kafka_bufq_t *rkbufq) {
+ TAILQ_INIT(&rkbufq->rkbq_bufs);
+ rd_atomic32_set(&rkbufq->rkbq_cnt, 0);
+ rd_atomic32_set(&rkbufq->rkbq_msg_cnt, 0);
+}
+
/**
* Concat all buffers from 'src' to tail of 'dst'
*/
@@ -252,7 +255,7 @@ void rd_kafka_bufq_concat(rd_kafka_bufq_t *dst, rd_kafka_bufq_t *src) {
(void)rd_atomic32_add(&dst->rkbq_cnt, rd_atomic32_get(&src->rkbq_cnt));
(void)rd_atomic32_add(&dst->rkbq_msg_cnt,
rd_atomic32_get(&src->rkbq_msg_cnt));
- rd_kafka_bufq_init(src);
+ rd_kafka_bufq_reset(src);
}
/**
@@ -393,7 +396,7 @@ int rd_kafka_buf_retry(rd_kafka_broker_t *rkb, rd_kafka_buf_t *rkbuf) {
rd_assert(rd_buf_len(&rkbuf->rkbuf_buf) > 0);
if (unlikely(!rkb || rkb->rkb_source == RD_KAFKA_INTERNAL ||
- rd_kafka_terminating(rkb->rkb_rk) ||
+ rd_kafka_broker_or_instance_terminating(rkb) ||
rkbuf->rkbuf_retries + incr_retry >
rkbuf->rkbuf_max_retries))
return 0;
diff --git a/src/rdkafka_buf.h b/src/rdkafka_buf.h
index 37938999dd..9682d099a0 100644
--- a/src/rdkafka_buf.h
+++ b/src/rdkafka_buf.h
@@ -379,11 +379,12 @@ struct rd_kafka_buf_s { /* rd_kafka_buf_t */
rd_list_t *
topic_ids; /* Requested topic ids rd_kafka_Uuid_t */
char *reason; /* Textual reason */
- rd_kafka_op_t *rko; /* Originating rko with replyq
- * (if any) */
rd_bool_t all_topics; /**< Full/All topics requested */
rd_bool_t cgrp_update; /**< Update cgrp with topic
* status from response. */
+ int32_t cgrp_subscription_version;
+ /**< Consumer group subscription version, to
+ * check before updating cgrp state. */
rd_bool_t force_racks; /**< Force the returned metadata
* to contain partition to
* rack mapping. */
@@ -524,7 +525,7 @@ struct rd_kafka_buf_s { /* rd_kafka_buf_t */
#define rd_kafka_buf_skip_to(rkbuf, pos) \
do { \
size_t __len1 = \
- (size_t)(pos)-rd_slice_offset(&(rkbuf)->rkbuf_reader); \
+ (size_t)(pos) - rd_slice_offset(&(rkbuf)->rkbuf_reader); \
if (__len1 && \
!rd_slice_read(&(rkbuf)->rkbuf_reader, NULL, __len1)) \
rd_kafka_buf_check_len(rkbuf, __len1); \
diff --git a/src/rdkafka_cert.c b/src/rdkafka_cert.c
index a14814d0a1..e4393c3288 100644
--- a/src/rdkafka_cert.c
+++ b/src/rdkafka_cert.c
@@ -78,6 +78,8 @@ static void rd_kafka_cert_destroy(rd_kafka_cert_t *cert) {
if (cert->x509)
X509_free(cert->x509);
+ if (cert->chain)
+ sk_X509_pop_free(cert->chain, X509_free);
if (cert->pkey)
EVP_PKEY_free(cert->pkey);
if (cert->store)
@@ -314,10 +316,11 @@ static rd_kafka_cert_t *rd_kafka_cert_new(const rd_kafka_conf_t *conf,
switch (encoding) {
case RD_KAFKA_CERT_ENC_PKCS12: {
EVP_PKEY *ign_pkey;
+ STACK_OF(X509) *ca = NULL;
action = "parse PKCS#12";
if (!PKCS12_parse(p12, conf->ssl.key_password,
- &ign_pkey, &cert->x509, NULL))
+ &ign_pkey, &cert->x509, &ca))
goto fail;
EVP_PKEY_free(ign_pkey);
@@ -325,6 +328,13 @@ static rd_kafka_cert_t *rd_kafka_cert_new(const rd_kafka_conf_t *conf,
action = "retrieve public key";
if (!cert->x509)
goto fail;
+
+ if (ca) {
+ if (sk_X509_num(ca) > 0)
+ cert->chain = ca;
+ else
+ sk_X509_pop_free(ca, X509_free);
+ }
} break;
case RD_KAFKA_CERT_ENC_DER:
@@ -341,6 +351,20 @@ static rd_kafka_cert_t *rd_kafka_cert_new(const rd_kafka_conf_t *conf,
(void *)conf);
if (!cert->x509)
goto fail;
+
+ cert->chain = sk_X509_new_null();
+ if (rd_kafka_ssl_read_cert_chain_from_BIO(
+ bio, cert->chain, rd_kafka_conf_ssl_passwd_cb,
+ (void *)conf) != 0) {
+ sk_X509_pop_free(cert->chain, X509_free);
+ cert->chain = NULL;
+ goto fail;
+ }
+
+ if (sk_X509_num(cert->chain) == 0) {
+ sk_X509_pop_free(cert->chain, X509_free);
+ cert->chain = NULL;
+ }
break;
default:
diff --git a/src/rdkafka_cert.h b/src/rdkafka_cert.h
index 819773ba30..753223c539 100644
--- a/src/rdkafka_cert.h
+++ b/src/rdkafka_cert.h
@@ -43,9 +43,10 @@ typedef struct rd_kafka_cert_s {
rd_kafka_cert_enc_t encoding;
rd_refcnt_t refcnt;
#if WITH_SSL
- X509 *x509; /**< Certificate (public key) */
- EVP_PKEY *pkey; /**< Private key */
- X509_STORE *store; /**< CA certificate chain store */
+ X509 *x509; /**< Certificate (public key) */
+ STACK_OF(X509) * chain; /**< Certificate chain (public key) */
+ EVP_PKEY *pkey; /**< Private key */
+ X509_STORE *store; /**< CA trusted certificates */
#endif
} rd_kafka_cert_t;
diff --git a/src/rdkafka_cgrp.c b/src/rdkafka_cgrp.c
index 1917991ddd..0d85cbde32 100644
--- a/src/rdkafka_cgrp.c
+++ b/src/rdkafka_cgrp.c
@@ -177,9 +177,7 @@ rd_kafka_cgrp_rebalance_protocol(rd_kafka_cgrp_t *rkcg) {
RD_KAFKA_CGRP_CONSUMER_F_SUBSCRIBED_ONCE))
return RD_KAFKA_REBALANCE_PROTOCOL_NONE;
- return rkcg->rkcg_rk->rk_conf.partition_assignors_cooperative
- ? RD_KAFKA_REBALANCE_PROTOCOL_COOPERATIVE
- : RD_KAFKA_REBALANCE_PROTOCOL_EAGER;
+ return RD_KAFKA_REBALANCE_PROTOCOL_COOPERATIVE;
}
if (!rkcg->rkcg_assignor)
@@ -265,11 +263,28 @@ typedef RD_MAP_TYPE(const rd_kafka_topic_partition_t *,
/**
* @returns true if consumer has joined the group and thus requires a leave.
+ *
+ * `rkcg_member_id` is sufficient to know this with "classic" group protocol.
*/
-#define RD_KAFKA_CGRP_HAS_JOINED(rkcg) \
- (rkcg->rkcg_member_id != NULL && \
+#define RD_KAFKA_CGRP_HAS_JOINED_CLASSIC(rkcg) \
+ (rkcg->rkcg_group_protocol == RD_KAFKA_GROUP_PROTOCOL_CLASSIC && \
+ rkcg->rkcg_member_id != NULL && \
RD_KAFKAP_STR_LEN((rkcg)->rkcg_member_id) > 0)
+/**
+ * @returns true if consumer has joined the group and thus requires a leave.
+ *
+ * With "consumer" group protocol we cannot rely on the `rkcg_member_id`
+ * as it's client generated.
+ */
+#define RD_KAFKA_CGRP_HAS_JOINED_CONSUMER(rkcg) \
+ (rkcg->rkcg_group_protocol == RD_KAFKA_GROUP_PROTOCOL_CONSUMER && \
+ rkcg->rkcg_generation_id > 0)
+
+#define RD_KAFKA_CGRP_HAS_JOINED(rkcg) \
+ (RD_KAFKA_CGRP_HAS_JOINED_CLASSIC(rkcg) || \
+ RD_KAFKA_CGRP_HAS_JOINED_CONSUMER(rkcg))
+
/**
* @returns true if cgrp is waiting for a rebalance_cb to be handled by
@@ -355,11 +370,53 @@ static int rd_kafka_cgrp_set_state(rd_kafka_cgrp_t *rkcg, int state) {
return 1;
}
+/**
+ * @brief Set the cgrp last error and current timestamp
+ * as last error timestamp.
+ */
+static void rd_kafka_cgrp_set_last_err(rd_kafka_cgrp_t *rkcg,
+ rd_kafka_resp_err_t rkcg_last_err) {
+ rkcg->rkcg_last_err = rkcg_last_err;
+ rkcg->rkcg_ts_last_err = rd_clock();
+}
+
+/**
+ * @brief Clears cgrp last error and its timestamp.
+ */
+static void rd_kafka_cgrp_clear_last_err(rd_kafka_cgrp_t *rkcg) {
+ rkcg->rkcg_last_err = RD_KAFKA_RESP_ERR_NO_ERROR;
+ rkcg->rkcg_ts_last_err = 0;
+}
+
+/**
+ * @brief Clears cgrp last error if it's an heartbeat related error like
+ * a topic authorization failed one.
+ */
+static void
+rd_kafka_cgrp_maybe_clear_heartbeat_failed_err(rd_kafka_cgrp_t *rkcg) {
+ if (rkcg->rkcg_last_err ==
+ RD_KAFKA_RESP_ERR_TOPIC_AUTHORIZATION_FAILED) {
+ rd_kafka_cgrp_clear_last_err(rkcg);
+ }
+}
+
void rd_kafka_cgrp_set_join_state(rd_kafka_cgrp_t *rkcg, int join_state) {
if ((int)rkcg->rkcg_join_state == join_state)
return;
+ if (rkcg->rkcg_join_state == RD_KAFKA_CGRP_JOIN_STATE_INIT ||
+ rkcg->rkcg_join_state == RD_KAFKA_CGRP_JOIN_STATE_STEADY) {
+ /* Start timer when leaving the INIT or STEADY state */
+ rkcg->rkcg_ts_rebalance_start = rd_clock();
+ } else if (join_state == RD_KAFKA_CGRP_JOIN_STATE_STEADY) {
+ /* End timer when reaching the STEADY state */
+ rd_dassert(rkcg->rkcg_ts_rebalance_start);
+ rd_avg_add(&rkcg->rkcg_rk->rk_telemetry.rd_avg_current
+ .rk_avg_rebalance_latency,
+ rd_clock() - rkcg->rkcg_ts_rebalance_start);
+ }
+
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "CGRPJOINSTATE",
"Group \"%.*s\" changed join state %s -> %s "
"(state %s)",
@@ -374,6 +431,8 @@ void rd_kafka_cgrp_set_join_state(rd_kafka_cgrp_t *rkcg, int join_state) {
void rd_kafka_cgrp_destroy_final(rd_kafka_cgrp_t *rkcg) {
rd_kafka_assert(rkcg->rkcg_rk, !rkcg->rkcg_subscription);
rd_kafka_assert(rkcg->rkcg_rk, !rkcg->rkcg_group_leader.members);
+ rd_kafka_assert(rkcg->rkcg_rk, !rkcg->rkcg_subscription_topics);
+ rd_kafka_assert(rkcg->rkcg_rk, !rkcg->rkcg_subscription_regex);
rd_kafka_cgrp_set_member_id(rkcg, NULL);
rd_kafka_topic_partition_list_destroy(rkcg->rkcg_current_assignment);
RD_IF_FREE(rkcg->rkcg_target_assignment,
@@ -447,6 +506,7 @@ rd_kafka_cgrp_t *rd_kafka_cgrp_new(rd_kafka_t *rk,
rd_kafkap_str_new(rk->rk_conf.group_instance_id, -1);
rkcg->rkcg_group_remote_assignor =
rd_kafkap_str_new(rk->rk_conf.group_remote_assignor, -1);
+
if (!RD_KAFKAP_STR_LEN(rkcg->rkcg_rk->rk_conf.client_rack))
rkcg->rkcg_client_rack = rd_kafkap_str_new(NULL, -1);
else
@@ -455,15 +515,24 @@ rd_kafka_cgrp_t *rd_kafka_cgrp_new(rd_kafka_t *rk,
rkcg->rkcg_next_subscription = NULL;
TAILQ_INIT(&rkcg->rkcg_topics);
rd_list_init(&rkcg->rkcg_toppars, 32, NULL);
- rd_kafka_cgrp_set_member_id(rkcg, "");
+
+ if (rkcg->rkcg_group_protocol == RD_KAFKA_GROUP_PROTOCOL_CONSUMER) {
+ rd_kafka_Uuid_t uuid = rd_kafka_Uuid_random();
+ rd_kafka_cgrp_set_member_id(rkcg,
+ rd_kafka_Uuid_base64str(&uuid));
+ } else {
+ rd_kafka_cgrp_set_member_id(rkcg, "");
+ }
+
rkcg->rkcg_subscribed_topics =
- rd_list_new(0, (void *)rd_kafka_topic_info_destroy);
+ rd_list_new(0, rd_kafka_topic_info_destroy_free);
rd_interval_init(&rkcg->rkcg_coord_query_intvl);
rd_interval_init(&rkcg->rkcg_heartbeat_intvl);
rd_interval_init(&rkcg->rkcg_join_intvl);
rd_interval_init(&rkcg->rkcg_timeout_scan_intvl);
rd_atomic32_init(&rkcg->rkcg_assignment_lost, rd_false);
rd_atomic32_init(&rkcg->rkcg_terminated, rd_false);
+ rd_atomic32_init(&rkcg->rkcg_subscription_version, 0);
rkcg->rkcg_current_assignment = rd_kafka_topic_partition_list_new(0);
rkcg->rkcg_target_assignment = NULL;
rkcg->rkcg_next_target_assignment = NULL;
@@ -486,13 +555,6 @@ rd_kafka_cgrp_t *rd_kafka_cgrp_new(rd_kafka_t *rk,
rk->rk_conf.auto_commit_interval_ms * 1000ll,
rd_kafka_cgrp_offset_commit_tmr_cb, rkcg);
- if (rkcg->rkcg_group_protocol == RD_KAFKA_GROUP_PROTOCOL_CONSUMER) {
- rd_kafka_log(
- rk, LOG_WARNING, "CGRP",
- "KIP-848 Consumer Group Protocol is in Early Access "
- "and MUST NOT be used in production");
- }
-
return rkcg;
}
@@ -586,6 +648,9 @@ static int rd_kafka_cgrp_coord_update(rd_kafka_cgrp_t *rkcg, int32_t coord_id) {
/* Clear previous broker handle, if any */
if (rkcg->rkcg_curr_coord)
rd_kafka_cgrp_coord_clear_broker(rkcg);
+
+ rd_kafka_cgrp_consumer_expedite_next_heartbeat(
+ rkcg, "coordinator changed");
}
@@ -720,6 +785,8 @@ static void rd_kafka_cgrp_handle_FindCoordinator(rd_kafka_t *rk,
RD_KAFKA_ERR_ACTION_RETRY, RD_KAFKA_RESP_ERR__TIMED_OUT_QUEUE,
+ RD_KAFKA_ERR_ACTION_RETRY, RD_KAFKA_RESP_ERR__DESTROY_BROKER,
+
RD_KAFKA_ERR_ACTION_END);
@@ -736,7 +803,13 @@ static void rd_kafka_cgrp_handle_FindCoordinator(rd_kafka_t *rk,
"FindCoordinator response error: %s", errstr);
/* Suppress repeated errors */
- rkcg->rkcg_last_err = ErrorCode;
+ rd_kafka_cgrp_set_last_err(rkcg, ErrorCode);
+ }
+
+ if (ErrorCode == RD_KAFKA_RESP_ERR__DESTROY_BROKER) {
+ /* This error is one-time and should cause
+ * an immediate retry. */
+ rd_interval_reset(&rkcg->rkcg_coord_query_intvl);
}
/* Retries are performed by the timer-intervalled
@@ -842,6 +915,44 @@ rd_kafka_broker_t *rd_kafka_cgrp_get_coord(rd_kafka_cgrp_t *rkcg) {
return rkcg->rkcg_coord;
}
+#define rd_kafka_cgrp_will_leave(rkcg) \
+ (rkcg->rkcg_flags & (RD_KAFKA_CGRP_F_LEAVE_ON_UNASSIGN_DONE | \
+ RD_KAFKA_CGRP_F_WAIT_LEAVE))
+
+#define rd_kafka_cgrp_consumer_will_rejoin(rkcg) \
+ (rkcg->rkcg_consumer_flags & \
+ (RD_KAFKA_CGRP_CONSUMER_F_WAIT_REJOIN | \
+ RD_KAFKA_CGRP_CONSUMER_F_WAIT_REJOIN_TO_COMPLETE))
+
+#define rd_kafka_cgrp_consumer_subscription_preconditions_met(rkcg) \
+ (!RD_KAFKA_CGRP_REBALANCING(rkcg) && \
+ rkcg->rkcg_consumer_flags & \
+ RD_KAFKA_CGRP_CONSUMER_F_SEND_NEW_SUBSCRIPTION)
+
+static int32_t
+rd_kafka_cgrp_subscription_set(rd_kafka_cgrp_t *rkcg,
+ rd_kafka_topic_partition_list_t *rktparlist);
+
+/**
+ * @brief Apply next subscription in \p rkcg , if set.
+ */
+static void rd_kafka_cgrp_consumer_apply_next_subscribe(rd_kafka_cgrp_t *rkcg) {
+ if (rkcg->rkcg_next_subscription) {
+ if (unlikely(rkcg->rkcg_flags & RD_KAFKA_CGRP_F_TERMINATE)) {
+ rd_kafka_topic_partition_list_destroy(
+ rkcg->rkcg_next_subscription);
+ rkcg->rkcg_next_subscription = NULL;
+ return;
+ }
+
+ rd_kafka_cgrp_subscription_set(rkcg,
+ rkcg->rkcg_next_subscription);
+ rkcg->rkcg_next_subscription = NULL;
+
+ rd_kafka_cgrp_consumer_expedite_next_heartbeat(
+ rkcg, "subscription changed");
+ }
+}
/**
* @brief cgrp handling of LeaveGroup responses
@@ -998,7 +1109,9 @@ static void rd_kafka_cgrp_consumer_leave(rd_kafka_cgrp_t *rkcg) {
rkcg->rkcg_coord, rkcg->rkcg_group_id, rkcg->rkcg_member_id,
member_epoch, rkcg->rkcg_group_instance_id,
NULL /* no rack */, -1 /* no rebalance_timeout_ms */,
- NULL /* no subscription */, NULL /* no remote assignor */,
+ NULL /* no subscription topics */,
+ NULL /* no regex subscription */,
+ NULL /* no remote assignor */,
NULL /* no current assignment */,
RD_KAFKA_REPLYQ(rkcg->rkcg_ops, 0),
rd_kafka_cgrp_handle_ConsumerGroupHeartbeat_leave, rkcg);
@@ -1327,6 +1440,50 @@ static void rd_kafka_rebalance_op(rd_kafka_cgrp_t *rkcg,
rd_kafka_cgrp_group_assignment_set(rkcg, NULL);
}
+/**
+ * @brief Rejoin the group (KIP-848).
+ */
+static void
+rd_kafka_cgrp_consumer_rejoin(rd_kafka_cgrp_t *rkcg, const char *fmt, ...) {
+ char reason[512];
+ va_list ap;
+ char astr[128];
+
+ va_start(ap, fmt);
+ rd_vsnprintf(reason, sizeof(reason), fmt, ap);
+ va_end(ap);
+
+ if (rkcg->rkcg_group_assignment)
+ rd_snprintf(astr, sizeof(astr), " with %d owned partition(s)",
+ rkcg->rkcg_group_assignment->cnt);
+ else
+ rd_snprintf(astr, sizeof(astr), " without an assignment");
+
+ if (rkcg->rkcg_subscription || rkcg->rkcg_next_subscription) {
+ rd_kafka_dbg(
+ rkcg->rkcg_rk, CONSUMER | RD_KAFKA_DBG_CGRP, "REJOIN",
+ "Group \"%s\": %s group%s: %s", rkcg->rkcg_group_id->str,
+ rkcg->rkcg_join_state == RD_KAFKA_CGRP_JOIN_STATE_INIT
+ ? "Joining"
+ : "Rejoining",
+ astr, reason);
+ } else {
+ rd_kafka_dbg(
+ rkcg->rkcg_rk, CONSUMER | RD_KAFKA_DBG_CGRP, "NOREJOIN",
+ "Group \"%s\": Not %s group%s: %s: "
+ "no subscribed topics",
+ rkcg->rkcg_group_id->str,
+ rkcg->rkcg_join_state == RD_KAFKA_CGRP_JOIN_STATE_INIT
+ ? "joining"
+ : "rejoining",
+ astr, reason);
+ }
+
+ rd_kafka_cgrp_leave_maybe(rkcg);
+ rd_kafka_cgrp_consumer_reset(rkcg);
+ rd_kafka_cgrp_set_join_state(rkcg, RD_KAFKA_CGRP_JOIN_STATE_INIT);
+ rd_kafka_cgrp_consumer_expedite_next_heartbeat(rkcg, "rejoining");
+}
/**
* @brief Rejoin the group.
@@ -1341,6 +1498,10 @@ static void rd_kafka_cgrp_rejoin(rd_kafka_cgrp_t *rkcg, const char *fmt, ...) {
char reason[512];
va_list ap;
char astr[128];
+ if (rkcg->rkcg_group_protocol == RD_KAFKA_GROUP_PROTOCOL_CONSUMER) {
+ rd_kafka_cgrp_consumer_rejoin(rkcg, fmt, ap);
+ return;
+ }
va_start(ap, fmt);
rd_vsnprintf(reason, sizeof(reason), fmt, ap);
@@ -1374,9 +1535,7 @@ static void rd_kafka_cgrp_rejoin(rd_kafka_cgrp_t *rkcg, const char *fmt, ...) {
rd_kafka_cgrp_leave_maybe(rkcg);
}
- rd_kafka_cgrp_consumer_reset(rkcg);
rd_kafka_cgrp_set_join_state(rkcg, RD_KAFKA_CGRP_JOIN_STATE_INIT);
- rd_kafka_cgrp_consumer_expedite_next_heartbeat(rkcg, "rejoining");
}
@@ -2220,6 +2379,8 @@ static void rd_kafka_cgrp_handle_JoinGroup(rd_kafka_t *rk,
* on receiving the response since some topics
* may be missing. */
rd_false,
+ /* cgrp_update=false: no subscription version is used */
+ -1,
/* force_racks is true if any memeber has a client rack set,
since we will require partition to rack mapping in that
case for rack-aware assignors. */
@@ -2324,7 +2485,10 @@ static rd_kafka_op_res_t rd_kafka_cgrp_handle_Metadata_op(rd_kafka_t *rk,
if (rko->rko_err == RD_KAFKA_RESP_ERR__DESTROY)
return RD_KAFKA_OP_RES_HANDLED; /* Terminating */
- rd_kafka_cgrp_metadata_update_check(rkcg, rd_false /*dont rejoin*/);
+ if (rd_atomic32_get(&rkcg->rkcg_subscription_version) ==
+ rko->rko_u.metadata.subscription_version)
+ rd_kafka_cgrp_metadata_update_check(rkcg,
+ rd_false /*dont rejoin*/);
return RD_KAFKA_OP_RES_HANDLED;
}
@@ -2341,6 +2505,7 @@ static rd_kafka_op_res_t rd_kafka_cgrp_handle_Metadata_op(rd_kafka_t *rk,
*/
static int rd_kafka_cgrp_metadata_refresh(rd_kafka_cgrp_t *rkcg,
int *metadata_agep,
+ int32_t cgrp_subscription_version,
const char *reason) {
rd_kafka_t *rk = rkcg->rkcg_rk;
rd_kafka_op_t *rko;
@@ -2350,8 +2515,8 @@ static int rd_kafka_cgrp_metadata_refresh(rd_kafka_cgrp_t *rkcg,
rd_list_init(&topics, 8, rd_free);
/* Insert all non-wildcard topics in cache. */
- rd_kafka_metadata_cache_hint_rktparlist(
- rkcg->rkcg_rk, rkcg->rkcg_subscription, NULL, 0 /*dont replace*/);
+ rd_kafka_metadata_cache_hint_rktparlist(rkcg->rkcg_rk,
+ rkcg->rkcg_subscription, NULL);
if (rkcg->rkcg_flags & RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION) {
/* For wildcard subscriptions make sure the
@@ -2409,10 +2574,19 @@ static int rd_kafka_cgrp_metadata_refresh(rd_kafka_cgrp_t *rkcg,
rd_kafka_cgrp_handle_Metadata_op);
rd_kafka_op_set_replyq(rko, rkcg->rkcg_ops, 0);
- err = rd_kafka_metadata_request(rkcg->rkcg_rk, NULL, &topics,
- rd_false /*!allow auto create */,
- rd_true /*cgrp_update*/, reason, rko);
+ err = rd_kafka_metadata_request(
+ rkcg->rkcg_rk, NULL, &topics, rd_false /*!allow auto create */,
+ rd_true /*cgrp_update*/, cgrp_subscription_version, reason, rko);
if (err) {
+ /* Hint cache that something is interested in
+ * these topics so that they will be included in
+ * a future all known_topics query. */
+
+ rd_kafka_wrlock(rk);
+ rd_kafka_metadata_cache_hint(rk, &topics, NULL,
+ RD_KAFKA_RESP_ERR__NOENT);
+ rd_kafka_wrunlock(rk);
+
rd_kafka_dbg(rk, CGRP | RD_KAFKA_DBG_METADATA, "CGRPMETADATA",
"%s: need to refresh metadata (%dms old) "
"but no usable brokers available: %s",
@@ -2427,8 +2601,9 @@ static int rd_kafka_cgrp_metadata_refresh(rd_kafka_cgrp_t *rkcg,
-static void rd_kafka_cgrp_join(rd_kafka_cgrp_t *rkcg) {
- int metadata_age;
+static void rd_kafka_cgrp_join(rd_kafka_cgrp_t *rkcg,
+ int32_t cgrp_subscription_version) {
+ int metadata_age, metadata_refresh_outcome;
if (rkcg->rkcg_state != RD_KAFKA_CGRP_STATE_UP ||
rkcg->rkcg_join_state != RD_KAFKA_CGRP_JOIN_STATE_INIT ||
@@ -2462,8 +2637,9 @@ static void rd_kafka_cgrp_join(rd_kafka_cgrp_t *rkcg) {
*/
/* We need up-to-date full metadata to continue,
* refresh metadata if necessary. */
- if (rd_kafka_cgrp_metadata_refresh(rkcg, &metadata_age,
- "consumer join") == 1) {
+ metadata_refresh_outcome = rd_kafka_cgrp_metadata_refresh(
+ rkcg, &metadata_age, cgrp_subscription_version, "consumer join");
+ if (metadata_refresh_outcome == 1) {
rd_kafka_dbg(rkcg->rkcg_rk, CGRP | RD_KAFKA_DBG_CONSUMER,
"JOIN",
"Group \"%.*s\": "
@@ -2480,6 +2656,14 @@ static void rd_kafka_cgrp_join(rd_kafka_cgrp_t *rkcg) {
rkcg, RD_KAFKA_CGRP_JOIN_STATE_WAIT_METADATA);
return; /* ^ async call */
+ } else if (metadata_refresh_outcome == -1) {
+ rd_kafka_dbg(rkcg->rkcg_rk, CGRP | RD_KAFKA_DBG_CONSUMER,
+ "JOIN",
+ "Group \"%.*s\": "
+ "postponing join until up-to-date "
+ "metadata can be requested",
+ RD_KAFKAP_STR_PR(rkcg->rkcg_group_id));
+ return; /* ^ async call */
}
if (rd_list_empty(rkcg->rkcg_subscribed_topics))
@@ -2566,7 +2750,7 @@ static rd_bool_t rd_kafka_cgrp_update_subscribed_topics(rd_kafka_cgrp_t *rkcg,
"clearing subscribed topics list (%d)",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_list_cnt(rkcg->rkcg_subscribed_topics));
- tinfos = rd_list_new(0, (void *)rd_kafka_topic_info_destroy);
+ tinfos = rd_list_new(0, rd_kafka_topic_info_destroy_free);
} else {
if (rd_list_cnt(tinfos) == 0)
@@ -2867,7 +3051,8 @@ void rd_kafka_cgrp_consumer_next_target_assignment_request_metadata(
rd_kafka_op_set_replyq(rko, rkcg->rkcg_ops, NULL);
rd_kafka_MetadataRequest(
rkb, NULL, missing_topic_ids, "ConsumerGroupHeartbeat API Response",
- rd_false /*!allow_auto_create*/, rd_false, rd_false, rko);
+ rd_false /*!allow_auto_create*/, rd_false,
+ -1 /* no subscription version is used */, rd_false, rko);
rd_list_destroy(missing_topic_ids);
}
@@ -2884,7 +3069,7 @@ void rd_kafka_cgrp_handle_ConsumerGroupHeartbeat(rd_kafka_t *rk,
const int log_decode_errors = LOG_ERR;
int16_t error_code = 0;
int actions = 0;
- rd_kafkap_str_t error_str;
+ rd_kafkap_str_t error_str = RD_KAFKAP_STR_INITIALIZER_EMPTY;
rd_kafkap_str_t member_id;
int32_t member_epoch;
int32_t heartbeat_interval_ms;
@@ -2908,19 +3093,30 @@ void rd_kafka_cgrp_handle_ConsumerGroupHeartbeat(rd_kafka_t *rk,
}
rd_kafka_buf_read_str(rkbuf, &member_id);
- rd_kafka_buf_read_i32(rkbuf, &member_epoch);
- rd_kafka_buf_read_i32(rkbuf, &heartbeat_interval_ms);
-
- int8_t are_assignments_present;
- rd_kafka_buf_read_i8(rkbuf, &are_assignments_present);
if (!RD_KAFKAP_STR_IS_NULL(&member_id)) {
rd_kafka_cgrp_set_member_id(rkcg, member_id.str);
}
+
+ rd_kafka_buf_read_i32(rkbuf, &member_epoch);
rkcg->rkcg_generation_id = member_epoch;
+
+ rd_kafka_dbg(rk, CGRP, "HEARTBEAT",
+ "ConsumerGroupHeartbeat response received for "
+ "member id \"%.*s\" with epoch %d",
+ RD_KAFKAP_STR_PR(&member_id), member_epoch);
+
+ if (rd_kafka_cgrp_will_leave(rkcg)) {
+ err = RD_KAFKA_RESP_ERR__OUTDATED;
+ goto err;
+ }
+
+ rd_kafka_buf_read_i32(rkbuf, &heartbeat_interval_ms);
if (heartbeat_interval_ms > 0) {
rkcg->rkcg_heartbeat_intvl_ms = heartbeat_interval_ms;
}
+ int8_t are_assignments_present;
+ rd_kafka_buf_read_i8(rkbuf, &are_assignments_present);
if (are_assignments_present == 1) {
rd_kafka_topic_partition_list_t *assigned_topic_partitions;
const rd_kafka_topic_partition_field_t assignments_fields[] = {
@@ -2940,11 +3136,10 @@ void rd_kafka_cgrp_handle_ConsumerGroupHeartbeat(rd_kafka_t *rk,
sizeof(assigned_topic_partitions_str), 0);
}
- rd_kafka_dbg(
- rk, CGRP, "HEARTBEAT",
- "ConsumerGroupHeartbeat response received target "
- "assignment \"%s\"",
- assigned_topic_partitions_str);
+ rd_kafka_dbg(rk, CGRP, "HEARTBEAT",
+ "ConsumerGroupHeartbeat received target "
+ "assignment \"%s\"",
+ assigned_topic_partitions_str);
}
if (assigned_topic_partitions) {
@@ -3026,11 +3221,16 @@ void rd_kafka_cgrp_handle_ConsumerGroupHeartbeat(rd_kafka_t *rk,
}
}
+ if (rd_kafka_cgrp_consumer_subscription_preconditions_met(rkcg))
+ rd_kafka_cgrp_consumer_expedite_next_heartbeat(
+ rkcg, "send new subscription");
+
rkcg->rkcg_flags &= ~RD_KAFKA_CGRP_F_HEARTBEAT_IN_TRANSIT;
rkcg->rkcg_consumer_flags &=
~RD_KAFKA_CGRP_CONSUMER_F_SENDING_NEW_SUBSCRIPTION &
~RD_KAFKA_CGRP_CONSUMER_F_SEND_FULL_REQUEST &
~RD_KAFKA_CGRP_CONSUMER_F_SENDING_ACK;
+ rd_kafka_cgrp_maybe_clear_heartbeat_failed_err(rkcg);
rkcg->rkcg_last_heartbeat_err = RD_KAFKA_RESP_ERR_NO_ERROR;
rkcg->rkcg_expedite_heartbeat_retries = 0;
@@ -3098,8 +3298,13 @@ void rd_kafka_cgrp_handle_ConsumerGroupHeartbeat(rd_kafka_t *rk,
actions = RD_KAFKA_ERR_ACTION_FATAL;
break;
default:
- actions = rd_kafka_err_action(rkb, err, request,
- RD_KAFKA_ERR_ACTION_END);
+ actions = rd_kafka_err_action(
+ rkb, err, request,
+
+ RD_KAFKA_ERR_ACTION_SPECIAL,
+ RD_KAFKA_RESP_ERR_TOPIC_AUTHORIZATION_FAILED,
+
+ RD_KAFKA_ERR_ACTION_END);
break;
}
@@ -3127,12 +3332,31 @@ void rd_kafka_cgrp_handle_ConsumerGroupHeartbeat(rd_kafka_t *rk,
rkcg->rkcg_consumer_flags |=
RD_KAFKA_CGRP_CONSUMER_F_SEND_FULL_REQUEST;
rd_kafka_cgrp_coord_query(rkcg, rd_kafka_err2str(err));
- rd_kafka_cgrp_consumer_expedite_next_heartbeat(
- rkcg, "coordinator query");
+ /* If coordinator changes, HB will be expedited. */
+ }
+
+ if (actions & RD_KAFKA_ERR_ACTION_SPECIAL) {
+ rd_ts_t min_error_interval =
+ RD_MAX(rkcg->rkcg_heartbeat_intvl_ms * 1000,
+ /* default group.consumer.heartbeat.interval.ms */
+ 5000000);
+ if (rkcg->rkcg_last_err != err ||
+ (rd_clock() >
+ rkcg->rkcg_ts_last_err + min_error_interval)) {
+ rd_kafka_cgrp_set_last_err(rkcg, err);
+ rd_kafka_consumer_err(
+ rkcg->rkcg_q, rd_kafka_broker_id(rkb), err, 0, NULL,
+ NULL, err,
+ "ConsumerGroupHeartbeat failed: %s%s%.*s",
+ rd_kafka_err2str(err),
+ RD_KAFKAP_STR_LEN(&error_str) ? ": " : "",
+ RD_KAFKAP_STR_PR(&error_str));
+ }
}
if (actions & RD_KAFKA_ERR_ACTION_RETRY &&
rkcg->rkcg_flags & RD_KAFKA_CGRP_F_SUBSCRIPTION &&
+ !rd_kafka_cgrp_will_leave(rkcg) &&
rd_kafka_buf_retry(rkb, request)) {
/* Retry */
rkcg->rkcg_flags |= RD_KAFKA_CGRP_F_HEARTBEAT_IN_TRANSIT;
@@ -3319,13 +3543,15 @@ static void rd_kafka_cgrp_terminated(rd_kafka_cgrp_t *rkcg) {
rd_kafka_q_purge(rkcg->rkcg_wait_coord_q);
- /* Disable and empty ops queue since there will be no
+ /* Disable ops queue since there will be no
* (broker) thread serving it anymore after the unassign_broker
* below.
- * This prevents hang on destroy where responses are enqueued on
- * rkcg_ops without anything serving the queue. */
+ * As queue is forwarded to rk_ops, it cannot be purged,
+ * so consumer group operation need to be served with a no-op
+ * when `rkcg_terminated` is true. */
+
+ rd_atomic32_set(&rkcg->rkcg_terminated, rd_true);
rd_kafka_q_disable(rkcg->rkcg_ops);
- rd_kafka_q_purge(rkcg->rkcg_ops);
if (rkcg->rkcg_curr_coord)
rd_kafka_cgrp_coord_clear_broker(rkcg);
@@ -3335,8 +3561,6 @@ static void rd_kafka_cgrp_terminated(rd_kafka_cgrp_t *rkcg) {
rkcg->rkcg_coord = NULL;
}
- rd_atomic32_set(&rkcg->rkcg_terminated, rd_true);
-
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "CGRPTERM",
"Consumer group sub-system terminated%s",
rkcg->rkcg_reply_rko ? " (will enqueue reply)" : "");
@@ -3462,10 +3686,6 @@ static void rd_kafka_cgrp_partition_add(rd_kafka_cgrp_t *rkcg,
*/
static void rd_kafka_cgrp_partition_del(rd_kafka_cgrp_t *rkcg,
rd_kafka_toppar_t *rktp) {
- int cnt = 0, barrier_cnt = 0, message_cnt = 0, other_cnt = 0;
- rd_kafka_op_t *rko;
- rd_kafka_q_t *rkq;
-
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "PARTDEL",
"Group \"%s\": delete %s [%" PRId32 "]",
rkcg->rkcg_group_id->str, rktp->rktp_rkt->rkt_topic->str,
@@ -3475,54 +3695,7 @@ static void rd_kafka_cgrp_partition_del(rd_kafka_cgrp_t *rkcg,
rd_assert(rktp->rktp_flags & RD_KAFKA_TOPPAR_F_ON_CGRP);
rktp->rktp_flags &= ~RD_KAFKA_TOPPAR_F_ON_CGRP;
- if (rktp->rktp_flags & RD_KAFKA_TOPPAR_F_REMOVE) {
- /* Partition is being removed from the cluster and it's stopped,
- * so rktp->rktp_fetchq->rkq_fwdq is NULL.
- * Purge remaining operations in rktp->rktp_fetchq->rkq_q,
- * while holding lock, to avoid circular references */
- rkq = rktp->rktp_fetchq;
- mtx_lock(&rkq->rkq_lock);
- rd_assert(!rkq->rkq_fwdq);
-
- rko = TAILQ_FIRST(&rkq->rkq_q);
- while (rko) {
- if (rko->rko_type != RD_KAFKA_OP_BARRIER &&
- rko->rko_type != RD_KAFKA_OP_FETCH) {
- rd_kafka_log(
- rkcg->rkcg_rk, LOG_WARNING, "PARTDEL",
- "Purging toppar fetch queue buffer op"
- "with unexpected type: %s",
- rd_kafka_op2str(rko->rko_type));
- }
-
- if (rko->rko_type == RD_KAFKA_OP_BARRIER)
- barrier_cnt++;
- else if (rko->rko_type == RD_KAFKA_OP_FETCH)
- message_cnt++;
- else
- other_cnt++;
-
- rko = TAILQ_NEXT(rko, rko_link);
- cnt++;
- }
-
- mtx_unlock(&rkq->rkq_lock);
-
- if (cnt) {
- rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "PARTDEL",
- "Purge toppar fetch queue buffer "
- "containing %d op(s) "
- "(%d barrier(s), %d message(s), %d other)"
- " to avoid "
- "circular references",
- cnt, barrier_cnt, message_cnt, other_cnt);
- rd_kafka_q_purge(rktp->rktp_fetchq);
- } else {
- rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "PARTDEL",
- "Not purging toppar fetch queue buffer."
- " No ops present in the buffer.");
- }
- }
+ rd_kafka_toppar_purge_internal_fetch_queue_maybe(rktp);
rd_kafka_toppar_unlock(rktp);
@@ -3832,7 +4005,12 @@ static void rd_kafka_cgrp_op_handle_OffsetCommit(rd_kafka_t *rk,
break;
case RD_KAFKA_RESP_ERR_ILLEGAL_GENERATION:
- /* Revoke assignment and rebalance on illegal generation */
+ /* Revoke assignment and rebalance on illegal generation,
+ * only if not rebalancing, because a new generation id
+ * can be received soon after this error. */
+ if (RD_KAFKA_CGRP_REBALANCING(rkcg))
+ break;
+
rk->rk_cgrp->rkcg_generation_id = -1;
rd_kafka_cgrp_revoke_all_rejoin_maybe(
rkcg, rd_true /*assignment is lost*/,
@@ -5175,21 +5353,60 @@ rd_kafka_cgrp_calculate_subscribe_revoking_partitions(
return revoking;
}
-static void
+/**
+ * @brief Set the new subscription and increase subscription version.
+ *
+ * @return The new subscription version.
+ */
+static int32_t
rd_kafka_cgrp_subscription_set(rd_kafka_cgrp_t *rkcg,
rd_kafka_topic_partition_list_t *rktparlist) {
+
+ rkcg->rkcg_flags &= ~(RD_KAFKA_CGRP_F_SUBSCRIPTION |
+ RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION);
+ RD_IF_FREE(rkcg->rkcg_subscription,
+ rd_kafka_topic_partition_list_destroy);
+ RD_IF_FREE(rkcg->rkcg_subscription_topics,
+ rd_kafka_topic_partition_list_destroy);
+ RD_IF_FREE(rkcg->rkcg_subscription_regex, rd_kafkap_str_destroy);
+
rkcg->rkcg_subscription = rktparlist;
+
if (rkcg->rkcg_subscription) {
- /* Insert all non-wildcard topics in cache immediately.
- * Otherwise a manual full metadata request could
- * not cache the hinted topic and return an
- * UNKNOWN_TOPIC_OR_PART error to the user. See #4589. */
- rd_kafka_metadata_cache_hint_rktparlist(
- rkcg->rkcg_rk, rkcg->rkcg_subscription, NULL,
- 0 /*dont replace*/);
+ rkcg->rkcg_flags |= RD_KAFKA_CGRP_F_SUBSCRIPTION;
+ if (rd_kafka_topic_partition_list_regex_cnt(
+ rkcg->rkcg_subscription) > 0)
+ rkcg->rkcg_flags |=
+ RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION;
+
+ if (rkcg->rkcg_group_protocol ==
+ RD_KAFKA_GROUP_PROTOCOL_CONSUMER) {
+ rkcg->rkcg_subscription_regex =
+ rd_kafka_topic_partition_list_combine_regexes(
+ rkcg->rkcg_subscription);
+ rkcg->rkcg_subscription_topics =
+ rd_kafka_topic_partition_list_remove_regexes(
+ rkcg->rkcg_subscription);
+ rkcg->rkcg_consumer_flags |=
+ RD_KAFKA_CGRP_CONSUMER_F_SUBSCRIBED_ONCE |
+ RD_KAFKA_CGRP_CONSUMER_F_SEND_NEW_SUBSCRIPTION;
+ rd_kafka_cgrp_maybe_clear_heartbeat_failed_err(rkcg);
+ }
+ } else {
+ rkcg->rkcg_subscription_regex = NULL;
+ rkcg->rkcg_subscription_topics = NULL;
+ if (rkcg->rkcg_next_subscription) {
+ /* When unsubscribing clear next subscription too */
+ rd_kafka_topic_partition_list_destroy(
+ rkcg->rkcg_next_subscription);
+ rkcg->rkcg_next_subscription = NULL;
+ }
}
+
+ return rd_atomic32_add(&rkcg->rkcg_subscription_version, 1);
}
+
/**
* @brief Handle a new subscription that is modifying an existing subscription
* in the COOPERATIVE case.
@@ -5205,11 +5422,7 @@ rd_kafka_cgrp_modify_subscription(rd_kafka_cgrp_t *rkcg,
rd_kafka_topic_partition_list_t *errored;
int metadata_age;
int old_cnt = rkcg->rkcg_subscription->cnt;
-
- rkcg->rkcg_flags &= ~RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION;
-
- if (rd_kafka_topic_partition_list_regex_cnt(rktparlist) > 0)
- rkcg->rkcg_flags |= RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION;
+ int32_t cgrp_subscription_version;
/* Topics in rkcg_subscribed_topics that don't match any pattern in
the new subscription. */
@@ -5220,10 +5433,11 @@ rd_kafka_cgrp_modify_subscription(rd_kafka_cgrp_t *rkcg,
revoking = rd_kafka_cgrp_calculate_subscribe_revoking_partitions(
rkcg, unsubscribing_topics);
- rd_kafka_topic_partition_list_destroy(rkcg->rkcg_subscription);
- rd_kafka_cgrp_subscription_set(rkcg, rktparlist);
+ cgrp_subscription_version =
+ rd_kafka_cgrp_subscription_set(rkcg, rktparlist);
if (rd_kafka_cgrp_metadata_refresh(rkcg, &metadata_age,
+ cgrp_subscription_version,
"modify subscription") == 1) {
rd_kafka_dbg(rkcg->rkcg_rk, CGRP | RD_KAFKA_DBG_CONSUMER,
"MODSUB",
@@ -5266,7 +5480,7 @@ rd_kafka_cgrp_modify_subscription(rd_kafka_cgrp_t *rkcg,
/* Create a list of the topics in metadata that matches the new
* subscription */
tinfos = rd_list_new(rkcg->rkcg_subscription->cnt,
- (void *)rd_kafka_topic_info_destroy);
+ rd_kafka_topic_info_destroy_free);
/* Unmatched topics will be added to the errored list. */
errored = rd_kafka_topic_partition_list_new(0);
@@ -5306,12 +5520,55 @@ rd_kafka_cgrp_modify_subscription(rd_kafka_cgrp_t *rkcg,
return RD_KAFKA_RESP_ERR_NO_ERROR;
}
+/**
+ * Remove existing topic subscription (KIP 848).
+ */
+static rd_kafka_resp_err_t
+rd_kafka_cgrp_consumer_unsubscribe(rd_kafka_cgrp_t *rkcg,
+ rd_bool_t leave_group) {
+
+ rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "UNSUBSCRIBE",
+ "Group \"%.*s\": unsubscribe from current %ssubscription "
+ "of size %d (leave group=%s, has joined=%s, %s, "
+ "join-state %s)",
+ RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
+ rkcg->rkcg_subscription ? "" : "unset ",
+ rkcg->rkcg_subscription ? rkcg->rkcg_subscription->cnt : 0,
+ RD_STR_ToF(leave_group),
+ RD_STR_ToF(RD_KAFKA_CGRP_HAS_JOINED(rkcg)),
+ rkcg->rkcg_member_id ? rkcg->rkcg_member_id->str : "n/a",
+ rd_kafka_cgrp_join_state_names[rkcg->rkcg_join_state]);
+
+ rd_kafka_timer_stop(&rkcg->rkcg_rk->rk_timers,
+ &rkcg->rkcg_max_poll_interval_tmr, 1 /*lock*/);
+
+ rd_kafka_cgrp_subscription_set(rkcg, NULL);
+
+ /* When group is rejoining the leave group call is either:
+ * - been done on max.poll.interval reached
+ * - not necessary because member has been fenced
+ *
+ * When group is already leaving we just wait that previous
+ * leave request finishes.
+ */
+ if (leave_group && !rd_kafka_cgrp_consumer_will_rejoin(rkcg) &&
+ RD_KAFKA_CGRP_HAS_JOINED(rkcg) && !rd_kafka_cgrp_will_leave(rkcg)) {
+ rkcg->rkcg_flags |= RD_KAFKA_CGRP_F_LEAVE_ON_UNASSIGN_DONE;
+ rd_kafka_cgrp_revoke_all_rejoin(rkcg, rd_false /*not lost*/,
+ rd_true /*initiating*/,
+ "unsubscribe");
+ }
+
+ return RD_KAFKA_RESP_ERR_NO_ERROR;
+}
/**
* Remove existing topic subscription.
*/
static rd_kafka_resp_err_t rd_kafka_cgrp_unsubscribe(rd_kafka_cgrp_t *rkcg,
rd_bool_t leave_group) {
+ if (rkcg->rkcg_group_protocol == RD_KAFKA_GROUP_PROTOCOL_CONSUMER)
+ return rd_kafka_cgrp_consumer_unsubscribe(rkcg, leave_group);
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "UNSUBSCRIBE",
"Group \"%.*s\": unsubscribe from current %ssubscription "
@@ -5328,13 +5585,9 @@ static rd_kafka_resp_err_t rd_kafka_cgrp_unsubscribe(rd_kafka_cgrp_t *rkcg,
rd_kafka_timer_stop(&rkcg->rkcg_rk->rk_timers,
&rkcg->rkcg_max_poll_interval_tmr, 1 /*lock*/);
- if (rkcg->rkcg_subscription) {
- rd_kafka_topic_partition_list_destroy(rkcg->rkcg_subscription);
- rd_kafka_cgrp_subscription_set(rkcg, NULL);
- }
+ rd_kafka_cgrp_subscription_set(rkcg, NULL);
- if (rkcg->rkcg_group_protocol == RD_KAFKA_GROUP_PROTOCOL_CLASSIC)
- rd_kafka_cgrp_update_subscribed_topics(rkcg, NULL);
+ rd_kafka_cgrp_update_subscribed_topics(rkcg, NULL);
/*
* Clean-up group leader duties, if any.
@@ -5350,9 +5603,6 @@ static rd_kafka_resp_err_t rd_kafka_cgrp_unsubscribe(rd_kafka_cgrp_t *rkcg,
rd_true /*initiating*/,
"unsubscribe");
- rkcg->rkcg_flags &= ~(RD_KAFKA_CGRP_F_SUBSCRIPTION |
- RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION);
-
return RD_KAFKA_RESP_ERR_NO_ERROR;
}
@@ -5362,7 +5612,7 @@ static rd_kafka_resp_err_t rd_kafka_cgrp_unsubscribe(rd_kafka_cgrp_t *rkcg,
static rd_kafka_resp_err_t
rd_kafka_cgrp_subscribe(rd_kafka_cgrp_t *rkcg,
rd_kafka_topic_partition_list_t *rktparlist) {
-
+ int32_t subscription_version;
rd_kafka_dbg(rkcg->rkcg_rk, CGRP | RD_KAFKA_DBG_CONSUMER, "SUBSCRIBE",
"Group \"%.*s\": subscribe to new %ssubscription "
"of %d topics (join-state %s)",
@@ -5423,14 +5673,9 @@ rd_kafka_cgrp_subscribe(rd_kafka_cgrp_t *rkcg,
if (!rktparlist)
return RD_KAFKA_RESP_ERR_NO_ERROR;
- rkcg->rkcg_flags |= RD_KAFKA_CGRP_F_SUBSCRIPTION;
-
- if (rd_kafka_topic_partition_list_regex_cnt(rktparlist) > 0)
- rkcg->rkcg_flags |= RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION;
+ subscription_version = rd_kafka_cgrp_subscription_set(rkcg, rktparlist);
- rd_kafka_cgrp_subscription_set(rkcg, rktparlist);
-
- rd_kafka_cgrp_join(rkcg);
+ rd_kafka_cgrp_join(rkcg, subscription_version);
return RD_KAFKA_RESP_ERR_NO_ERROR;
}
@@ -5479,6 +5724,11 @@ void rd_kafka_cgrp_terminate0(rd_kafka_cgrp_t *rkcg, rd_kafka_op_t *rko) {
/* Mark for stopping, the actual state transition
* is performed when all toppars have left. */
rkcg->rkcg_flags |= RD_KAFKA_CGRP_F_TERMINATE;
+ if (rkcg->rkcg_group_protocol == RD_KAFKA_GROUP_PROTOCOL_CONSUMER) {
+ rkcg->rkcg_consumer_flags &=
+ ~RD_KAFKA_CGRP_CONSUMER_F_WAIT_REJOIN &
+ ~RD_KAFKA_CGRP_CONSUMER_F_WAIT_REJOIN_TO_COMPLETE;
+ }
rkcg->rkcg_ts_terminate = rd_clock();
rkcg->rkcg_reply_rko = rko;
@@ -5788,7 +6038,8 @@ static void rd_kafka_cgrp_join_state_serve(rd_kafka_cgrp_t *rkcg) {
if (rd_interval_immediate(&rkcg->rkcg_join_intvl, 1000 * 1000,
now) > 0)
- rd_kafka_cgrp_join(rkcg);
+ rd_kafka_cgrp_join(
+ rkcg, -1 /* current subscription version */);
break;
case RD_KAFKA_CGRP_JOIN_STATE_WAIT_JOIN:
@@ -5813,16 +6064,18 @@ static void rd_kafka_cgrp_join_state_serve(rd_kafka_cgrp_t *rkcg) {
}
}
+
void rd_kafka_cgrp_consumer_group_heartbeat(rd_kafka_cgrp_t *rkcg,
rd_bool_t full_request,
rd_bool_t send_ack) {
- rd_kafkap_str_t *rkcg_group_instance_id = NULL;
- rd_kafkap_str_t *rkcg_client_rack = NULL;
- int max_poll_interval_ms = -1;
- rd_kafka_topic_partition_list_t *rkcg_subscription = NULL;
- rd_kafkap_str_t *rkcg_group_remote_assignor = NULL;
- rd_kafka_topic_partition_list_t *rkcg_group_assignment = NULL;
+ rd_kafkap_str_t *rkcg_group_instance_id = NULL;
+ rd_kafkap_str_t *rkcg_client_rack = NULL;
+ int max_poll_interval_ms = -1;
+ rd_kafka_topic_partition_list_t *rkcg_subscription_topics = NULL;
+ rd_kafkap_str_t *rkcg_subscription_regex = NULL;
+ rd_kafkap_str_t *rkcg_group_remote_assignor = NULL;
+ rd_kafka_topic_partition_list_t *rkcg_group_assignment = NULL;
int32_t member_epoch = rkcg->rkcg_generation_id;
if (member_epoch < 0)
member_epoch = 0;
@@ -5836,7 +6089,8 @@ void rd_kafka_cgrp_consumer_group_heartbeat(rd_kafka_cgrp_t *rkcg,
rkcg_client_rack = rkcg->rkcg_client_rack;
max_poll_interval_ms =
rkcg->rkcg_rk->rk_conf.max_poll_interval_ms;
- rkcg_subscription = rkcg->rkcg_subscription;
+ rkcg_subscription_topics = rkcg->rkcg_subscription_topics;
+ rkcg_subscription_regex = rkcg->rkcg_subscription_regex;
rkcg_group_remote_assignor = rkcg->rkcg_group_remote_assignor;
}
@@ -5863,23 +6117,22 @@ void rd_kafka_cgrp_consumer_group_heartbeat(rd_kafka_cgrp_t *rkcg,
rkcg_group_assignment = rkcg->rkcg_current_assignment;
}
- if (rkcg->rkcg_join_state == RD_KAFKA_CGRP_JOIN_STATE_STEADY &&
- (rkcg->rkcg_consumer_flags &
- RD_KAFKA_CGRP_CONSUMER_F_SEND_NEW_SUBSCRIPTION ||
- rkcg->rkcg_consumer_flags &
- RD_KAFKA_CGRP_CONSUMER_F_SENDING_NEW_SUBSCRIPTION)) {
+ if (rd_kafka_cgrp_consumer_subscription_preconditions_met(rkcg) ||
+ rkcg->rkcg_consumer_flags &
+ RD_KAFKA_CGRP_CONSUMER_F_SENDING_NEW_SUBSCRIPTION) {
rkcg->rkcg_consumer_flags =
(rkcg->rkcg_consumer_flags &
~RD_KAFKA_CGRP_CONSUMER_F_SEND_NEW_SUBSCRIPTION) |
RD_KAFKA_CGRP_CONSUMER_F_SENDING_NEW_SUBSCRIPTION;
- rkcg_subscription = rkcg->rkcg_subscription;
+ rkcg_subscription_topics = rkcg->rkcg_subscription_topics;
+ rkcg_subscription_regex = rkcg->rkcg_subscription_regex;
if (rd_kafka_is_dbg(rkcg->rkcg_rk, CGRP)) {
char rkcg_new_subscription_str[512] = "NULL";
- if (rkcg_subscription) {
+ if (rkcg->rkcg_subscription) {
rd_kafka_topic_partition_list_str(
- rkcg_subscription,
+ rkcg->rkcg_subscription,
rkcg_new_subscription_str,
sizeof(rkcg_new_subscription_str), 0);
}
@@ -5894,13 +6147,18 @@ void rd_kafka_cgrp_consumer_group_heartbeat(rd_kafka_cgrp_t *rkcg,
rd_kafka_ConsumerGroupHeartbeatRequest(
rkcg->rkcg_coord, rkcg->rkcg_group_id, rkcg->rkcg_member_id,
member_epoch, rkcg_group_instance_id, rkcg_client_rack,
- max_poll_interval_ms, rkcg_subscription, rkcg_group_remote_assignor,
+ max_poll_interval_ms, rkcg_subscription_topics,
+ rkcg_subscription_regex, rkcg_group_remote_assignor,
rkcg_group_assignment, RD_KAFKA_REPLYQ(rkcg->rkcg_ops, 0),
rd_kafka_cgrp_handle_ConsumerGroupHeartbeat, NULL);
}
static rd_bool_t
rd_kafka_cgrp_consumer_heartbeat_preconditions_met(rd_kafka_cgrp_t *rkcg) {
+ rd_dassert(
+ !(rkcg->rkcg_join_state == RD_KAFKA_CGRP_JOIN_STATE_INIT &&
+ rkcg->rkcg_flags & RD_KAFKA_CGRP_F_LEAVE_ON_UNASSIGN_DONE));
+
if (!(rkcg->rkcg_flags & RD_KAFKA_CGRP_F_SUBSCRIPTION))
return rd_false;
@@ -5915,6 +6173,9 @@ rd_kafka_cgrp_consumer_heartbeat_preconditions_met(rd_kafka_cgrp_t *rkcg) {
rd_kafka_max_poll_exceeded(rkcg->rkcg_rk))
return rd_false;
+ if (rd_kafka_cgrp_will_leave(rkcg))
+ return rd_false;
+
return rd_true;
}
@@ -5948,6 +6209,7 @@ void rd_kafka_cgrp_consumer_serve(rd_kafka_cgrp_t *rkcg) {
case RD_KAFKA_CGRP_JOIN_STATE_INIT:
rkcg->rkcg_consumer_flags &=
~RD_KAFKA_CGRP_CONSUMER_F_WAIT_REJOIN_TO_COMPLETE;
+ rd_kafka_cgrp_consumer_apply_next_subscribe(rkcg);
full_request = rd_true;
break;
case RD_KAFKA_CGRP_JOIN_STATE_STEADY:
@@ -6023,27 +6285,20 @@ rd_kafka_cgrp_consumer_subscribe(rd_kafka_cgrp_t *rkcg,
return RD_KAFKA_RESP_ERR__FATAL;
}
- rkcg->rkcg_flags &= ~RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION;
if (rktparlist) {
- if (rkcg->rkcg_subscription)
+ if (rkcg->rkcg_next_subscription)
rd_kafka_topic_partition_list_destroy(
- rkcg->rkcg_subscription);
-
- rkcg->rkcg_flags |= RD_KAFKA_CGRP_F_SUBSCRIPTION;
-
- if (rd_kafka_topic_partition_list_regex_cnt(rktparlist) > 0)
- rkcg->rkcg_flags |=
- RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION;
-
- rkcg->rkcg_consumer_flags |=
- RD_KAFKA_CGRP_CONSUMER_F_SUBSCRIBED_ONCE |
- RD_KAFKA_CGRP_CONSUMER_F_SEND_NEW_SUBSCRIPTION;
-
- rd_kafka_cgrp_subscription_set(rkcg, rktparlist);
- rd_kafka_cgrp_consumer_expedite_next_heartbeat(
- rkcg, "subscription changed");
+ rkcg->rkcg_next_subscription);
+ rkcg->rkcg_next_subscription = rktparlist;
+
+ /* If member is leaving, new subscription
+ * will be applied after the leave
+ * ConsumerGroupHeartbeat */
+ if (!rd_kafka_cgrp_will_leave(rkcg))
+ rd_kafka_cgrp_consumer_apply_next_subscribe(rkcg);
} else {
- rd_kafka_cgrp_unsubscribe(rkcg, rd_true /*leave group*/);
+ rd_kafka_cgrp_consumer_unsubscribe(rkcg,
+ rd_true /*leave group*/);
}
return RD_KAFKA_RESP_ERR_NO_ERROR;
@@ -6063,6 +6318,9 @@ static void rd_kafka_cgrp_consumer_incr_unassign_done(rd_kafka_cgrp_t *rkcg) {
"unassign",
rkcg->rkcg_group_id->str);
rd_kafka_cgrp_unassign(rkcg);
+
+ /* Leave group, if desired. */
+ rd_kafka_cgrp_leave_maybe(rkcg);
return;
}
@@ -6142,24 +6400,10 @@ static void rd_kafka_cgrp_consumer_assignment_done(rd_kafka_cgrp_t *rkcg) {
rd_kafka_cgrp_consumer_incr_unassign_done(rkcg);
break;
- case RD_KAFKA_CGRP_JOIN_STATE_STEADY:
- rd_kafka_cgrp_consumer_expedite_next_heartbeat(
- rkcg, "back to steady state");
-
- if (rkcg->rkcg_rebalance_rejoin) {
- rkcg->rkcg_rebalance_rejoin = rd_false;
- rd_kafka_cgrp_rejoin(
- rkcg,
- "rejoining group to redistribute "
- "previously owned partitions to other "
- "group members");
- break;
- }
-
- /* FALLTHRU */
+ case RD_KAFKA_CGRP_JOIN_STATE_INIT:
+ case RD_KAFKA_CGRP_JOIN_STATE_STEADY: {
- case RD_KAFKA_CGRP_JOIN_STATE_INIT: {
- rd_bool_t still_in_group = rd_true;
+ rd_bool_t not_in_group = rd_false;
/*
* There maybe a case when there are no assignments are
* assigned to this consumer. In this case, while terminating
@@ -6167,18 +6411,38 @@ static void rd_kafka_cgrp_consumer_assignment_done(rd_kafka_cgrp_t *rkcg) {
* to intermediate state. In this scenario, last leave call is
* done from here.
*/
- still_in_group &= !rd_kafka_cgrp_leave_maybe(rkcg);
+ not_in_group |= rd_kafka_cgrp_leave_maybe(rkcg);
/* Check if cgrp is trying to terminate, which is safe to do
* in these two states. Otherwise we'll need to wait for
* the current state to decommission. */
- still_in_group &= !rd_kafka_cgrp_try_terminate(rkcg);
+ not_in_group |= rd_kafka_cgrp_try_terminate(rkcg);
- if (still_in_group)
+ if (not_in_group)
+ break;
+
+ if (rkcg->rkcg_join_state == RD_KAFKA_CGRP_JOIN_STATE_INIT) {
+ rd_kafka_cgrp_consumer_expedite_next_heartbeat(
+ rkcg, "Assignment Done: in init state");
+ } else if (rkcg->rkcg_rebalance_rejoin) {
+ /* No need to expedite the HB here as it's being
+ * expedited in the rejoin call.*/
+ rkcg->rkcg_rebalance_rejoin = rd_false;
+ rd_kafka_cgrp_rejoin(
+ rkcg,
+ "Assignment Done: rejoining group to redistribute "
+ "previously owned partitions to other "
+ "group members");
+ } else if (rkcg->rkcg_consumer_flags &
+ RD_KAFKA_CGRP_CONSUMER_F_WAIT_ACK) {
rd_kafka_cgrp_consumer_expedite_next_heartbeat(
- rkcg, "back to init state");
+ rkcg,
+ "Assignment Done: in steady state, waiting for "
+ "ack");
+ }
break;
}
+
default:
break;
}
@@ -6397,6 +6661,12 @@ static rd_kafka_op_res_t rd_kafka_cgrp_op_serve(rd_kafka_t *rk,
rd_kafka_toppar_t *rktp;
rd_kafka_resp_err_t err;
const int silent_op = rko->rko_type == RD_KAFKA_OP_RECV_BUF;
+ if (unlikely(rd_atomic32_get(&rkcg->rkcg_terminated) == rd_true)) {
+ if (rko)
+ rd_kafka_op_destroy(rko);
+ return RD_KAFKA_OP_RES_HANDLED;
+ }
+
rktp = rko->rko_rktp;
@@ -6497,7 +6767,9 @@ static rd_kafka_op_res_t rd_kafka_cgrp_op_serve(rd_kafka_t *rk,
break;
case RD_KAFKA_OP_SUBSCRIBE:
- rd_kafka_app_polled(rk);
+ /* We just want to avoid reaching max poll interval,
+ * without anything else is done on poll. */
+ rd_atomic64_set(&rk->rk_ts_last_poll, rd_clock());
/* New atomic subscription (may be NULL) */
if (rkcg->rkcg_group_protocol ==
@@ -6652,7 +6924,7 @@ void rd_kafka_cgrp_metadata_update_check(rd_kafka_cgrp_t *rkcg,
* Create a list of the topics in metadata that matches our subscription
*/
tinfos = rd_list_new(rkcg->rkcg_subscription->cnt,
- (void *)rd_kafka_topic_info_destroy);
+ rd_kafka_topic_info_destroy_free);
if (rkcg->rkcg_flags & RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION)
rd_kafka_metadata_topic_match(rkcg->rkcg_rk, tinfos,
@@ -6791,11 +7063,27 @@ rd_kafka_consumer_group_metadata(rd_kafka_t *rk) {
return cgmetadata;
}
+const char *rd_kafka_consumer_group_metadata_group_id(
+ const rd_kafka_consumer_group_metadata_t *group_metadata) {
+ return group_metadata->group_id;
+}
+
const char *rd_kafka_consumer_group_metadata_member_id(
const rd_kafka_consumer_group_metadata_t *group_metadata) {
return group_metadata->member_id;
}
+const char *rd_kafka_consumer_group_metadata_group_instance_id(
+ const rd_kafka_consumer_group_metadata_t *group_metadata) {
+ return group_metadata->group_instance_id;
+}
+
+int32_t rd_kafka_consumer_group_metadata_generation_id(
+ const rd_kafka_consumer_group_metadata_t *group_metadata) {
+ return group_metadata->generation_id;
+}
+
+
void rd_kafka_consumer_group_metadata_destroy(
rd_kafka_consumer_group_metadata_t *cgmetadata) {
rd_free(cgmetadata->group_id);
@@ -7213,7 +7501,7 @@ static int unittest_list_to_map(void) {
}
int unittest_member_metadata_serdes(void) {
- rd_list_t *topics = rd_list_new(0, (void *)rd_kafka_topic_info_destroy);
+ rd_list_t *topics = rd_list_new(0, rd_kafka_topic_info_destroy_free);
rd_kafka_topic_partition_list_t *owned_partitions =
rd_kafka_topic_partition_list_new(0);
rd_kafkap_str_t *rack_id = rd_kafkap_str_new("myrack", -1);
diff --git a/src/rdkafka_cgrp.h b/src/rdkafka_cgrp.h
index afb671f02a..79a734f5fb 100644
--- a/src/rdkafka_cgrp.h
+++ b/src/rdkafka_cgrp.h
@@ -230,9 +230,33 @@ typedef struct rd_kafka_cgrp_s {
rd_kafka_topic_partition_list_t *rkcg_errored_topics;
/** If a SUBSCRIBE op is received during a COOPERATIVE rebalance,
* actioning this will be postponed until after the rebalance
- * completes. The waiting subscription is stored here.
- * Mutually exclusive with rkcg_next_subscription. */
+ * completes. The waiting subscription is stored here. */
rd_kafka_topic_partition_list_t *rkcg_next_subscription;
+
+ /**
+ * Subscription regex pattern. All the provided regex patterns are
+ * stored as a single string with each pattern separated by '|'.
+ *
+ * Only applicable for the consumer protocol introduced in KIP-848.
+ *
+ * rkcg_subscription = rkcg_subscription_topics +
+ * rkcg_subscription_regex
+ */
+ rd_kafkap_str_t *rkcg_subscription_regex;
+
+ /**
+ * Full topic names extracted out from the rkcg_subscription.
+ *
+ * Only applicable for the consumer protocol introduced in KIP-848.
+ *
+ * For the consumer protocol, this field doesn't include regex
+ * subscriptions. For that please refer `rkcg_subscription_regex`
+ *
+ * rkcg_subscription = rkcg_subscription_topics +
+ * rkcg_subscription_regex
+ */
+ rd_kafka_topic_partition_list_t *rkcg_subscription_topics;
+
/** If a (un)SUBSCRIBE op is received during a COOPERATIVE rebalance,
* actioning this will be posponed until after the rebalance
* completes. This flag is used to signal a waiting unsubscribe
@@ -312,6 +336,8 @@ typedef struct rd_kafka_cgrp_s {
* incremental unassign. */
rd_bool_t rkcg_rebalance_rejoin;
+ rd_ts_t rkcg_ts_last_err; /* Timestamp of last error
+ * propagated to application */
rd_kafka_resp_err_t rkcg_last_err; /* Last error propagated to
* application.
* This is for silencing
@@ -333,6 +359,8 @@ typedef struct rd_kafka_cgrp_s {
rd_atomic32_t rkcg_terminated; /**< Consumer has been closed */
+ rd_atomic32_t rkcg_subscription_version; /**< Subscription version */
+
/* Protected by rd_kafka_*lock() */
struct {
rd_ts_t ts_rebalance; /* Timestamp of
@@ -346,6 +374,9 @@ typedef struct rd_kafka_cgrp_s {
* assignment */
} rkcg_c;
+ /* Timestamp of last rebalance start */
+ rd_ts_t rkcg_ts_rebalance_start;
+
} rd_kafka_cgrp_t;
@@ -400,6 +431,12 @@ void rd_kafka_cgrp_metadata_update_check(rd_kafka_cgrp_t *rkcg,
rd_bool_t do_join);
#define rd_kafka_cgrp_get(rk) ((rk)->rk_cgrp)
+#define rd_kafka_cgrp_same_subscription_version(rk_cgrp, \
+ cgrp_subscription_version) \
+ ((rk_cgrp) && \
+ (cgrp_subscription_version == -1 || \
+ rd_atomic32_get(&(rk_cgrp)->rkcg_subscription_version) == \
+ cgrp_subscription_version))
void rd_kafka_cgrp_assigned_offsets_commit(
rd_kafka_cgrp_t *rkcg,
diff --git a/src/rdkafka_conf.c b/src/rdkafka_conf.c
index 84262d56e4..1f8bbf106b 100644
--- a/src/rdkafka_conf.c
+++ b/src/rdkafka_conf.c
@@ -56,24 +56,29 @@
#include
#endif
+#ifdef WITH_OAUTHBEARER_OIDC
+#include
+#endif
+
struct rd_kafka_property {
rd_kafka_conf_scope_t scope;
const char *name;
- enum { _RK_C_STR,
- _RK_C_INT,
- _RK_C_DBL, /* Double */
- _RK_C_S2I, /* String to Integer mapping.
- * Supports limited canonical str->int mappings
- * using s2i[] */
- _RK_C_S2F, /* CSV String to Integer flag mapping (OR:ed) */
- _RK_C_BOOL,
- _RK_C_PTR, /* Only settable through special set functions */
- _RK_C_PATLIST, /* Pattern list */
- _RK_C_KSTR, /* Kafka string */
- _RK_C_ALIAS, /* Alias: points to other property through .sdef */
- _RK_C_INTERNAL, /* Internal, don't expose to application */
- _RK_C_INVALID, /* Invalid property, used to catch known
- * but unsupported Java properties. */
+ enum {
+ _RK_C_STR,
+ _RK_C_INT,
+ _RK_C_DBL, /* Double */
+ _RK_C_S2I, /* String to Integer mapping.
+ * Supports limited canonical str->int mappings
+ * using s2i[] */
+ _RK_C_S2F, /* CSV String to Integer flag mapping (OR:ed) */
+ _RK_C_BOOL,
+ _RK_C_PTR, /* Only settable through special set functions */
+ _RK_C_PATLIST, /* Pattern list */
+ _RK_C_KSTR, /* Kafka string */
+ _RK_C_ALIAS, /* Alias: points to other property through .sdef */
+ _RK_C_INTERNAL, /* Internal, don't expose to application */
+ _RK_C_INVALID, /* Invalid property, used to catch known
+ * but unsupported Java properties. */
} type;
int offset;
const char *desc;
@@ -198,6 +203,15 @@ struct rd_kafka_property {
"available at build time"
#endif
+#if WITH_OAUTHBEARER_OIDC
+#define _UNSUPPORTED_HTTPS .unsupported = NULL
+#else
+#define _UNSUPPORTED_HTTPS \
+ .unsupported = \
+ "HTTPS calls depend on libcurl and OpenSSL which were not " \
+ "available at build time"
+#endif
+
#ifdef _WIN32
#define _UNSUPPORTED_WIN32_GSSAPI \
.unsupported = \
@@ -437,6 +451,34 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
1, 1000000, 1000000},
{_RK_GLOBAL, "max.in.flight", _RK_C_ALIAS,
.sdef = "max.in.flight.requests.per.connection"},
+ {_RK_GLOBAL, "metadata.recovery.strategy", _RK_C_S2I,
+ _RK(metadata_recovery_strategy),
+ "Controls how the client recovers when none of the brokers known to it "
+ "is available. If set to `none`, the client doesn't re-bootstrap. "
+ "If set to `rebootstrap`, the client repeats the bootstrap process "
+ "using `bootstrap.servers` and brokers added through "
+ "`rd_kafka_brokers_add()`. Rebootstrapping is useful when a client "
+ "communicates with brokers so infrequently that the set of brokers "
+ "may change entirely before the client refreshes metadata. "
+ "Metadata recovery is triggered when all last-known brokers appear "
+ "unavailable simultaneously or the client cannot refresh metadata within "
+ "`metadata.recovery.rebootstrap.trigger.ms` or it's requested in a "
+ "metadata response.",
+ .vdef = RD_KAFKA_METADATA_RECOVERY_STRATEGY_REBOOTSTRAP,
+ .s2i = {{RD_KAFKA_METADATA_RECOVERY_STRATEGY_NONE, "none"},
+ {RD_KAFKA_METADATA_RECOVERY_STRATEGY_REBOOTSTRAP, "rebootstrap"},
+ {0, NULL}}},
+ {_RK_GLOBAL, "metadata.recovery.rebootstrap.trigger.ms", _RK_C_INT,
+ _RK(metadata_recovery_rebootstrap_trigger_ms),
+ "If a client configured to rebootstrap using "
+ "`metadata.recovery.strategy=rebootstrap` "
+ "is unable to obtain metadata from any "
+ "of the brokers for this interval, "
+ "client repeats the bootstrap process using "
+ "`bootstrap.servers` configuration "
+ "and brokers added through "
+ "`rd_kafka_brokers_add()`.",
+ 0, INT_MAX, 300000},
{_RK_GLOBAL | _RK_DEPRECATED | _RK_HIDDEN, "metadata.request.timeout.ms",
_RK_C_INT, _RK(metadata_request_timeout_ms), "Not used.", 10, 900 * 1000,
10},
@@ -540,7 +582,7 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
#endif
},
{_RK_GLOBAL, "socket.nagle.disable", _RK_C_BOOL, _RK(socket_nagle_disable),
- "Disable the Nagle algorithm (TCP_NODELAY) on broker sockets.", 0, 1, 0
+ "Disable the Nagle algorithm (TCP_NODELAY) on broker sockets.", 0, 1, 1
#ifndef TCP_NODELAY
,
.unsupported = "TCP_NODELAY not available at build time"
@@ -581,7 +623,10 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
"If this property is left at its default value some heuristics are "
"performed to determine a suitable default value, this is currently "
"limited to identifying brokers on Azure "
- "(see librdkafka issue #3109 for more info).",
+ "(see librdkafka issue #3109 for more info). "
+ "Actual value can be lower, up to 2s lower, "
+ "only if `connections.max.idle.ms` >= 4s, "
+ "as jitter is added to avoid disconnecting all brokers at the same time.",
0, INT_MAX, 0},
{_RK_GLOBAL | _RK_MED | _RK_HIDDEN, "enable.sparse.connections", _RK_C_BOOL,
_RK(sparse_connections),
@@ -702,8 +747,10 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
"The application should mask this signal as an internal "
"signal handler is installed.",
0, 128, 0},
- {_RK_GLOBAL | _RK_HIGH, "api.version.request", _RK_C_BOOL,
+ {_RK_GLOBAL | _RK_HIGH | _RK_DEPRECATED, "api.version.request", _RK_C_BOOL,
_RK(api_version_request),
+ "**Post-deprecation actions: remove this configuration property, "
+ "brokers < 0.10.0 won't be supported anymore in librdkafka 3.x.** "
"Request broker's supported API versions to adjust functionality to "
"available protocol features. If set to false, or the "
"ApiVersionRequest fails, the fallback version "
@@ -715,16 +762,20 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
{_RK_GLOBAL, "api.version.request.timeout.ms", _RK_C_INT,
_RK(api_version_request_timeout_ms),
"Timeout for broker API version requests.", 1, 5 * 60 * 1000, 10 * 1000},
- {_RK_GLOBAL | _RK_MED, "api.version.fallback.ms", _RK_C_INT,
- _RK(api_version_fallback_ms),
+ {_RK_GLOBAL | _RK_MED | _RK_DEPRECATED, "api.version.fallback.ms",
+ _RK_C_INT, _RK(api_version_fallback_ms),
+ "**Post-deprecation actions: remove this configuration property, "
+ "brokers < 0.10.0 won't be supported anymore in librdkafka 3.x.** "
"Dictates how long the `broker.version.fallback` fallback is used "
"in the case the ApiVersionRequest fails. "
"**NOTE**: The ApiVersionRequest is only issued when a new connection "
"to the broker is made (such as after an upgrade).",
0, 86400 * 7 * 1000, 0},
- {_RK_GLOBAL | _RK_MED, "broker.version.fallback", _RK_C_STR,
- _RK(broker_version_fallback),
+ {_RK_GLOBAL | _RK_MED | _RK_DEPRECATED, "broker.version.fallback",
+ _RK_C_STR, _RK(broker_version_fallback),
+ "**Post-deprecation actions: remove this configuration property, "
+ "brokers < 0.10.0 won't be supported anymore in librdkafka 3.x.** "
"Older broker versions (before 0.10.0) provide no way for a client to "
"query "
"for supported protocol features "
@@ -758,10 +809,10 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
_RK(security_protocol), "Protocol used to communicate with brokers.",
.vdef = RD_KAFKA_PROTO_PLAINTEXT,
.s2i = {{RD_KAFKA_PROTO_PLAINTEXT, "plaintext"},
- {RD_KAFKA_PROTO_SSL, "ssl", _UNSUPPORTED_SSL},
- {RD_KAFKA_PROTO_SASL_PLAINTEXT, "sasl_plaintext"},
- {RD_KAFKA_PROTO_SASL_SSL, "sasl_ssl", _UNSUPPORTED_SSL},
- {0, NULL}}},
+ {RD_KAFKA_PROTO_SSL, "ssl", _UNSUPPORTED_SSL},
+ {RD_KAFKA_PROTO_SASL_PLAINTEXT, "sasl_plaintext"},
+ {RD_KAFKA_PROTO_SASL_SSL, "sasl_ssl", _UNSUPPORTED_SSL},
+ {0, NULL}}},
{_RK_GLOBAL, "ssl.cipher.suites", _RK_C_STR, _RK(ssl.cipher_suites),
"A cipher suite is a named combination of authentication, "
@@ -825,6 +876,29 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
"If OpenSSL is dynamically linked the OpenSSL library's default "
"path will be used (see `OPENSSLDIR` in `openssl version -a`).",
_UNSUPPORTED_SSL},
+ {_RK_GLOBAL, "https.ca.location", _RK_C_STR, _RK(https.ca_location),
+ "File or directory path to CA certificate(s) for verifying "
+ "HTTPS endpoints, like `sasl.oauthbearer.token.endpoint.url` used for "
+ "OAUTHBEARER/OIDC authentication. "
+ "Mutually exclusive with `https.ca.pem`. "
+ "Defaults: "
+ "On Windows the system's CA certificates are automatically looked "
+ "up in the Windows Root certificate store. "
+ "On Mac OSX this configuration defaults to `probe`. "
+ "It is recommended to install openssl using Homebrew, "
+ "to provide CA certificates. "
+ "On Linux install the distribution's ca-certificates package. "
+ "If OpenSSL is statically linked or `https.ca.location` is set to "
+ "`probe` a list of standard paths will be probed and the first one "
+ "found will be used as the default CA certificate location path. "
+ "If OpenSSL is dynamically linked the OpenSSL library's default "
+ "path will be used (see `OPENSSLDIR` in `openssl version -a`).",
+ _UNSUPPORTED_HTTPS},
+ {_RK_GLOBAL, "https.ca.pem", _RK_C_STR, _RK(https.ca_pem),
+ "CA certificate string (PEM format) for verifying HTTPS endpoints. "
+ "Mutually exclusive with `https.ca.location`. "
+ "Optional: see `https.ca.location`.",
+ _UNSUPPORTED_HTTPS},
{_RK_GLOBAL | _RK_SENSITIVE, "ssl.ca.pem", _RK_C_STR, _RK(ssl.ca_pem),
"CA certificate string (PEM format) for verifying the broker's key.",
_UNSUPPORTED_SSL},
@@ -889,7 +963,7 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
"OpenSSL >= 1.0.2 required.",
.vdef = RD_KAFKA_SSL_ENDPOINT_ID_HTTPS,
.s2i = {{RD_KAFKA_SSL_ENDPOINT_ID_NONE, "none"},
- {RD_KAFKA_SSL_ENDPOINT_ID_HTTPS, "https"}},
+ {RD_KAFKA_SSL_ENDPOINT_ID_HTTPS, "https"}},
_UNSUPPORTED_OPENSSL_1_0_2},
{_RK_GLOBAL, "ssl.certificate.verify_cb", _RK_C_PTR,
_RK(ssl.cert_verify_cb),
@@ -1009,7 +1083,7 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
"and `sasl.oauthbearer.token.endpoint.url`.",
.vdef = RD_KAFKA_SASL_OAUTHBEARER_METHOD_DEFAULT,
.s2i = {{RD_KAFKA_SASL_OAUTHBEARER_METHOD_DEFAULT, "default"},
- {RD_KAFKA_SASL_OAUTHBEARER_METHOD_OIDC, "oidc"}},
+ {RD_KAFKA_SASL_OAUTHBEARER_METHOD_OIDC, "oidc"}},
_UNSUPPORTED_OIDC},
{_RK_GLOBAL, "sasl.oauthbearer.client.id", _RK_C_STR,
_RK(sasl.oauthbearer.client_id),
@@ -1018,7 +1092,11 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
"authorization server handles. "
"Only used when `sasl.oauthbearer.method` is set to \"oidc\".",
_UNSUPPORTED_OIDC},
- {_RK_GLOBAL, "sasl.oauthbearer.client.secret", _RK_C_STR,
+ {_RK_GLOBAL, "sasl.oauthbearer.client.credentials.client.id", _RK_C_ALIAS,
+ .sdef = "sasl.oauthbearer.client.id"},
+ {_RK_GLOBAL, "sasl.oauthbearer.client.credentials.client.secret",
+ _RK_C_ALIAS, .sdef = "sasl.oauthbearer.client.secret"},
+ {_RK_GLOBAL | _RK_SENSITIVE, "sasl.oauthbearer.client.secret", _RK_C_STR,
_RK(sasl.oauthbearer.client_secret),
"Client secret only known to the application and the "
"authorization server. This should be a sufficiently random string "
@@ -1043,6 +1121,113 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
"OAuth/OIDC issuer token endpoint HTTP(S) URI used to retrieve token. "
"Only used when `sasl.oauthbearer.method` is set to \"oidc\".",
_UNSUPPORTED_OIDC},
+ {
+ _RK_GLOBAL,
+ "sasl.oauthbearer.grant.type",
+ _RK_C_S2I,
+ _RK(sasl.oauthbearer.grant_type),
+ "OAuth grant type to use when communicating with the identity "
+ "provider.",
+ _UNSUPPORTED_OIDC,
+ .vdef = RD_KAFKA_SASL_OAUTHBEARER_GRANT_TYPE_CLIENT_CREDENTIALS,
+ .s2i = {{RD_KAFKA_SASL_OAUTHBEARER_GRANT_TYPE_CLIENT_CREDENTIALS,
+ "client_credentials"},
+ {RD_KAFKA_SASL_OAUTHBEARER_GRANT_TYPE_JWT_BEARER,
+ "urn:ietf:params:oauth:grant-type:jwt-bearer"}},
+ },
+ {_RK_GLOBAL, "sasl.oauthbearer.assertion.algorithm", _RK_C_S2I,
+ _RK(sasl.oauthbearer.assertion.algorithm),
+ "Algorithm the client should use to sign the assertion sent "
+ "to the identity provider and in the OAuth alg header in the JWT "
+ "assertion.",
+ _UNSUPPORTED_OIDC,
+ .vdef = RD_KAFKA_SASL_OAUTHBEARER_ASSERTION_ALGORITHM_RS256,
+ .s2i = {{RD_KAFKA_SASL_OAUTHBEARER_ASSERTION_ALGORITHM_RS256, "RS256"},
+ {RD_KAFKA_SASL_OAUTHBEARER_ASSERTION_ALGORITHM_ES256, "ES256"}}},
+ {_RK_GLOBAL | _RK_SENSITIVE, "sasl.oauthbearer.assertion.private.key.file",
+ _RK_C_STR, _RK(sasl.oauthbearer.assertion.private_key.file),
+ "Path to client's private key (PEM) used for authentication "
+ "when using the JWT assertion.",
+ _UNSUPPORTED_OIDC},
+ {_RK_GLOBAL | _RK_SENSITIVE,
+ "sasl.oauthbearer.assertion.private.key.passphrase", _RK_C_STR,
+ _RK(sasl.oauthbearer.assertion.private_key.passphrase),
+ "Private key passphrase for `sasl.oauthbearer.assertion.private.key.file`"
+ " or `sasl.oauthbearer.assertion.private.key.pem`.",
+ _UNSUPPORTED_OIDC},
+ {_RK_GLOBAL | _RK_SENSITIVE, "sasl.oauthbearer.assertion.private.key.pem",
+ _RK_C_STR, _RK(sasl.oauthbearer.assertion.private_key.pem),
+ "Client's private key (PEM) used for authentication "
+ "when using the JWT assertion.",
+ _UNSUPPORTED_OIDC},
+ {_RK_GLOBAL, "sasl.oauthbearer.assertion.file", _RK_C_STR,
+ _RK(sasl.oauthbearer.assertion.file),
+ "Path to the assertion file. "
+ "Only used when `sasl.oauthbearer.method` is set to \"oidc\" and JWT "
+ "assertion is needed.",
+ _UNSUPPORTED_OIDC},
+ {_RK_GLOBAL, "sasl.oauthbearer.assertion.claim.aud", _RK_C_STR,
+ _RK(sasl.oauthbearer.assertion.claim.audience),
+ "JWT audience claim. "
+ "Only used when `sasl.oauthbearer.method` is set to \"oidc\" and JWT "
+ "assertion is needed.",
+ _UNSUPPORTED_OIDC},
+ {_RK_GLOBAL, "sasl.oauthbearer.assertion.claim.exp.seconds", _RK_C_INT,
+ _RK(sasl.oauthbearer.assertion.claim.expiration_s),
+ "Assertion expiration time in seconds. "
+ "Only used when `sasl.oauthbearer.method` is set to \"oidc\" and JWT "
+ "assertion is needed.",
+ 1, INT_MAX, 300, _UNSUPPORTED_OIDC},
+ {_RK_GLOBAL, "sasl.oauthbearer.assertion.claim.iss", _RK_C_STR,
+ _RK(sasl.oauthbearer.assertion.claim.issuer),
+ "JWT issuer claim. "
+ "Only used when `sasl.oauthbearer.method` is set to \"oidc\" and JWT "
+ "assertion is needed.",
+ _UNSUPPORTED_OIDC},
+ {_RK_GLOBAL, "sasl.oauthbearer.assertion.claim.jti.include", _RK_C_BOOL,
+ _RK(sasl.oauthbearer.assertion.claim.jti_include),
+ "JWT ID claim. When set to `true`, a random UUID is generated. "
+ "Only used when `sasl.oauthbearer.method` is set to \"oidc\" and JWT "
+ "assertion is needed.",
+ 0, 1, 0, _UNSUPPORTED_OIDC},
+ {_RK_GLOBAL, "sasl.oauthbearer.assertion.claim.nbf.seconds", _RK_C_INT,
+ _RK(sasl.oauthbearer.assertion.claim.not_before_s),
+ "Assertion not before time in seconds. "
+ "Only used when `sasl.oauthbearer.method` is set to \"oidc\" and JWT "
+ "assertion is needed.",
+ 0, INT_MAX, 60, _UNSUPPORTED_OIDC},
+ {_RK_GLOBAL, "sasl.oauthbearer.assertion.claim.sub", _RK_C_STR,
+ _RK(sasl.oauthbearer.assertion.claim.subject),
+ "JWT subject claim. "
+ "Only used when `sasl.oauthbearer.method` is set to \"oidc\" and JWT "
+ "assertion is needed.",
+ _UNSUPPORTED_OIDC},
+ {_RK_GLOBAL, "sasl.oauthbearer.assertion.jwt.template.file", _RK_C_STR,
+ _RK(sasl.oauthbearer.assertion.jwt_template_file),
+ "Path to the JWT template file. "
+ "Only used when `sasl.oauthbearer.method` is set to \"oidc\" and JWT "
+ "assertion is needed.",
+ _UNSUPPORTED_OIDC},
+ {
+ _RK_GLOBAL,
+ "sasl.oauthbearer.metadata.authentication.type",
+ _RK_C_S2I,
+ _RK(sasl.oauthbearer.metadata_authentication.type),
+ "Type of metadata-based authentication to use for OAUTHBEARER/OIDC "
+ "`azure_imds` authenticates using the Azure IMDS endpoint. "
+ "Sets a default value for `sasl.oauthbearer.token.endpoint.url` if "
+ "missing. "
+ "Configuration values specific of chosen authentication type can be "
+ "passed "
+ "through `sasl.oauthbearer.config`.",
+ _UNSUPPORTED_OIDC,
+ .vdef = RD_KAFKA_SASL_OAUTHBEARER_METADATA_AUTHENTICATION_TYPE_NONE,
+ .s2i =
+ {{RD_KAFKA_SASL_OAUTHBEARER_METADATA_AUTHENTICATION_TYPE_NONE,
+ "none"},
+ {RD_KAFKA_SASL_OAUTHBEARER_METADATA_AUTHENTICATION_TYPE_AZURE_IMDS,
+ "azure_imds"}},
+ },
/* Plugins */
{_RK_GLOBAL, "plugin.library.paths", _RK_C_STR, _RK(plugin_paths),
@@ -1110,9 +1295,10 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
"members of the group to assign partitions to group members. If "
"there is more than one eligible strategy, preference is "
"determined by the order of this list (strategies earlier in the "
- "list have higher priority). "
- "Cooperative and non-cooperative (eager) strategies must not be "
- "mixed. "
+ "list have higher priority). Cooperative and non-cooperative (eager)"
+ "strategies must not be mixed. `partition.assignment.strategy` is not "
+ "supported for "
+ "`group.protocol=consumer`. Use `group.remote.assignor` instead. "
"Available strategies: range, roundrobin, cooperative-sticky.",
.sdef = "range,roundrobin"},
{_RK_GLOBAL | _RK_CGRP | _RK_HIGH, "session.timeout.ms", _RK_C_INT,
@@ -1122,20 +1308,35 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
"to indicate its liveness to the broker. If no hearts are "
"received by the broker for a group member within the "
"session timeout, the broker will remove the consumer from "
- "the group and trigger a rebalance. "
- "The allowed range is configured with the **broker** configuration "
+ "the group and trigger a rebalance. The "
+ "allowed range is configured with the **broker** configuration "
"properties `group.min.session.timeout.ms` and "
- "`group.max.session.timeout.ms`. "
+ "`group.max.session.timeout.ms`. `session.timeout.ms` is not supported "
+ "for `group.protocol=consumer`. It is set with the broker configuration "
+ "property "
+ "`group.consumer.session.timeout.ms` by default or can be configured "
+ "through the AdminClient IncrementalAlterConfigs API. "
+ "The allowed range is configured with the broker configuration "
+ "properties `group.consumer.min.session.timeout.ms` and "
+ "`group.consumer.max.session.timeout.ms`. "
"Also see `max.poll.interval.ms`.",
1, 3600 * 1000, 45 * 1000},
{_RK_GLOBAL | _RK_CGRP, "heartbeat.interval.ms", _RK_C_INT,
_RK(group_heartbeat_intvl_ms),
- "Group session keepalive heartbeat interval.", 1, 3600 * 1000, 3 * 1000},
+ "Group session keepalive heartbeat interval. "
+ "`heartbeat.interval.ms` is not supported for `group.protocol=consumer`. "
+ "It is set with the broker configuration property "
+ "`group.consumer.heartbeat.interval.ms` by default or can be configured "
+ "through the AdminClient IncrementalAlterConfigs API. The allowed range "
+ "is configured with the broker configuration properties "
+ "`group.consumer.min.heartbeat.interval.ms` and "
+ "`group.consumer.max.heartbeat.interval.ms`.",
+ 1, 3600 * 1000, 3 * 1000},
{_RK_GLOBAL | _RK_CGRP, "group.protocol.type", _RK_C_KSTR,
_RK(group_protocol_type),
"Group protocol type for the `classic` group protocol. NOTE: Currently, "
- "the only supported group "
- "protocol type is `consumer`.",
+ "the only supported group protocol type is `consumer`. "
+ "`group.protocol.type` is not supported for `group.protocol=consumer`",
.sdef = "consumer"},
{_RK_GLOBAL | _RK_CGRP | _RK_HIGH, "group.protocol", _RK_C_S2I,
_RK(group_protocol),
@@ -1146,7 +1347,7 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
"but will change to `consumer` in next releases.",
.vdef = RD_KAFKA_GROUP_PROTOCOL_CLASSIC,
.s2i = {{RD_KAFKA_GROUP_PROTOCOL_CLASSIC, "classic"},
- {RD_KAFKA_GROUP_PROTOCOL_CONSUMER, "consumer"}}},
+ {RD_KAFKA_GROUP_PROTOCOL_CONSUMER, "consumer"}}},
{_RK_GLOBAL | _RK_CGRP | _RK_MED, "group.remote.assignor", _RK_C_STR,
_RK(group_remote_assignor),
"Server side assignor to use. Keep it null to make server select a "
@@ -1272,8 +1473,8 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
"(requires Apache Kafka 0.8.2 or later on the broker).",
.vdef = RD_KAFKA_OFFSET_METHOD_BROKER,
.s2i = {{RD_KAFKA_OFFSET_METHOD_NONE, "none"},
- {RD_KAFKA_OFFSET_METHOD_FILE, "file"},
- {RD_KAFKA_OFFSET_METHOD_BROKER, "broker"}}},
+ {RD_KAFKA_OFFSET_METHOD_FILE, "file"},
+ {RD_KAFKA_OFFSET_METHOD_BROKER, "broker"}}},
{_RK_GLOBAL | _RK_CONSUMER | _RK_HIGH, "isolation.level", _RK_C_S2I,
_RK(isolation_level),
"Controls how to read messages written transactionally: "
@@ -1282,7 +1483,7 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
"transactional messages which have been aborted.",
.vdef = RD_KAFKA_READ_COMMITTED,
.s2i = {{RD_KAFKA_READ_UNCOMMITTED, "read_uncommitted"},
- {RD_KAFKA_READ_COMMITTED, "read_committed"}}},
+ {RD_KAFKA_READ_COMMITTED, "read_committed"}}},
{_RK_GLOBAL | _RK_CONSUMER, "consume_cb", _RK_C_PTR, _RK(consume_cb),
"Message consume callback (set with rd_kafka_conf_set_consume_cb())"},
{_RK_GLOBAL | _RK_CONSUMER, "rebalance_cb", _RK_C_PTR, _RK(rebalance_cb),
@@ -1365,7 +1566,8 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
{_RK_GLOBAL | _RK_PRODUCER | _RK_HIGH, "queue.buffering.max.messages",
_RK_C_INT, _RK(queue_buffering_max_msgs),
"Maximum number of messages allowed on the producer queue. "
- "This queue is shared by all topics and partitions. A value of 0 disables "
+ "This queue is shared by all topics and partitions. A value of 0 "
+ "disables "
"this limit.",
0, INT_MAX, 100000},
{_RK_GLOBAL | _RK_PRODUCER | _RK_HIGH, "queue.buffering.max.kbytes",
@@ -1403,7 +1605,8 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
{_RK_GLOBAL | _RK_MED, "retry.backoff.max.ms", _RK_C_INT,
_RK(retry_backoff_max_ms),
- "The max backoff time in milliseconds before retrying a protocol request, "
+ "The max backoff time in milliseconds before retrying a protocol "
+ "request, "
"this is the atmost backoff allowed for exponentially backed off "
"requests.",
1, 300 * 1000, 1000},
@@ -1427,11 +1630,11 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
"the topic configuration property `compression.codec`. ",
.vdef = RD_KAFKA_COMPRESSION_NONE,
.s2i = {{RD_KAFKA_COMPRESSION_NONE, "none"},
- {RD_KAFKA_COMPRESSION_GZIP, "gzip", _UNSUPPORTED_ZLIB},
- {RD_KAFKA_COMPRESSION_SNAPPY, "snappy", _UNSUPPORTED_SNAPPY},
- {RD_KAFKA_COMPRESSION_LZ4, "lz4"},
- {RD_KAFKA_COMPRESSION_ZSTD, "zstd", _UNSUPPORTED_ZSTD},
- {0}}},
+ {RD_KAFKA_COMPRESSION_GZIP, "gzip", _UNSUPPORTED_ZLIB},
+ {RD_KAFKA_COMPRESSION_SNAPPY, "snappy", _UNSUPPORTED_SNAPPY},
+ {RD_KAFKA_COMPRESSION_LZ4, "lz4"},
+ {RD_KAFKA_COMPRESSION_ZSTD, "zstd", _UNSUPPORTED_ZSTD},
+ {0}}},
{_RK_GLOBAL | _RK_PRODUCER | _RK_MED, "compression.type", _RK_C_ALIAS,
.sdef = "compression.codec"},
{_RK_GLOBAL | _RK_PRODUCER | _RK_MED, "batch.num.messages", _RK_C_INT,
@@ -1472,7 +1675,8 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
0, 900000, 10},
{_RK_GLOBAL, "client.dns.lookup", _RK_C_S2I, _RK(client_dns_lookup),
"Controls how the client uses DNS lookups. By default, when the lookup "
- "returns multiple IP addresses for a hostname, they will all be attempted "
+ "returns multiple IP addresses for a hostname, they will all be "
+ "attempted "
"for connection before the connection is considered failed. This applies "
"to both bootstrap and advertised servers. If the value is set to "
"`resolve_canonical_bootstrap_servers_only`, each entry will be resolved "
@@ -1485,8 +1689,8 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
"hostname. ",
.vdef = RD_KAFKA_USE_ALL_DNS_IPS,
.s2i = {{RD_KAFKA_USE_ALL_DNS_IPS, "use_all_dns_ips"},
- {RD_KAFKA_RESOLVE_CANONICAL_BOOTSTRAP_SERVERS_ONLY,
- "resolve_canonical_bootstrap_servers_only"}}},
+ {RD_KAFKA_RESOLVE_CANONICAL_BOOTSTRAP_SERVERS_ONLY,
+ "resolve_canonical_bootstrap_servers_only"}}},
{_RK_GLOBAL, "enable.metrics.push", _RK_C_BOOL, _RK(enable_metrics_push),
"Whether to enable pushing of client metrics to the cluster, if the "
"cluster has a client metrics subscription which matches this client",
@@ -1558,7 +1762,8 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
"`murmur2_random` - Java Producer compatible Murmur2 hash of key "
"(NULL keys are randomly partitioned. This is functionally equivalent "
"to the default partitioner in the Java Producer.), "
- "`fnv1a` - FNV-1a hash of key (NULL keys are mapped to single partition), "
+ "`fnv1a` - FNV-1a hash of key (NULL keys are mapped to single "
+ "partition), "
"`fnv1a_random` - FNV-1a hash of key (NULL keys are randomly "
"partitioned).",
.sdef = "consistent_random",
@@ -1579,12 +1784,12 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
"inherit = inherit global compression.codec configuration.",
.vdef = RD_KAFKA_COMPRESSION_INHERIT,
.s2i = {{RD_KAFKA_COMPRESSION_NONE, "none"},
- {RD_KAFKA_COMPRESSION_GZIP, "gzip", _UNSUPPORTED_ZLIB},
- {RD_KAFKA_COMPRESSION_SNAPPY, "snappy", _UNSUPPORTED_SNAPPY},
- {RD_KAFKA_COMPRESSION_LZ4, "lz4"},
- {RD_KAFKA_COMPRESSION_ZSTD, "zstd", _UNSUPPORTED_ZSTD},
- {RD_KAFKA_COMPRESSION_INHERIT, "inherit"},
- {0}}},
+ {RD_KAFKA_COMPRESSION_GZIP, "gzip", _UNSUPPORTED_ZLIB},
+ {RD_KAFKA_COMPRESSION_SNAPPY, "snappy", _UNSUPPORTED_SNAPPY},
+ {RD_KAFKA_COMPRESSION_LZ4, "lz4"},
+ {RD_KAFKA_COMPRESSION_ZSTD, "zstd", _UNSUPPORTED_ZSTD},
+ {RD_KAFKA_COMPRESSION_INHERIT, "inherit"},
+ {0}}},
{_RK_TOPIC | _RK_PRODUCER | _RK_HIGH, "compression.type", _RK_C_ALIAS,
.sdef = "compression.codec"},
{_RK_TOPIC | _RK_PRODUCER | _RK_MED, "compression.level", _RK_C_INT,
@@ -1668,7 +1873,7 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
"Apache Kafka 0.8.2 or later on the broker.).",
.vdef = RD_KAFKA_OFFSET_METHOD_BROKER,
.s2i = {{RD_KAFKA_OFFSET_METHOD_FILE, "file"},
- {RD_KAFKA_OFFSET_METHOD_BROKER, "broker"}}},
+ {RD_KAFKA_OFFSET_METHOD_BROKER, "broker"}}},
{_RK_TOPIC | _RK_CONSUMER, "consume.callback.max.messages", _RK_C_INT,
_RKT(consume_callback_max_msgs),
@@ -2328,7 +2533,7 @@ static int rd_kafka_anyconf_set(int scope,
const struct rd_kafka_property *_prop; \
rd_kafka_conf_res_t _res; \
_prop = rd_kafka_conf_prop_find(SCOPE, NAME); \
- rd_assert(_prop && * "invalid property name"); \
+ rd_assert(_prop && *"invalid property name"); \
_res = rd_kafka_anyconf_set_prop( \
SCOPE, CONF, _prop, (const void *)VALUE, \
1 /*allow-specifics*/, NULL, 0); \
@@ -3735,6 +3940,201 @@ char **rd_kafka_conf_kv_split(const char **input, size_t incnt, size_t *cntp) {
return out;
}
+/**
+ * @brief Get value for the config param corresponding to \p key in
+ * \p config, using \p pairs_sep for splitting it
+ * into key-value pairs and '=' for splitting keys and values.
+ */
+char *rd_kafka_conf_kv_get(const char *config,
+ const char *key,
+ const char pairs_sep) {
+ size_t i, config_pair_cnt, config_key_value_cnt;
+ char *ret = NULL;
+ char **config_key_values;
+ if (!config)
+ return NULL;
+
+ char **config_pairs =
+ rd_string_split(config, pairs_sep, rd_true, &config_pair_cnt);
+
+ config_key_values =
+ rd_kafka_conf_kv_split((const char **)config_pairs, config_pair_cnt,
+ &config_key_value_cnt);
+ for (i = 0; i < config_key_value_cnt / 2; i += 2) {
+ char *config_key = config_key_values[i];
+ if (!rd_strcmp(config_key, key)) {
+ ret = rd_strdup(config_key_values[i + 1]);
+ break;
+ }
+ }
+ rd_free(config_key_values);
+ rd_free(config_pairs);
+ return ret;
+}
+
+const char *
+rd_kafka_conf_finalize_oauthbearer_oidc_grant_type(rd_kafka_conf_t *conf) {
+ switch (conf->sasl.oauthbearer.grant_type) {
+ case RD_KAFKA_SASL_OAUTHBEARER_GRANT_TYPE_CLIENT_CREDENTIALS:
+ if (!conf->sasl.oauthbearer.client_id)
+ return "`sasl.oauthbearer.client.id` is "
+ "mandatory when `sasl.oauthbearer.method=oidc` "
+ "is set";
+
+ if (!conf->sasl.oauthbearer.client_secret) {
+ return "`sasl.oauthbearer.client.secret` is "
+ "mandatory when `sasl.oauthbearer.method=oidc` "
+ "is set";
+ }
+ break;
+ case RD_KAFKA_SASL_OAUTHBEARER_GRANT_TYPE_JWT_BEARER:
+ if (conf->sasl.oauthbearer.assertion.file) {
+ if (conf->sasl.oauthbearer.assertion.private_key.file)
+ return "Mutually exclusive properties set. "
+ "`sasl.oauthbearer.assertion.file` and "
+ "`sasl.oauthbearer.assertion.private."
+ "key.file` cannot both be set";
+
+ if (conf->sasl.oauthbearer.assertion.private_key.pem)
+ return "Mutually exclusive properties set. "
+ "`sasl.oauthbearer.assertion.file` and "
+ "`sasl.oauthbearer.assertion.private."
+ "key.pem` cannot both be set";
+
+ if (conf->sasl.oauthbearer.assertion.private_key
+ .passphrase)
+ return "Mutually exclusive properties set. "
+ "`sasl.oauthbearer.assertion.file` and "
+ "`sasl.oauthbearer.assertion.private."
+ "key.passphrase` cannot both be set";
+
+ if (conf->sasl.oauthbearer.assertion.jwt_template_file)
+ return "Mutually exclusive properties set. "
+ "`sasl.oauthbearer.assertion.file` and "
+ "`sasl.oauthbearer.assertion."
+ "jwt.template.file` cannot both be set";
+
+ if (conf->sasl.oauthbearer.assertion.claim.subject)
+ return "Mutually exclusive properties set. "
+ "`sasl.oauthbearer.assertion.file` and "
+ "`sasl.oauthbearer.assertion."
+ "claim.sub` cannot both be set";
+
+ if (conf->sasl.oauthbearer.assertion.claim.audience)
+ return "Mutually exclusive properties set. "
+ "`sasl.oauthbearer.assertion.file` and "
+ "`sasl.oauthbearer.assertion."
+ "claim.aud` cannot both be set";
+
+ if (conf->sasl.oauthbearer.assertion.claim.issuer)
+ return "Mutually exclusive properties set. "
+ "`sasl.oauthbearer.assertion.file` and "
+ "`sasl.oauthbearer.assertion."
+ "claim.iss` cannot both be set";
+
+ if (rd_kafka_conf_is_modified(
+ conf,
+ "sasl.oauthbearer."
+ "assertion.claim.jti.include"))
+ return "Mutually exclusive properties set. "
+ "`sasl.oauthbearer.assertion.file` and "
+ "`sasl.oauthbearer.assertion."
+ "claim.jti.include` cannot both be set";
+
+ if (rd_kafka_conf_is_modified(
+ conf,
+ "sasl.oauthbearer."
+ "assertion.claim.exp.seconds"))
+ return "Mutually exclusive properties set. "
+ "`sasl.oauthbearer.assertion.file` and "
+ "`sasl.oauthbearer.assertion."
+ "claim.exp.seconds` cannot both be set";
+
+
+ if (rd_kafka_conf_is_modified(
+ conf,
+ "sasl.oauthbearer."
+ "assertion.claim.nbf.seconds"))
+ return "Mutually exclusive properties set. "
+ "`sasl.oauthbearer.assertion.file` and "
+ "`sasl.oauthbearer.assertion."
+ "claim.nbf.seconds` cannot both be set";
+ } else {
+ if (conf->sasl.oauthbearer.assertion.private_key.file &&
+ conf->sasl.oauthbearer.assertion.private_key.pem)
+ return "Mutually exclusive properties set. "
+ "`sasl.oauthbearer.assertion.private."
+ "key.file` and "
+ "`sasl.oauthbearer.assertion.private."
+ "key.pem` cannot both be set";
+
+ if (!conf->sasl.oauthbearer.assertion.private_key
+ .file &&
+ !conf->sasl.oauthbearer.assertion.private_key.pem)
+ return "`sasl.oauthbearer.assertion.private."
+ "key.file` or "
+ "`sasl.oauthbearer.assertion.private."
+ "key.pem` is mandatory when "
+ "`sasl.oauthbearer.grant.type` "
+ "is set to "
+ "`urn:ietf:params:oauth:grant-"
+ "type:jwt-bearer`";
+ }
+ break;
+ default:
+ break;
+ }
+ return NULL;
+}
+
+const char *rd_kafka_conf_finalize_oauthbearer_oidc(rd_kafka_conf_t *conf) {
+ const char *errstr;
+ if (conf->sasl.oauthbearer.method !=
+ RD_KAFKA_SASL_OAUTHBEARER_METHOD_OIDC)
+ return NULL;
+
+ if (conf->sasl.enable_oauthbearer_unsecure_jwt)
+ return "`enable.sasl.oauthbearer.unsecure.jwt` and "
+ "`sasl.oauthbearer.method=oidc` are "
+ "mutually exclusive";
+
+ if (!conf->sasl.oauthbearer.token_endpoint_url) {
+ const char *errstr =
+ "`sasl.oauthbearer.token.endpoint.url` "
+ "is mandatory when "
+ "`sasl.oauthbearer.method=oidc` is set";
+ if (conf->sasl.oauthbearer.metadata_authentication.type ==
+ RD_KAFKA_SASL_OAUTHBEARER_METADATA_AUTHENTICATION_TYPE_AZURE_IMDS) {
+ char *query = rd_kafka_conf_kv_get(
+ conf->sasl.oauthbearer_config, "query", ',');
+ if (!query)
+ return "`sasl.oauthbearer.token.endpoint.url` "
+ "is mandatory for Azure IMDS "
+ "authentication "
+ "when `query` isn't set";
+ rd_free(query);
+ } else {
+ return errstr;
+ }
+ }
+
+ if (conf->sasl.oauthbearer.metadata_authentication.type ==
+ RD_KAFKA_SASL_OAUTHBEARER_METADATA_AUTHENTICATION_TYPE_NONE) {
+ if ((errstr =
+ rd_kafka_conf_finalize_oauthbearer_oidc_grant_type(
+ conf))) {
+ return errstr;
+ }
+ }
+
+ /* Enable background thread for the builtin OIDC handler,
+ * unless a refresh callback has been set. */
+ if (!conf->sasl.oauthbearer.token_refresh_cb) {
+ conf->enabled_events |= RD_KAFKA_EVENT_BACKGROUND;
+ conf->sasl.enable_callback_queue = 1;
+ }
+ return NULL;
+}
/**
* @brief Verify configuration \p conf is
@@ -3776,10 +4176,33 @@ const char *rd_kafka_conf_finalize(rd_kafka_type_t cltype,
if (conf->ssl.ca && (conf->ssl.ca_location || conf->ssl.ca_pem))
return "`ssl.ca.location` or `ssl.ca.pem`, and memory-based "
"set_ssl_cert(CERT_CA) are mutually exclusive.";
+
+#if WITH_OAUTHBEARER_OIDC
+ if (conf->https.ca_location && conf->https.ca_pem)
+ return "`https.ca.location` and `https.ca.pem` "
+ "are mutually exclusive";
+
+ if (conf->https.ca_location &&
+ rd_strcmp(conf->https.ca_location, "probe") &&
+ !rd_file_stat(conf->https.ca_location, NULL))
+ return "`https.ca.location` must be "
+ "an existing file or directory";
+
+#if !CURL_AT_LEAST_VERSION(7, 77, 0)
+ if (conf->https.ca_pem)
+ return "`https.ca.pem` requires libcurl 7.77.0 or later";
+#endif
+#endif
+
+
#ifdef __APPLE__
else if (!conf->ssl.ca && !conf->ssl.ca_location && !conf->ssl.ca_pem)
/* Default ssl.ca.location to 'probe' on OSX */
rd_kafka_conf_set(conf, "ssl.ca.location", "probe", NULL, 0);
+
+ /* Default https.ca.location to 'probe' on OSX */
+ if (!conf->https.ca_location && !conf->https.ca_pem)
+ rd_kafka_conf_set(conf, "https.ca.location", "probe", NULL, 0);
#endif
#endif
@@ -3791,46 +4214,51 @@ const char *rd_kafka_conf_finalize(rd_kafka_type_t cltype,
"`oauthbearer_token_refresh_cb` are "
"mutually exclusive";
- if (conf->sasl.enable_oauthbearer_unsecure_jwt &&
- conf->sasl.oauthbearer.method ==
- RD_KAFKA_SASL_OAUTHBEARER_METHOD_OIDC)
- return "`enable.sasl.oauthbearer.unsecure.jwt` and "
- "`sasl.oauthbearer.method=oidc` are "
- "mutually exclusive";
+ errstr = rd_kafka_conf_finalize_oauthbearer_oidc(conf);
+ if (errstr)
+ return errstr;
+ }
- if (conf->sasl.oauthbearer.method ==
- RD_KAFKA_SASL_OAUTHBEARER_METHOD_OIDC) {
- if (!conf->sasl.oauthbearer.client_id)
- return "`sasl.oauthbearer.client.id` is "
- "mandatory when "
- "`sasl.oauthbearer.method=oidc` is set";
-
- if (!conf->sasl.oauthbearer.client_secret) {
- return "`sasl.oauthbearer.client.secret` is "
- "mandatory when "
- "`sasl.oauthbearer.method=oidc` is set";
- }
+#endif
- if (!conf->sasl.oauthbearer.token_endpoint_url) {
- return "`sasl.oauthbearer.token.endpoint.url` "
- "is mandatory when "
- "`sasl.oauthbearer.method=oidc` is set";
+ if (cltype == RD_KAFKA_CONSUMER) {
+
+ if (conf->group_protocol == RD_KAFKA_GROUP_PROTOCOL_CLASSIC) {
+ if (conf->max_poll_interval_ms <
+ conf->group_session_timeout_ms)
+ return "`max.poll.interval.ms`must be >= "
+ "`session.timeout.ms`";
+ } else {
+
+ if (rd_kafka_conf_is_modified(conf,
+ "session.timeout.ms")) {
+ return "`session.timeout.ms` is not supported "
+ "for `group.protocol=consumer`. It is "
+ "defined broker side";
}
- }
- /* Enable background thread for the builtin OIDC handler,
- * unless a refresh callback has been set. */
- if (conf->sasl.oauthbearer.method ==
- RD_KAFKA_SASL_OAUTHBEARER_METHOD_OIDC &&
- !conf->sasl.oauthbearer.token_refresh_cb) {
- conf->enabled_events |= RD_KAFKA_EVENT_BACKGROUND;
- conf->sasl.enable_callback_queue = 1;
- }
- }
+ if (rd_kafka_conf_is_modified(
+ conf, "partition.assignment.strategy")) {
+ return "`partition.assignment.strategy` is not "
+ "supported for "
+ "`group.protocol=consumer`. Use "
+ "`group.remote.assignor` instead";
+ }
-#endif
+ if (rd_kafka_conf_is_modified(conf,
+ "group.protocol.type")) {
+ return "`group.protocol.type` is not supported "
+ "for `group.protocol=consumer`";
+ }
- if (cltype == RD_KAFKA_CONSUMER) {
+ if (rd_kafka_conf_is_modified(
+ conf, "heartbeat.interval.ms")) {
+ return "`heartbeat.interval.ms` is not "
+ "supported "
+ "for `group.protocol=consumer`. It is "
+ "defined broker side";
+ }
+ }
/* Automatically adjust `fetch.max.bytes` to be >=
* `message.max.bytes` and <= `queued.max.message.kbytes`
@@ -3862,10 +4290,6 @@ const char *rd_kafka_conf_finalize(rd_kafka_type_t cltype,
conf->fetch_max_bytes + 512);
}
- if (conf->max_poll_interval_ms < conf->group_session_timeout_ms)
- return "`max.poll.interval.ms`must be >= "
- "`session.timeout.ms`";
-
/* Simplifies rd_kafka_is_idempotent() which is producer-only */
conf->eos.idempotence = 0;
@@ -3960,7 +4384,7 @@ const char *rd_kafka_conf_finalize(rd_kafka_type_t cltype,
if (conf->reconnect_backoff_max_ms < conf->reconnect_backoff_ms)
return "`reconnect.backoff.max.ms` must be >= "
- "`reconnect.max.ms`";
+ "`reconnect.backoff.ms`";
if (conf->sparse_connections) {
/* Set sparse connection random selection interval to
@@ -4423,7 +4847,7 @@ int unittest_conf(void) {
/* Verify that software.client.* string-safing works */
conf = rd_kafka_conf_new();
res = rd_kafka_conf_set(conf, "client.software.name",
- " .~aba. va! !.~~", NULL, 0);
+ " .~aba. va! !.~~", NULL, 0);
RD_UT_ASSERT(res == RD_KAFKA_CONF_OK, "%d", res);
res = rd_kafka_conf_set(conf, "client.software.version",
"!1.2.3.4.5!!! a", NULL, 0);
@@ -4442,7 +4866,7 @@ int unittest_conf(void) {
readlen = sizeof(readval);
res2 = rd_kafka_conf_get(conf, "client.software.version", readval,
- &readlen);
+ &readlen);
RD_UT_ASSERT(res2 == RD_KAFKA_CONF_OK, "%d", res2);
RD_UT_ASSERT(!strcmp(readval, "1.2.3.4.5----a"),
"client.software.* safification failed: \"%s\"", readval);
diff --git a/src/rdkafka_conf.h b/src/rdkafka_conf.h
index 5c41513043..92e5193eb7 100644
--- a/src/rdkafka_conf.h
+++ b/src/rdkafka_conf.h
@@ -34,7 +34,7 @@
#include "rdkafka_cert.h"
#if WITH_SSL && OPENSSL_VERSION_NUMBER >= 0x10100000 && \
- !defined(OPENSSL_IS_BORINGSSL)
+ !defined(OPENSSL_NO_ENGINE)
#define WITH_SSL_ENGINE 1
/* Deprecated in OpenSSL 3 */
#include
@@ -151,9 +151,29 @@ typedef enum {
typedef enum {
RD_KAFKA_SASL_OAUTHBEARER_METHOD_DEFAULT,
- RD_KAFKA_SASL_OAUTHBEARER_METHOD_OIDC
+ RD_KAFKA_SASL_OAUTHBEARER_METHOD_OIDC,
} rd_kafka_oauthbearer_method_t;
+typedef enum {
+ RD_KAFKA_SASL_OAUTHBEARER_GRANT_TYPE_CLIENT_CREDENTIALS,
+ RD_KAFKA_SASL_OAUTHBEARER_GRANT_TYPE_JWT_BEARER,
+} rd_kafka_oauthbearer_grant_type_t;
+
+typedef enum {
+ RD_KAFKA_SASL_OAUTHBEARER_ASSERTION_ALGORITHM_RS256,
+ RD_KAFKA_SASL_OAUTHBEARER_ASSERTION_ALGORITHM_ES256,
+} rd_kafka_oauthbearer_assertion_algorithm_t;
+
+typedef enum {
+ RD_KAFKA_SASL_OAUTHBEARER_METADATA_AUTHENTICATION_TYPE_NONE,
+ RD_KAFKA_SASL_OAUTHBEARER_METADATA_AUTHENTICATION_TYPE_AZURE_IMDS,
+} rd_kafka_oauthbearer_metadata_authentication_type_t;
+
+
+#define RD_KAFKA_SASL_OAUTHBEARER_METADATA_AUTHENTICATION_URL_AZURE_IMDS \
+ "http://169.254.169.254/metadata/identity/oauth2/token"
+
+
typedef enum {
RD_KAFKA_SSL_ENDPOINT_ID_NONE,
RD_KAFKA_SSL_ENDPOINT_ID_HTTPS, /**< RFC2818 */
@@ -169,9 +189,14 @@ typedef enum {
RD_KAFKA_GROUP_PROTOCOL_CONSUMER,
} rd_kafka_group_protocol_t;
+typedef enum {
+ RD_KAFKA_METADATA_RECOVERY_STRATEGY_NONE,
+ RD_KAFKA_METADATA_RECOVERY_STRATEGY_REBOOTSTRAP,
+} rd_kafka_metadata_recovery_strategy_t;
+
/* Increase in steps of 64 as needed.
* This must be larger than sizeof(rd_kafka_[topic_]conf_t) */
-#define RD_KAFKA_CONF_PROPS_IDX_MAX (64 * 33)
+#define RD_KAFKA_CONF_PROPS_IDX_MAX (64 * 35)
/**
* @struct rd_kafka_anyconf_t
@@ -202,6 +227,7 @@ struct rd_kafka_conf_s {
int msg_copy_max_size;
int recv_max_msg_size;
int max_inflight;
+ int metadata_recovery_rebootstrap_trigger_ms;
int metadata_request_timeout_ms;
int metadata_refresh_interval_ms;
int metadata_refresh_fast_cnt;
@@ -236,6 +262,7 @@ struct rd_kafka_conf_s {
char *broker_version_fallback;
rd_kafka_secproto_t security_protocol;
rd_kafka_client_dns_lookup_t client_dns_lookup;
+ rd_kafka_metadata_recovery_strategy_t metadata_recovery_strategy;
struct {
#if WITH_SSL
@@ -281,6 +308,11 @@ struct rd_kafka_conf_s {
void *opaque);
} ssl;
+ struct {
+ char *ca_location;
+ char *ca_pem;
+ } https;
+
struct {
const struct rd_kafka_sasl_provider *provider;
char *principal;
@@ -310,11 +342,41 @@ struct rd_kafka_conf_s {
int enable_callback_queue;
struct {
rd_kafka_oauthbearer_method_t method;
+ rd_kafka_oauthbearer_grant_type_t grant_type;
char *token_endpoint_url;
char *client_id;
char *client_secret;
char *scope;
+ struct {
+ rd_kafka_oauthbearer_assertion_algorithm_t
+ algorithm;
+ char *file;
+ char *jwt_template_file;
+
+ struct {
+ char *subject;
+ char *audience;
+ char *issuer;
+ rd_bool_t jti_include;
+ int not_before_s;
+ int expiration_s;
+ } claim;
+ struct {
+ char *file;
+ char *passphrase;
+ char *pem;
+ } private_key;
+
+ } assertion;
+
+ struct {
+ rd_kafka_oauthbearer_metadata_authentication_type_t
+ type;
+ } metadata_authentication;
+
+
char *extensions_str;
+ rd_bool_t builtin_token_refresh_cb;
/* SASL/OAUTHBEARER token refresh event callback */
void (*token_refresh_cb)(rd_kafka_t *rk,
const char *oauthbearer_config,
@@ -388,7 +450,6 @@ struct rd_kafka_conf_s {
rd_kafkap_str_t *group_protocol_type;
char *partition_assignment_strategy;
rd_list_t partition_assignors;
- rd_bool_t partition_assignors_cooperative;
int enabled_assignor_cnt;
void (*rebalance_cb)(rd_kafka_t *rk,
@@ -637,6 +698,9 @@ struct rd_kafka_topic_conf_s {
char **rd_kafka_conf_kv_split(const char **input, size_t incnt, size_t *cntp);
+char *
+rd_kafka_conf_kv_get(const char *config, const char *key, const char pairs_sep);
+
void rd_kafka_anyconf_destroy(int scope, void *conf);
rd_bool_t rd_kafka_conf_is_modified(const rd_kafka_conf_t *conf,
diff --git a/src/rdkafka_event.c b/src/rdkafka_event.c
index 6ea366a5a8..7e8cd200ae 100644
--- a/src/rdkafka_event.c
+++ b/src/rdkafka_event.c
@@ -97,6 +97,8 @@ const char *rd_kafka_event_name(const rd_kafka_event_t *rkev) {
return "AlterUserScramCredentials";
case RD_KAFKA_EVENT_LISTOFFSETS_RESULT:
return "ListOffsetsResult";
+ case RD_KAFKA_EVENT_ELECTLEADERS_RESULT:
+ return "ElectLeadersResult";
default:
return "?unknown?";
}
@@ -490,3 +492,11 @@ rd_kafka_event_ListConsumerGroupOffsets_result(rd_kafka_event_t *rkev) {
else
return (const rd_kafka_ListConsumerGroupOffsets_result_t *)rkev;
}
+
+const rd_kafka_ElectLeaders_result_t *
+rd_kafka_event_ElectLeaders_result(rd_kafka_event_t *rkev) {
+ if (!rkev || rkev->rko_evtype != RD_KAFKA_EVENT_ELECTLEADERS_RESULT)
+ return NULL;
+ else
+ return (const rd_kafka_ElectLeaders_result_t *)rkev;
+}
diff --git a/src/rdkafka_event.h b/src/rdkafka_event.h
index 5d22456b38..cf63e414eb 100644
--- a/src/rdkafka_event.h
+++ b/src/rdkafka_event.h
@@ -117,6 +117,7 @@ static RD_UNUSED RD_INLINE int rd_kafka_event_setup(rd_kafka_t *rk,
case RD_KAFKA_EVENT_DESCRIBEUSERSCRAMCREDENTIALS_RESULT:
case RD_KAFKA_EVENT_ALTERUSERSCRAMCREDENTIALS_RESULT:
case RD_KAFKA_EVENT_LISTOFFSETS_RESULT:
+ case RD_KAFKA_EVENT_ELECTLEADERS_RESULT:
return 1;
default:
diff --git a/src/rdkafka_feature.c b/src/rdkafka_feature.c
index b32cdf689d..a381d3c930 100644
--- a/src/rdkafka_feature.c
+++ b/src/rdkafka_feature.c
@@ -30,6 +30,7 @@
#include "rdkafka_int.h"
#include "rdkafka_feature.h"
+#include "rdunittest.h"
#include
@@ -74,8 +75,8 @@ static const struct rd_kafka_feature_map {
.feature = RD_KAFKA_FEATURE_MSGVER1,
.depends =
{
- {RD_KAFKAP_Produce, 2, 2},
- {RD_KAFKAP_Fetch, 2, 2},
+ {RD_KAFKAP_Produce, 2, RD_KAFKAP_RPC_VERSION_MAX},
+ {RD_KAFKAP_Fetch, 2, RD_KAFKAP_RPC_VERSION_MAX},
{-1},
},
},
@@ -84,8 +85,8 @@ static const struct rd_kafka_feature_map {
.feature = RD_KAFKA_FEATURE_MSGVER2,
.depends =
{
- {RD_KAFKAP_Produce, 3, 3},
- {RD_KAFKAP_Fetch, 4, 4},
+ {RD_KAFKAP_Produce, 3, RD_KAFKAP_RPC_VERSION_MAX},
+ {RD_KAFKAP_Fetch, 4, RD_KAFKAP_RPC_VERSION_MAX},
{-1},
},
},
@@ -97,7 +98,7 @@ static const struct rd_kafka_feature_map {
.feature = RD_KAFKA_FEATURE_APIVERSION,
.depends =
{
- {RD_KAFKAP_ApiVersion, 0, 0},
+ {RD_KAFKAP_ApiVersion, 0, RD_KAFKAP_RPC_VERSION_MAX},
{-1},
},
},
@@ -106,22 +107,34 @@ static const struct rd_kafka_feature_map {
.feature = RD_KAFKA_FEATURE_BROKER_GROUP_COORD,
.depends =
{
- {RD_KAFKAP_FindCoordinator, 0, 0},
+ {RD_KAFKAP_FindCoordinator, 0, RD_KAFKAP_RPC_VERSION_MAX},
{-1},
},
},
{
- /* @brief >=0.9.0: Broker-based balanced consumer groups. */
+ /* @brief >=0.9.0: Broker-based balanced consumer groups (classic). */
.feature = RD_KAFKA_FEATURE_BROKER_BALANCED_CONSUMER,
.depends =
{
- {RD_KAFKAP_FindCoordinator, 0, 0},
- {RD_KAFKAP_OffsetCommit, 1, 2},
- {RD_KAFKAP_OffsetFetch, 1, 1},
- {RD_KAFKAP_JoinGroup, 0, 0},
- {RD_KAFKAP_SyncGroup, 0, 0},
- {RD_KAFKAP_Heartbeat, 0, 0},
- {RD_KAFKAP_LeaveGroup, 0, 0},
+ {RD_KAFKAP_FindCoordinator, 0, RD_KAFKAP_RPC_VERSION_MAX},
+ {RD_KAFKAP_OffsetCommit, 1, RD_KAFKAP_RPC_VERSION_MAX},
+ {RD_KAFKAP_OffsetFetch, 1, RD_KAFKAP_RPC_VERSION_MAX},
+ {RD_KAFKAP_JoinGroup, 0, RD_KAFKAP_RPC_VERSION_MAX},
+ {RD_KAFKAP_SyncGroup, 0, RD_KAFKAP_RPC_VERSION_MAX},
+ {RD_KAFKAP_Heartbeat, 0, RD_KAFKAP_RPC_VERSION_MAX},
+ {RD_KAFKAP_LeaveGroup, 0, RD_KAFKAP_RPC_VERSION_MAX},
+ {-1},
+ },
+ },
+ {
+ /* @brief Broker-based balanced consumer groups (KIP 848). */
+ .feature = RD_KAFKA_FEATURE_BROKER_BALANCED_CONSUMER,
+ .depends =
+ {
+ {RD_KAFKAP_ConsumerGroupHeartbeat, 0,
+ RD_KAFKAP_RPC_VERSION_MAX},
+ {RD_KAFKAP_OffsetCommit, 9, RD_KAFKAP_RPC_VERSION_MAX},
+ {RD_KAFKAP_OffsetFetch, 9, RD_KAFKAP_RPC_VERSION_MAX},
{-1},
},
},
@@ -130,8 +143,8 @@ static const struct rd_kafka_feature_map {
.feature = RD_KAFKA_FEATURE_THROTTLETIME,
.depends =
{
- {RD_KAFKAP_Produce, 1, 2},
- {RD_KAFKAP_Fetch, 1, 2},
+ {RD_KAFKAP_Produce, 1, RD_KAFKAP_RPC_VERSION_MAX},
+ {RD_KAFKAP_Fetch, 1, RD_KAFKAP_RPC_VERSION_MAX},
{-1},
},
@@ -145,7 +158,18 @@ static const struct rd_kafka_feature_map {
.feature = RD_KAFKA_FEATURE_SASL_GSSAPI,
.depends =
{
- {RD_KAFKAP_JoinGroup, 0, 0},
+ {RD_KAFKAP_JoinGroup, 0, RD_KAFKAP_RPC_VERSION_MAX},
+ {-1},
+ },
+ },
+ {
+ /* @brief >=0.10.0: SASL (GSSAPI) authentication.
+ * Fallback in case JoinGroup is removed along with the
+ * "classic" consumer group protocol. */
+ .feature = RD_KAFKA_FEATURE_SASL_GSSAPI,
+ .depends =
+ {
+ {RD_KAFKAP_SaslHandshake, 0, RD_KAFKAP_RPC_VERSION_MAX},
{-1},
},
},
@@ -156,7 +180,7 @@ static const struct rd_kafka_feature_map {
.feature = RD_KAFKA_FEATURE_SASL_HANDSHAKE,
.depends =
{
- {RD_KAFKAP_SaslHandshake, 0, 0},
+ {RD_KAFKAP_SaslHandshake, 0, RD_KAFKAP_RPC_VERSION_MAX},
{-1},
},
},
@@ -170,7 +194,7 @@ static const struct rd_kafka_feature_map {
.feature = RD_KAFKA_FEATURE_LZ4,
.depends =
{
- {RD_KAFKAP_FindCoordinator, 0, 0},
+ {RD_KAFKAP_FindCoordinator, 0, RD_KAFKAP_RPC_VERSION_MAX},
{-1},
},
},
@@ -179,14 +203,14 @@ static const struct rd_kafka_feature_map {
.feature = RD_KAFKA_FEATURE_OFFSET_TIME,
.depends =
{
- {RD_KAFKAP_ListOffsets, 1, 1},
+ {RD_KAFKAP_ListOffsets, 1, RD_KAFKAP_RPC_VERSION_MAX},
{-1},
}},
{/* @brief >=0.11.0.0: Idempotent Producer*/
.feature = RD_KAFKA_FEATURE_IDEMPOTENT_PRODUCER,
.depends =
{
- {RD_KAFKAP_InitProducerId, 0, 0},
+ {RD_KAFKAP_InitProducerId, 0, RD_KAFKAP_RPC_VERSION_MAX},
{-1},
}},
{
@@ -194,8 +218,8 @@ static const struct rd_kafka_feature_map {
.feature = RD_KAFKA_FEATURE_ZSTD,
.depends =
{
- {RD_KAFKAP_Produce, 7, 7},
- {RD_KAFKAP_Fetch, 10, 10},
+ {RD_KAFKAP_Produce, 7, RD_KAFKAP_RPC_VERSION_MAX},
+ {RD_KAFKAP_Fetch, 10, RD_KAFKAP_RPC_VERSION_MAX},
{-1},
},
},
@@ -204,8 +228,8 @@ static const struct rd_kafka_feature_map {
.feature = RD_KAFKA_FEATURE_SASL_AUTH_REQ,
.depends =
{
- {RD_KAFKAP_SaslHandshake, 1, 1},
- {RD_KAFKAP_SaslAuthenticate, 0, 1},
+ {RD_KAFKAP_SaslHandshake, 1, RD_KAFKAP_RPC_VERSION_MAX},
+ {RD_KAFKAP_SaslAuthenticate, 0, RD_KAFKAP_RPC_VERSION_MAX},
{-1},
},
},
@@ -273,20 +297,20 @@ int rd_kafka_get_legacy_ApiVersions(const char *broker_version,
struct rd_kafka_ApiVersion **apisp,
size_t *api_cntp,
const char *fallback) {
+#define _VERMAP(PFX, APIS) \
+ { PFX, APIS, RD_ARRAYSIZE(APIS) }
static const struct {
const char *pfx;
struct rd_kafka_ApiVersion *apis;
size_t api_cnt;
- } vermap[] = {
-#define _VERMAP(PFX, APIS) {PFX, APIS, RD_ARRAYSIZE(APIS)}
- _VERMAP("0.9.0", rd_kafka_ApiVersion_0_9_0),
- _VERMAP("0.8.2", rd_kafka_ApiVersion_0_8_2),
- _VERMAP("0.8.1", rd_kafka_ApiVersion_0_8_1),
- _VERMAP("0.8.0", rd_kafka_ApiVersion_0_8_0),
- {"0.7.", NULL}, /* Unsupported */
- {"0.6.", NULL}, /* Unsupported */
- _VERMAP("", rd_kafka_ApiVersion_Queryable),
- {NULL}};
+ } vermap[] = {_VERMAP("0.9.0", rd_kafka_ApiVersion_0_9_0),
+ _VERMAP("0.8.2", rd_kafka_ApiVersion_0_8_2),
+ _VERMAP("0.8.1", rd_kafka_ApiVersion_0_8_1),
+ _VERMAP("0.8.0", rd_kafka_ApiVersion_0_8_0),
+ {"0.7.", NULL}, /* Unsupported */
+ {"0.6.", NULL}, /* Unsupported */
+ _VERMAP("", rd_kafka_ApiVersion_Queryable),
+ {NULL}};
int i;
int fallback_i = -1;
int ret = 0;
@@ -459,3 +483,416 @@ const char *rd_kafka_features2str(int features) {
return ret[reti];
}
+
+int rd_ut_features_check_assert_features(const char *testname,
+ int features,
+ rd_bool_t all_enabled) {
+ int i;
+ for (i = 0; rd_kafka_feature_map[i].feature != 0; i++) {
+ rd_bool_t enabled =
+ (features & rd_kafka_feature_map[i].feature) > 0;
+ RD_UT_ASSERT(
+ enabled == all_enabled,
+ "%s: Feature \"%s\" should %sbe enabled", testname,
+ rd_kafka_features2str(rd_kafka_feature_map[i].feature),
+ all_enabled ? "" : "not ");
+ }
+ return 0;
+}
+
+int rd_ut_features_check(void) {
+ rd_kafka_t rk = RD_ZERO_INIT;
+ rd_kafka_broker_t rkb = RD_ZERO_INIT;
+ rkb.rkb_rk = &rk;
+ int features;
+ int fails = 0;
+
+ /* ApiVersions must be sorted in these structs */
+
+ /* No ApiVersions for testing matching no feature */
+ struct rd_kafka_ApiVersion api_versions_none[] = {0};
+
+ /* ApiVersions removed in KIP-896 */
+ struct rd_kafka_ApiVersion api_versions_kip_896[] = {
+ {
+ .ApiKey = RD_KAFKAP_Produce,
+ .MinVer = 3,
+ .MaxVer = 12,
+ },
+ {
+ .ApiKey = RD_KAFKAP_Fetch,
+ .MinVer = 4,
+ .MaxVer = 17,
+ },
+ {
+ .ApiKey = RD_KAFKAP_ListOffsets,
+ .MinVer = 1,
+ .MaxVer = 10,
+ },
+ {
+ .ApiKey = RD_KAFKAP_Metadata,
+ .MinVer = 0,
+ .MaxVer = 13,
+ },
+ {
+ .ApiKey = RD_KAFKAP_OffsetCommit,
+ .MinVer = 2,
+ .MaxVer = 9,
+ },
+ {
+ .ApiKey = RD_KAFKAP_OffsetFetch,
+ .MinVer = 1,
+ .MaxVer = 9,
+ },
+ {
+ .ApiKey = RD_KAFKAP_FindCoordinator,
+ .MinVer = 0,
+ .MaxVer = 6,
+ },
+ {
+ .ApiKey = RD_KAFKAP_JoinGroup,
+ .MinVer = 2,
+ .MaxVer = 9,
+ },
+ {
+ .ApiKey = RD_KAFKAP_Heartbeat,
+ .MinVer = 0,
+ .MaxVer = 4,
+ },
+ {
+ .ApiKey = RD_KAFKAP_LeaveGroup,
+ .MinVer = 0,
+ .MaxVer = 5,
+ },
+ {
+ .ApiKey = RD_KAFKAP_SyncGroup,
+ .MinVer = 0,
+ .MaxVer = 5,
+ },
+ {
+ .ApiKey = RD_KAFKAP_SaslHandshake,
+ .MinVer = 0,
+ .MaxVer = 1,
+ },
+ {
+ .ApiKey = RD_KAFKAP_ApiVersion,
+ .MinVer = 0,
+ .MaxVer = 4,
+ },
+ {
+ .ApiKey = RD_KAFKAP_InitProducerId,
+ .MinVer = 0,
+ .MaxVer = 5,
+ },
+ {
+ .ApiKey = RD_KAFKAP_SaslAuthenticate,
+ .MinVer = 0,
+ .MaxVer = 2,
+ },
+ };
+
+ /* ApiVersions to remove in next Apache Kafka version
+ * (see baseline in KIP-896) */
+ struct rd_kafka_ApiVersion api_versions_next_apache_kafka_next[] = {
+ {
+ .ApiKey = RD_KAFKAP_Produce,
+ .MinVer = 7,
+ .MaxVer = 12,
+ },
+ {
+ .ApiKey = RD_KAFKAP_Fetch,
+ .MinVer = 10,
+ .MaxVer = 17,
+ },
+ {
+ .ApiKey = RD_KAFKAP_ListOffsets,
+ .MinVer = 4,
+ .MaxVer = 10,
+ },
+ {
+ .ApiKey = RD_KAFKAP_Metadata,
+ .MinVer = 7,
+ .MaxVer = 13,
+ },
+ {
+ .ApiKey = RD_KAFKAP_OffsetCommit,
+ .MinVer = 6,
+ .MaxVer = 9,
+ },
+ {
+ .ApiKey = RD_KAFKAP_OffsetFetch,
+ .MinVer = 5,
+ .MaxVer = 9,
+ },
+ {
+ .ApiKey = RD_KAFKAP_FindCoordinator,
+ .MinVer = 2,
+ .MaxVer = 6,
+ },
+ {
+ .ApiKey = RD_KAFKAP_JoinGroup,
+ .MinVer = 3,
+ .MaxVer = 9,
+ },
+ {
+ .ApiKey = RD_KAFKAP_Heartbeat,
+ .MinVer = 2,
+ .MaxVer = 4,
+ },
+ {
+ .ApiKey = RD_KAFKAP_LeaveGroup,
+ .MinVer = 2,
+ .MaxVer = 5,
+ },
+ {
+ .ApiKey = RD_KAFKAP_SyncGroup,
+ .MinVer = 2,
+ .MaxVer = 5,
+ },
+ {
+ .ApiKey = RD_KAFKAP_SaslHandshake,
+ .MinVer = 1,
+ .MaxVer = 1,
+ },
+ {
+ .ApiKey = RD_KAFKAP_ApiVersion,
+ .MinVer = 2,
+ .MaxVer = 4,
+ },
+ {
+ .ApiKey = RD_KAFKAP_InitProducerId,
+ .MinVer = 1,
+ .MaxVer = 5,
+ },
+ {
+ .ApiKey = RD_KAFKAP_SaslAuthenticate,
+ .MinVer = 0,
+ .MaxVer = 2,
+ },
+ };
+
+ /* ApiVersions don't overlap with those enabling the feature.
+ * At least one condition isn't matching */
+ struct rd_kafka_ApiVersion api_versions_no_overlap[] = {
+ {
+ .ApiKey = RD_KAFKAP_Produce,
+ .MinVer = 2,
+ .MaxVer = 2,
+ },
+ {
+ .ApiKey = RD_KAFKAP_Fetch,
+ .MinVer = 0,
+ .MaxVer = 0,
+ },
+ {
+ .ApiKey = RD_KAFKAP_ListOffsets,
+ .MinVer = 0,
+ .MaxVer = 0,
+ },
+ {
+ .ApiKey = RD_KAFKAP_Metadata,
+ .MinVer = 7,
+ .MaxVer = 13,
+ },
+ {
+ .ApiKey = RD_KAFKAP_OffsetCommit,
+ .MinVer = 6,
+ .MaxVer = 9,
+ },
+ {
+ .ApiKey = RD_KAFKAP_OffsetFetch,
+ .MinVer = 5,
+ .MaxVer = 8,
+ },
+ {
+ .ApiKey = RD_KAFKAP_Heartbeat,
+ .MinVer = 2,
+ .MaxVer = 4,
+ },
+ {
+ .ApiKey = RD_KAFKAP_LeaveGroup,
+ .MinVer = 2,
+ .MaxVer = 5,
+ },
+ {
+ .ApiKey = RD_KAFKAP_SyncGroup,
+ .MinVer = 2,
+ .MaxVer = 5,
+ },
+ {
+ .ApiKey = RD_KAFKAP_SaslAuthenticate,
+ .MinVer = 0,
+ .MaxVer = 2,
+ },
+ };
+
+ /* ApiVersions where the maximum version corresponds to
+ * the minimum version necessary for enabling all features. */
+ struct rd_kafka_ApiVersion api_versions_overlap_by_one[] = {
+ {
+ .ApiKey = RD_KAFKAP_Produce,
+ .MinVer = 3,
+ .MaxVer = 7,
+ },
+ {
+ .ApiKey = RD_KAFKAP_Fetch,
+ .MinVer = 4,
+ .MaxVer = 10,
+ },
+ {
+ .ApiKey = RD_KAFKAP_ListOffsets,
+ .MinVer = 1,
+ .MaxVer = 1,
+ },
+ {
+ .ApiKey = RD_KAFKAP_Metadata,
+ .MinVer = 0,
+ .MaxVer = 13,
+ },
+ {
+ .ApiKey = RD_KAFKAP_OffsetCommit,
+ .MinVer = 2,
+ .MaxVer = 9,
+ },
+ {
+ .ApiKey = RD_KAFKAP_OffsetFetch,
+ .MinVer = 1,
+ .MaxVer = 9,
+ },
+ {
+ .ApiKey = RD_KAFKAP_FindCoordinator,
+ .MinVer = 0,
+ .MaxVer = 0,
+ },
+ {
+ .ApiKey = RD_KAFKAP_JoinGroup,
+ .MinVer = 0,
+ .MaxVer = 0,
+ },
+ {
+ .ApiKey = RD_KAFKAP_Heartbeat,
+ .MinVer = 0,
+ .MaxVer = 0,
+ },
+ {
+ .ApiKey = RD_KAFKAP_LeaveGroup,
+ .MinVer = 0,
+ .MaxVer = 0,
+ },
+ {
+ .ApiKey = RD_KAFKAP_SyncGroup,
+ .MinVer = 0,
+ .MaxVer = 0,
+ },
+ {
+ .ApiKey = RD_KAFKAP_SaslHandshake,
+ .MinVer = 0,
+ .MaxVer = 1,
+ },
+ {
+ .ApiKey = RD_KAFKAP_ApiVersion,
+ .MinVer = 0,
+ .MaxVer = 0,
+ },
+ {
+ .ApiKey = RD_KAFKAP_InitProducerId,
+ .MinVer = 0,
+ .MaxVer = 0,
+ },
+ {
+ .ApiKey = RD_KAFKAP_SaslAuthenticate,
+ .MinVer = 0,
+ .MaxVer = 0,
+ },
+ };
+
+ /* ApiVersions where the classic way of enabling SASL_GSSAPI
+ * or BROKER_BALANCED_CONSUMER was removed. */
+ struct rd_kafka_ApiVersion api_versions_kip_848[] = {
+ {
+ .ApiKey = RD_KAFKAP_OffsetCommit,
+ .MinVer = 6,
+ .MaxVer = 9,
+ },
+ {
+ .ApiKey = RD_KAFKAP_OffsetFetch,
+ .MinVer = 5,
+ .MaxVer = 9,
+ },
+ {
+ .ApiKey = RD_KAFKAP_SaslHandshake,
+ .MinVer = 0,
+ .MaxVer = 1,
+ },
+ {
+ .ApiKey = RD_KAFKAP_ConsumerGroupHeartbeat,
+ .MinVer = 0,
+ .MaxVer = 1,
+ },
+ };
+
+ /* With empty ApiVersions. */
+ features = rd_kafka_features_check(&rkb, api_versions_none,
+ RD_ARRAY_SIZE(api_versions_none));
+ fails += rd_ut_features_check_assert_features("no API versions",
+ features, rd_false);
+
+ /* Without KIP-896 removed versions. */
+ features = rd_kafka_features_check(&rkb, api_versions_kip_896,
+ RD_ARRAY_SIZE(api_versions_kip_896));
+ fails += rd_ut_features_check_assert_features("KIP-896 API versions",
+ features, rd_true);
+
+ /* Without versions to remove in next Apache Kafka major version. */
+ features = rd_kafka_features_check(
+ &rkb, api_versions_next_apache_kafka_next,
+ RD_ARRAY_SIZE(api_versions_next_apache_kafka_next));
+ fails += rd_ut_features_check_assert_features(
+ "next Apache Kafka API versions", features, rd_true);
+
+ /* With a RPC version range non overlapping with the one necessary
+ * for the feature. */
+ features =
+ rd_kafka_features_check(&rkb, api_versions_no_overlap,
+ RD_ARRAY_SIZE(api_versions_no_overlap));
+ fails += rd_ut_features_check_assert_features(
+ "not overlapping API versions", features, rd_false);
+
+ /* Only a single RPC version is overlapping with broker
+ * supported ones. */
+ features =
+ rd_kafka_features_check(&rkb, api_versions_overlap_by_one,
+ RD_ARRAY_SIZE(api_versions_overlap_by_one));
+ fails += rd_ut_features_check_assert_features(
+ "overlapping API versions by one version", features, rd_true);
+
+ /* Even when removing the classic protocol and its APIs,
+ * these features must be enabled. */
+ features = rd_kafka_features_check(&rkb, api_versions_kip_848,
+ RD_ARRAY_SIZE(api_versions_kip_848));
+ if (!(features & RD_KAFKA_FEATURE_BROKER_BALANCED_CONSUMER) ||
+ !(features & RD_KAFKA_FEATURE_SASL_GSSAPI)) {
+ fails++;
+ }
+
+
+ if (fails)
+ return fails;
+
+ RD_UT_PASS();
+}
+
+/**
+ * @name Unit tests
+ * @{
+ *
+ */
+int unittest_feature(void) {
+ int fails = 0;
+
+ fails += rd_ut_features_check();
+
+ return fails;
+}
+
+/**@}*/
diff --git a/src/rdkafka_feature.h b/src/rdkafka_feature.h
index 9597956ee8..fab4005aac 100644
--- a/src/rdkafka_feature.h
+++ b/src/rdkafka_feature.h
@@ -99,4 +99,6 @@ int rd_kafka_features_check(rd_kafka_broker_t *rkb,
const char *rd_kafka_features2str(int features);
+int unittest_feature(void);
+
#endif /* _RDKAFKA_FEATURE_H_ */
diff --git a/src/rdkafka_fetcher.c b/src/rdkafka_fetcher.c
index 98f5e72f92..e275ee5a88 100644
--- a/src/rdkafka_fetcher.c
+++ b/src/rdkafka_fetcher.c
@@ -677,7 +677,7 @@ static rd_kafka_resp_err_t rd_kafka_fetch_reply_handle_partition(
* desynchronized clusters): if so ignore it. */
tver_skel.rktp = rktp;
tver = rd_list_find(request->rkbuf_rktp_vers, &tver_skel,
- rd_kafka_toppar_ver_cmp);
+ rd_kafka_toppar_ver_cmp);
rd_kafka_assert(NULL, tver);
if (tver->rktp != rktp || tver->version < fetch_version) {
rd_rkb_dbg(rkb, MSG, "DROP",
@@ -943,7 +943,21 @@ static void rd_kafka_broker_fetch_reply(rd_kafka_t *rk,
}
}
+/**
+ * @brief Check if any toppars have a zero topic id.
+ *
+ */
+static rd_bool_t can_use_topic_ids(rd_kafka_broker_t *rkb) {
+ rd_kafka_toppar_t *rktp = rkb->rkb_active_toppar_next;
+ do {
+ if (RD_KAFKA_UUID_IS_ZERO(rktp->rktp_rkt->rkt_topic_id))
+ return rd_false;
+ } while ((rktp = CIRCLEQ_LOOP_NEXT(&rkb->rkb_active_toppars, rktp,
+ rktp_activelink)) !=
+ rkb->rkb_active_toppar_next);
+ return rd_true;
+}
/**
* @brief Build and send a Fetch request message for all underflowed toppars
@@ -979,7 +993,13 @@ int rd_kafka_broker_fetch_toppars(rd_kafka_broker_t *rkb, rd_ts_t now) {
ApiVersion = rd_kafka_broker_ApiVersion_supported(rkb, RD_KAFKAP_Fetch,
0, 16, NULL);
- rkbuf = rd_kafka_buf_new_flexver_request(
+
+ /* Fallback to version 12 if topic id is null which can happen if
+ * inter.broker.protocol.version is < 2.8 */
+ if (ApiVersion > 12 && !can_use_topic_ids(rkb))
+ ApiVersion = 12;
+
+ rkbuf = rd_kafka_buf_new_flexver_request(
rkb, RD_KAFKAP_Fetch, 1,
/* MaxWaitTime+MinBytes+MaxBytes+IsolationLevel+
* SessionId+Epoch+TopicCnt */
@@ -1207,7 +1227,7 @@ int rd_kafka_broker_fetch_toppars(rd_kafka_broker_t *rkb, rd_ts_t now) {
* @locality any
* @locks toppar_lock() MUST be held
*/
-rd_bool_t rd_kafka_toppar_fetch_decide_start_from_next_fetch_start(
+static rd_bool_t rd_kafka_toppar_fetch_decide_start_from_next_fetch_start(
rd_kafka_toppar_t *rktp) {
return rktp->rktp_op_version > rktp->rktp_fetch_version ||
rd_kafka_fetch_pos_cmp(&rktp->rktp_next_fetch_start,
@@ -1215,6 +1235,26 @@ rd_bool_t rd_kafka_toppar_fetch_decide_start_from_next_fetch_start(
rktp->rktp_offsets.fetch_pos.offset == RD_KAFKA_OFFSET_INVALID;
}
+/**
+ * @brief Return next fetch start position:
+ * if it should start fetching from next fetch start
+ * or continue with current fetch pos.
+ *
+ * @param rktp The toppar
+ *
+ * @returns Next fetch start position
+ *
+ * @locality any
+ * @locks toppar_lock() MUST be held
+ */
+rd_kafka_fetch_pos_t
+rd_kafka_toppar_fetch_decide_next_fetch_start_pos(rd_kafka_toppar_t *rktp) {
+ if (rd_kafka_toppar_fetch_decide_start_from_next_fetch_start(rktp))
+ return rktp->rktp_next_fetch_start;
+ else
+ return rktp->rktp_offsets.fetch_pos;
+}
+
/**
* @brief Decide whether this toppar should be on the fetch list or not.
*
diff --git a/src/rdkafka_fetcher.h b/src/rdkafka_fetcher.h
index 8c64f3b0d9..e304f1369f 100644
--- a/src/rdkafka_fetcher.h
+++ b/src/rdkafka_fetcher.h
@@ -33,8 +33,8 @@
int rd_kafka_broker_fetch_toppars(rd_kafka_broker_t *rkb, rd_ts_t now);
-rd_bool_t rd_kafka_toppar_fetch_decide_start_from_next_fetch_start(
- rd_kafka_toppar_t *rktp);
+rd_kafka_fetch_pos_t
+rd_kafka_toppar_fetch_decide_next_fetch_start_pos(rd_kafka_toppar_t *rktp);
rd_ts_t rd_kafka_toppar_fetch_decide(rd_kafka_toppar_t *rktp,
rd_kafka_broker_t *rkb,
diff --git a/src/rdkafka_int.h b/src/rdkafka_int.h
index ac6bb004a5..d8370ff599 100644
--- a/src/rdkafka_int.h
+++ b/src/rdkafka_int.h
@@ -290,17 +290,29 @@ struct rd_kafka_s {
TAILQ_HEAD(, rd_kafka_broker_s) rk_brokers;
rd_list_t rk_broker_by_id; /* Fast id lookups. */
rd_atomic32_t rk_broker_cnt;
- /**< Number of brokers in state >= UP */
+ /** Logical brokers count.
+ * Used for calculating ERR__ALL_BROKERS_DOWN. */
+ rd_atomic32_t rk_logical_broker_cnt;
+ /** Number of configured or learned brokers in state >= UP */
rd_atomic32_t rk_broker_up_cnt;
- /**< Number of logical brokers in state >= UP, this is a sub-set
- * of rk_broker_up_cnt. */
- rd_atomic32_t rk_logical_broker_up_cnt;
- /**< Number of brokers that are down, only includes brokers
- * that have had at least one connection attempt. */
+ /** Number of brokers that are down, only includes brokers
+ * that have had at least one connection attempt
+ * and are configured or learned. */
rd_atomic32_t rk_broker_down_cnt;
- /**< Logical brokers currently without an address.
- * Used for calculating ERR__ALL_BROKERS_DOWN. */
- rd_atomic32_t rk_broker_addrless_cnt;
+ /** Set to 1 when there's a re-bootstrap in progress.
+ * Set to 0 when the re-bootstrap is done.
+ * Accessed from the main thread and the broker threads. */
+ rd_atomic32_t rk_rebootstrap_in_progress;
+
+ /**< Additional bootstrap servers list.
+ * contains all brokers added through rd_kafka_brokers_add().
+ * Doesn't contain the initially configured bootstrap brokers. */
+ rd_list_t additional_brokerlists;
+
+ /** Decommissioned threads to await */
+ rd_list_t wait_decommissioned_thrds;
+ /** Decommissioned brokers to await */
+ rd_list_t wait_decommissioned_brokers;
mtx_t rk_internal_rkb_lock;
rd_kafka_broker_t *rk_internal_rkb;
@@ -380,7 +392,11 @@ struct rd_kafka_s {
* (or equivalent).
* Used to enforce
* max.poll.interval.ms.
- * Only relevant for consumer. */
+ * Set to INT64_MAX while polling
+ * to avoid reaching
+ * max.poll.interval.ms. during that time
+ * frame. Only relevant for consumer. */
+
/* First fatal error. */
struct {
rd_atomic32_t err; /**< rd_kafka_resp_err_t */
@@ -396,18 +412,20 @@ struct rd_kafka_s {
rd_ts_t rk_ts_metadata; /* Timestamp of most recent
* metadata. */
- rd_kafka_metadata_internal_t
- *rk_full_metadata; /* Last full metadata. */
- rd_ts_t rk_ts_full_metadata; /* Timestamp of .. */
+ rd_ts_t rk_ts_full_metadata; /* Timestamp of most
+ * recent full
+ * metadata */
struct rd_kafka_metadata_cache rk_metadata_cache; /* Metadata cache */
char *rk_clusterid; /* ClusterId from metadata */
int32_t rk_controllerid; /* ControllerId from metadata */
/**< Producer: Delivery report mode */
- enum { RD_KAFKA_DR_MODE_NONE, /**< No delivery reports */
- RD_KAFKA_DR_MODE_CB, /**< Delivery reports through callback */
- RD_KAFKA_DR_MODE_EVENT, /**< Delivery reports through event API*/
+ enum {
+ RD_KAFKA_DR_MODE_NONE, /**< No delivery reports */
+ RD_KAFKA_DR_MODE_CB, /**< Delivery reports through callback */
+ RD_KAFKA_DR_MODE_EVENT, /**< Delivery reports through event
+ API*/
} rk_drmode;
/* Simple consumer count:
@@ -595,6 +613,16 @@ struct rd_kafka_s {
} rk_curr_msgs;
rd_kafka_timers_t rk_timers;
+
+ /** Metadata refresh timer */
+ rd_kafka_timer_t metadata_refresh_tmr;
+ /** 1s interval timer */
+ rd_kafka_timer_t one_s_tmr;
+ /** Rebootstrap timer.
+ * Will add bootstrap brokers again
+ * when it's fired. */
+ rd_kafka_timer_t rebootstrap_tmr;
+
thrd_t rk_thread;
int rk_initialized; /**< Will be > 0 when the rd_kafka_t
@@ -639,6 +667,12 @@ struct rd_kafka_s {
* Use 10 < reconnect.backoff.jitter.ms / 2 < 1000.
*/
rd_interval_t sparse_connect_random;
+
+ /** Sparse connection timer: fires after remaining time of
+ * `sparse_connect_random` interval + 1ms.
+ */
+ rd_kafka_timer_t sparse_connect_random_tmr;
+
/**< Lock for sparse_connect_random */
mtx_t sparse_connect_lock;
@@ -696,8 +730,28 @@ struct rd_kafka_s {
rd_ts_t ts_last; /**< Timestamp of last push */
rd_ts_t ts_start; /**< Timestamp from when collection
* started */
+ /** Total rebalance latency (ms) up to previous push */
+ uint64_t rebalance_latency_total;
} rk_historic_c;
+ struct {
+ rd_avg_t rk_avg_poll_idle_ratio;
+ rd_avg_t rk_avg_commit_latency; /**< Current commit
+ * latency avg */
+ rd_avg_t
+ rk_avg_rebalance_latency; /**< Current rebalance
+ * latency avg */
+ } rd_avg_current;
+
+ struct {
+ rd_avg_t rk_avg_poll_idle_ratio;
+ rd_avg_t rk_avg_commit_latency; /**< Rolled over commit
+ * latency avg */
+ rd_avg_t
+ rk_avg_rebalance_latency; /**< Rolled over rebalance
+ * latency avg */
+ } rd_avg_rollover;
+
} rk_telemetry;
/* Test mocks */
@@ -844,15 +898,13 @@ rd_kafka_curr_msgs_wait_zero(rd_kafka_t *rk,
int timeout_ms,
unsigned int *curr_msgsp) {
unsigned int cnt;
- struct timespec tspec;
-
- rd_timeout_init_timespec(&tspec, timeout_ms);
+ rd_ts_t abs_timeout = rd_timeout_init(timeout_ms);
mtx_lock(&rk->rk_curr_msgs.lock);
while ((cnt = rk->rk_curr_msgs.cnt) > 0) {
if (cnd_timedwait_abs(&rk->rk_curr_msgs.cnd,
&rk->rk_curr_msgs.lock,
- &tspec) == thrd_timedout)
+ abs_timeout) == thrd_timedout)
break;
}
mtx_unlock(&rk->rk_curr_msgs.lock);
@@ -861,6 +913,9 @@ rd_kafka_curr_msgs_wait_zero(rd_kafka_t *rk,
return cnt == 0;
}
+void rd_kafka_decommissioned_broker_thread_join(rd_kafka_t *rk,
+ void *rkb_decommissioned);
+
void rd_kafka_destroy_final(rd_kafka_t *rk);
void rd_kafka_global_init(void);
@@ -1037,6 +1092,14 @@ int rd_kafka_set_fatal_error0(rd_kafka_t *rk,
rd_kafka_error_t *rd_kafka_get_fatal_error(rd_kafka_t *rk);
+#define rd_kafka_producer_can_have_fatal_errors(rk) \
+ (rk->rk_type == RD_KAFKA_PRODUCER && rk->rk_conf.eos.idempotence)
+
+#define rd_kafka_consumer_can_have_fatal_errors(rk) \
+ (rk->rk_type == RD_KAFKA_CONSUMER && \
+ (rk->rk_conf.group_instance_id || \
+ rk->rk_conf.group_protocol == RD_KAFKA_GROUP_PROTOCOL_CONSUMER))
+
static RD_INLINE RD_UNUSED rd_kafka_resp_err_t
rd_kafka_fatal_error_code(rd_kafka_t *rk) {
/* This is an optimization to avoid an atomic read which are costly
@@ -1046,10 +1109,8 @@ rd_kafka_fatal_error_code(rd_kafka_t *rk) {
* 2) static consumers (group.instance.id)
* 3) Group using consumer protocol (Introduced in KIP-848). See exact
* errors in rd_kafka_cgrp_handle_ConsumerGroupHeartbeat() */
- if ((rk->rk_type == RD_KAFKA_PRODUCER && rk->rk_conf.eos.idempotence) ||
- (rk->rk_type == RD_KAFKA_CONSUMER &&
- (rk->rk_conf.group_instance_id ||
- rk->rk_conf.group_protocol == RD_KAFKA_GROUP_PROTOCOL_CONSUMER)))
+ if (rd_kafka_producer_can_have_fatal_errors(rk) ||
+ rd_kafka_consumer_can_have_fatal_errors(rk))
return rd_atomic32_get(&rk->rk_fatal.err);
return RD_KAFKA_RESP_ERR_NO_ERROR;
@@ -1093,7 +1154,7 @@ static RD_INLINE RD_UNUSED int rd_kafka_max_poll_exceeded(rd_kafka_t *rk) {
last_poll = rd_atomic64_get(&rk->rk_ts_last_poll);
/* Application is blocked in librdkafka function, see
- * rd_kafka_app_poll_blocking(). */
+ * rd_kafka_app_poll_start(). */
if (last_poll == INT64_MAX)
return 0;
@@ -1119,9 +1180,32 @@ static RD_INLINE RD_UNUSED int rd_kafka_max_poll_exceeded(rd_kafka_t *rk) {
* @locality any
* @locks none
*/
-static RD_INLINE RD_UNUSED void rd_kafka_app_poll_blocking(rd_kafka_t *rk) {
- if (rk->rk_type == RD_KAFKA_CONSUMER)
+static RD_INLINE RD_UNUSED void rd_kafka_app_poll_start(rd_kafka_t *rk,
+ rd_kafka_q_t *rkq,
+ rd_ts_t now,
+ rd_bool_t is_blocking) {
+ if (rk->rk_type != RD_KAFKA_CONSUMER)
+ return;
+
+ if (!now)
+ now = rd_clock();
+ if (is_blocking)
rd_atomic64_set(&rk->rk_ts_last_poll, INT64_MAX);
+ if (rkq->rkq_ts_last_poll_end) {
+ int64_t poll_idle_ratio = 0;
+ rd_ts_t poll_interval = now - rkq->rkq_ts_last_poll_start;
+ if (poll_interval) {
+ rd_ts_t idle_interval = rkq->rkq_ts_last_poll_end -
+ rkq->rkq_ts_last_poll_start;
+ poll_idle_ratio =
+ idle_interval * 1000000 / poll_interval;
+ }
+ rd_avg_add(
+ &rk->rk_telemetry.rd_avg_current.rk_avg_poll_idle_ratio,
+ poll_idle_ratio);
+ rkq->rkq_ts_last_poll_start = now;
+ rkq->rkq_ts_last_poll_end = 0;
+ }
}
/**
@@ -1132,9 +1216,11 @@ static RD_INLINE RD_UNUSED void rd_kafka_app_poll_blocking(rd_kafka_t *rk) {
* @locality any
* @locks none
*/
-static RD_INLINE RD_UNUSED void rd_kafka_app_polled(rd_kafka_t *rk) {
+static RD_INLINE RD_UNUSED void rd_kafka_app_polled(rd_kafka_t *rk,
+ rd_kafka_q_t *rkq) {
if (rk->rk_type == RD_KAFKA_CONSUMER) {
- rd_atomic64_set(&rk->rk_ts_last_poll, rd_clock());
+ rd_ts_t now = rd_clock();
+ rd_atomic64_set(&rk->rk_ts_last_poll, now);
if (unlikely(rk->rk_cgrp &&
rk->rk_cgrp->rkcg_group_protocol ==
RD_KAFKA_GROUP_PROTOCOL_CONSUMER &&
@@ -1144,6 +1230,10 @@ static RD_INLINE RD_UNUSED void rd_kafka_app_polled(rd_kafka_t *rk) {
rk->rk_cgrp,
"app polled after poll interval exceeded");
}
+ if (!rkq->rkq_ts_last_poll_end)
+ rkq->rkq_ts_last_poll_end = now;
+ rd_dassert(rkq->rkq_ts_last_poll_end >=
+ rkq->rkq_ts_last_poll_start);
}
}
@@ -1159,5 +1249,12 @@ rd_kafka_resp_err_t rd_kafka_background_thread_create(rd_kafka_t *rk,
char *errstr,
size_t errstr_size);
+void rd_kafka_rebootstrap(rd_kafka_t *rk);
+
+void rd_kafka_rebootstrap_tmr_start_maybe(rd_kafka_t *rk);
+
+int rd_kafka_rebootstrap_tmr_stop(rd_kafka_t *rk);
+
+void rd_kafka_reset_any_broker_down_reported(rd_kafka_t *rk);
#endif /* _RDKAFKA_INT_H_ */
diff --git a/src/rdkafka_metadata.c b/src/rdkafka_metadata.c
index 26a989c0fa..d8d35daace 100644
--- a/src/rdkafka_metadata.c
+++ b/src/rdkafka_metadata.c
@@ -92,69 +92,85 @@ rd_kafka_metadata(rd_kafka_t *rk,
rd_kafka_q_t *rkq;
rd_kafka_broker_t *rkb;
rd_kafka_op_t *rko;
+ rd_kafka_resp_err_t err;
rd_ts_t ts_end = rd_timeout_init(timeout_ms);
rd_list_t topics;
rd_bool_t allow_auto_create_topics =
rk->rk_conf.allow_auto_create_topics;
- /* Query any broker that is up, and if none are up pick the first one,
- * if we're lucky it will be up before the timeout */
- rkb = rd_kafka_broker_any_usable(rk, timeout_ms, RD_DO_LOCK, 0,
- "application metadata request");
- if (!rkb)
- return RD_KAFKA_RESP_ERR__TRANSPORT;
+ do {
+ /* Query any broker that is up, and if none are up pick the
+ * first one, if we're lucky it will be up before the timeout.
+ * Previous decommissioning brokers won't be returned by the
+ * function after receiving the _DESTROY_BROKER error
+ * below. */
+ rkb =
+ rd_kafka_broker_any_usable(rk, timeout_ms, RD_DO_LOCK, 0,
+ "application metadata request");
+ if (!rkb)
+ return RD_KAFKA_RESP_ERR__TRANSPORT;
- rkq = rd_kafka_q_new(rk);
+ rkq = rd_kafka_q_new(rk);
- rd_list_init(&topics, 0, rd_free);
- if (!all_topics) {
- if (only_rkt)
- rd_list_add(&topics,
+ rd_list_init(&topics, 0, rd_free);
+ if (!all_topics) {
+ if (only_rkt)
+ rd_list_add(
+ &topics,
rd_strdup(rd_kafka_topic_name(only_rkt)));
- else {
- int cache_cnt;
- rd_kafka_local_topics_to_list(rkb->rkb_rk, &topics,
- &cache_cnt);
- /* Don't trigger auto-create for cached topics */
- if (rd_list_cnt(&topics) == cache_cnt)
- allow_auto_create_topics = rd_true;
+ else {
+ int cache_cnt;
+ rd_kafka_local_topics_to_list(
+ rkb->rkb_rk, &topics, &cache_cnt);
+ /* Don't trigger auto-create
+ * for cached topics */
+ if (rd_list_cnt(&topics) == cache_cnt)
+ allow_auto_create_topics = rd_true;
+ }
}
- }
- /* Async: request metadata */
- rko = rd_kafka_op_new(RD_KAFKA_OP_METADATA);
- rd_kafka_op_set_replyq(rko, rkq, 0);
- rko->rko_u.metadata.force = 1; /* Force metadata request regardless
- * of outstanding metadata requests. */
- rd_kafka_MetadataRequest(rkb, &topics, NULL, "application requested",
- allow_auto_create_topics,
- /* cgrp_update:
- * Only update consumer group state
- * on response if this lists all
- * topics in the cluster, since a
- * partial request may make it seem
- * like some subscribed topics are missing. */
- all_topics ? rd_true : rd_false,
- rd_false /* force_racks */, rko);
+ /* Async: request metadata */
+ rko = rd_kafka_op_new(RD_KAFKA_OP_METADATA);
+ rd_kafka_op_set_replyq(rko, rkq, 0);
+ rko->rko_u.metadata.force =
+ 1; /* Force metadata request regardless
+ * of outstanding metadata requests. */
+ rd_kafka_MetadataRequest(
+ rkb, &topics, NULL, "application requested",
+ allow_auto_create_topics,
+ /* cgrp_update:
+ * Only update consumer group state
+ * on response if this lists all
+ * topics in the cluster, since a
+ * partial request may make it seem
+ * like some subscribed topics are missing. */
+ all_topics ? rd_true : rd_false,
+ -1 /* same subscription version */,
+ rd_false /* force_racks */, rko);
+
+ rd_list_destroy(&topics);
+ rd_kafka_broker_destroy(rkb);
- rd_list_destroy(&topics);
- rd_kafka_broker_destroy(rkb);
+ /* Wait for reply (or timeout) */
+ rko = rd_kafka_q_pop(rkq, rd_timeout_remains_us(ts_end), 0);
- /* Wait for reply (or timeout) */
- rko = rd_kafka_q_pop(rkq, rd_timeout_remains_us(ts_end), 0);
+ rd_kafka_q_destroy_owner(rkq);
- rd_kafka_q_destroy_owner(rkq);
+ /* Timeout */
+ if (!rko)
+ return RD_KAFKA_RESP_ERR__TIMED_OUT;
- /* Timeout */
- if (!rko)
- return RD_KAFKA_RESP_ERR__TIMED_OUT;
+ /* Error */
+ err = rko->rko_err;
+ if (err) {
+ rd_kafka_op_destroy(rko);
+ if (err != RD_KAFKA_RESP_ERR__DESTROY_BROKER)
+ return err;
+ }
- /* Error */
- if (rko->rko_err) {
- rd_kafka_resp_err_t err = rko->rko_err;
- rd_kafka_op_destroy(rko);
- return err;
- }
+ /* In case selected broker was decommissioned,
+ * try again with a different broker. */
+ } while (err == RD_KAFKA_RESP_ERR__DESTROY_BROKER);
/* Reply: pass metadata pointer to application who now owns it*/
rd_kafka_assert(rk, rko->rko_u.metadata.md);
@@ -391,13 +407,7 @@ static void rd_kafka_parse_Metadata_update_topic(
* @locality rdkafka main thread
*/
rd_bool_t rd_kafka_has_reliable_leader_epochs(rd_kafka_broker_t *rkb) {
- int features;
- int16_t ApiVersion = 0;
-
- ApiVersion = rd_kafka_broker_ApiVersion_supported(
- rkb, RD_KAFKAP_Metadata, 0, 9, &features);
-
- return ApiVersion >= 9;
+ return rd_kafka_broker_ApiVersion_at_least(rkb, RD_KAFKAP_Metadata, 9);
}
/* Populates the topic partition to rack mapping for the the topic given by
@@ -460,6 +470,71 @@ rd_kafka_populate_metadata_topic_racks(rd_tmpabuf_t *tbuf,
}
}
+/**
+ * @brief Decommission brokers that are not in the metadata.
+ */
+static void rd_kafka_metadata_decommission_unavailable_brokers(
+ rd_kafka_t *rk,
+ rd_kafka_metadata_t *md,
+ rd_kafka_broker_t *rkb_current) {
+ rd_kafka_broker_t *rkb;
+ rd_bool_t has_learned_brokers = rd_false;
+ rd_list_t brokers_to_decommission;
+ int i;
+
+ rd_kafka_wrlock(rk);
+ TAILQ_FOREACH(rkb, &rk->rk_brokers, rkb_link) {
+ if (rkb->rkb_source == RD_KAFKA_LEARNED) {
+ has_learned_brokers = rd_true;
+ break;
+ }
+ }
+ if (!has_learned_brokers) {
+ rd_kafka_wrunlock(rk);
+ return;
+ }
+
+ rd_list_init(&brokers_to_decommission,
+ rd_atomic32_get(&rk->rk_broker_cnt), NULL);
+ TAILQ_FOREACH(rkb, &rk->rk_brokers, rkb_link) {
+ rd_bool_t purge_broker;
+
+ if (rkb->rkb_source == RD_KAFKA_LOGICAL)
+ continue;
+
+ purge_broker = rd_true;
+ if (rkb->rkb_source == RD_KAFKA_LEARNED) {
+ /* Don't purge the broker if it's available in
+ * metadata. */
+ for (i = 0; i < md->broker_cnt; i++) {
+ if (md->brokers[i].id == rkb->rkb_nodeid) {
+ purge_broker = rd_false;
+ break;
+ }
+ }
+ }
+
+ if (!purge_broker)
+ continue;
+
+ /* Don't try to decommission already decommissioning brokers
+ * otherwise they could be already destroyed when
+ * `rd_kafka_broker_decommission` is called below. */
+ if (rd_list_find(&rk->wait_decommissioned_brokers, rkb,
+ rd_list_cmp_ptr) != NULL)
+ continue;
+
+ rd_list_add(&brokers_to_decommission, rkb);
+ }
+ RD_LIST_FOREACH(rkb, &brokers_to_decommission, i) {
+ rd_kafka_broker_decommission(rk, rkb,
+ &rk->wait_decommissioned_thrds);
+ rd_list_add(&rk->wait_decommissioned_brokers, rkb);
+ }
+ rd_list_destroy(&brokers_to_decommission);
+ rd_kafka_wrunlock(rk);
+}
+
/* Internal implementation for parsing Metadata. */
static rd_kafka_resp_err_t
rd_kafka_parse_Metadata0(rd_kafka_broker_t *rkb,
@@ -484,12 +559,14 @@ rd_kafka_parse_Metadata0(rd_kafka_broker_t *rkb,
rd_bool_t cgrp_update = rd_false;
rd_bool_t has_reliable_leader_epochs =
rd_kafka_has_reliable_leader_epochs(rkb);
- int ApiVersion = rkbuf->rkbuf_reqhdr.ApiVersion;
- rd_kafkap_str_t cluster_id = RD_ZERO_INIT;
- int32_t controller_id = -1;
- rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR;
- int broker_changes = 0;
- int cache_changes = 0;
+ int ApiVersion = rkbuf->rkbuf_reqhdr.ApiVersion;
+ rd_kafkap_str_t cluster_id = RD_ZERO_INIT;
+ int32_t controller_id = -1;
+ rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR;
+ int broker_changes = 0;
+ int cache_changes = 0;
+ int cgrp_subscription_version = -1;
+ int16_t ErrorCode = 0;
/* If client rack is present, the metadata cache (topic or full) needs
* to contain the partition to rack map. */
@@ -504,6 +581,8 @@ rd_kafka_parse_Metadata0(rd_kafka_broker_t *rkb,
cgrp_update =
request->rkbuf_u.Metadata.cgrp_update && rk->rk_cgrp;
compute_racks |= request->rkbuf_u.Metadata.force_racks;
+ cgrp_subscription_version =
+ request->rkbuf_u.Metadata.cgrp_subscription_version;
}
/* If there's reason is NULL, set it to a human-readable string. */
@@ -788,8 +867,21 @@ rd_kafka_parse_Metadata0(rd_kafka_broker_t *rkb,
ClusterAuthorizedOperations;
}
+ if (ApiVersion >= 13) {
+ rd_kafka_buf_read_i16(rkbuf, &ErrorCode);
+ }
+
rd_kafka_buf_skip_tags(rkbuf);
+ if (ErrorCode) {
+ rd_rkb_dbg(rkb, METADATA, "METADATA",
+ "Metadata response: received top level "
+ "error code %" PRId16 ": %s",
+ ErrorCode, rd_kafka_err2str(ErrorCode));
+ err = ErrorCode;
+ goto err;
+ }
+
/* Entire Metadata response now parsed without errors:
* update our internal state according to the response. */
@@ -810,6 +902,8 @@ rd_kafka_parse_Metadata0(rd_kafka_broker_t *rkb,
&md->brokers[i], NULL);
}
+ rd_kafka_metadata_decommission_unavailable_brokers(rk, md, rkb);
+
for (i = 0; i < md->topic_cnt; i++) {
/* Ignore topics in blacklist */
@@ -854,19 +948,15 @@ rd_kafka_parse_Metadata0(rd_kafka_broker_t *rkb,
* for all topics or cache entry
* already exists. */
rd_kafka_wrlock(rk);
- cache_changes +=
- rd_kafka_metadata_cache_topic_update(
- rk, &md->topics[i], &mdi->topics[i],
- rd_false /*propagate later*/,
- /* use has_client_rack rather than
- compute_racks. We need cached rack ids
- only in case we need to rejoin the group
- if they change and client.rack is set
- (KIP-881). */
- has_client_rack, mdi->brokers,
- md->broker_cnt,
- all_topics /*cache entry needs to exist
- *if all_topics*/);
+ cache_changes += rd_kafka_metadata_cache_topic_update(
+ rk, &md->topics[i], &mdi->topics[i],
+ rd_false /*propagate later*/,
+ /* use has_client_rack rather than
+ compute_racks. We need cached rack ids
+ only in case we need to rejoin the group
+ if they change and client.rack is set
+ (KIP-881). */
+ has_client_rack, rd_kafka_has_reliable_leader_epochs(rkb));
rd_kafka_wrunlock(rk);
}
@@ -980,29 +1070,11 @@ rd_kafka_parse_Metadata0(rd_kafka_broker_t *rkb,
}
if (all_topics) {
- /* All hints have been replaced by the corresponding entry.
- * Rest of hints can be removed as topics aren't present
- * in full metadata. */
- rd_kafka_metadata_cache_purge_all_hints(rkb->rkb_rk);
- if (rkb->rkb_rk->rk_full_metadata)
- rd_kafka_metadata_destroy(
- &rkb->rkb_rk->rk_full_metadata->metadata);
-
- /* use has_client_rack rather than compute_racks. We need cached
- * rack ids only in case we need to rejoin the group if they
- * change and client.rack is set (KIP-881). */
- if (has_client_rack)
- rkb->rkb_rk->rk_full_metadata =
- rd_kafka_metadata_copy_add_racks(mdi, tbuf.of);
- else
- rkb->rkb_rk->rk_full_metadata =
- rd_kafka_metadata_copy(mdi, tbuf.of);
-
rkb->rkb_rk->rk_ts_full_metadata = rkb->rkb_rk->rk_ts_metadata;
rd_rkb_dbg(rkb, METADATA, "METADATA",
- "Caching full metadata with "
- "%d broker(s) and %d topic(s): %s",
- md->broker_cnt, md->topic_cnt, reason);
+ "Cached full metadata with "
+ " %d topic(s): %s",
+ md->topic_cnt, reason);
}
/* Remove cache hints for the originally requested topics. */
if (requested_topics)
@@ -1031,10 +1103,17 @@ rd_kafka_parse_Metadata0(rd_kafka_broker_t *rkb,
* the effective subscription of available topics) as to not
* propagate non-included topics as non-existent. */
if (cgrp_update &&
- (requested_topics || requested_topic_ids || all_topics))
+ (all_topics ||
+ ((requested_topics || requested_topic_ids) &&
+ rd_kafka_cgrp_same_subscription_version(
+ rkb->rkb_rk->rk_cgrp, cgrp_subscription_version))))
rd_kafka_cgrp_metadata_update_check(rkb->rkb_rk->rk_cgrp,
rd_true /*do join*/);
+ if (rk->rk_type == RD_KAFKA_CONSUMER && rk->rk_cgrp &&
+ rk->rk_cgrp->rkcg_group_protocol == RD_KAFKA_GROUP_PROTOCOL_CLASSIC)
+ rd_interval_reset(&rk->rk_cgrp->rkcg_join_intvl);
+
/* Try to acquire a Producer ID from this broker if we
* don't have one. */
if (rd_kafka_is_idempotent(rkb->rkb_rk)) {
@@ -1159,19 +1238,11 @@ rd_kafka_metadata_topic_match(rd_kafka_t *rk,
rd_kafka_topic_partition_list_t *errored) {
int ti, i;
size_t cnt = 0;
- rd_kafka_metadata_internal_t *mdi;
- struct rd_kafka_metadata *metadata;
rd_kafka_topic_partition_list_t *unmatched;
+ rd_list_t cached_topics;
+ const char *topic;
rd_kafka_rdlock(rk);
- mdi = rk->rk_full_metadata;
- metadata = &mdi->metadata;
-
- if (!mdi) {
- rd_kafka_rdunlock(rk);
- return 0;
- }
-
/* To keep track of which patterns and topics in `match` that
* did not match any topic (or matched an errored topic), we
* create a set of all topics to match in `unmatched` and then
@@ -1182,8 +1253,15 @@ rd_kafka_metadata_topic_match(rd_kafka_t *rk,
/* For each topic in the cluster, scan through the match list
* to find matching topic. */
- for (ti = 0; ti < metadata->topic_cnt; ti++) {
- const char *topic = metadata->topics[ti].topic;
+ rd_list_init(&cached_topics, rk->rk_metadata_cache.rkmc_cnt, rd_free);
+ rd_kafka_metadata_cache_topics_to_list(rk, &cached_topics, rd_false);
+ RD_LIST_FOREACH(topic, &cached_topics, ti) {
+ const rd_kafka_metadata_topic_internal_t *mdti;
+ const rd_kafka_metadata_topic_t *mdt =
+ rd_kafka_metadata_cache_topic_get(rk, topic, &mdti,
+ rd_true /* valid */);
+ if (!mdt)
+ continue;
/* Ignore topics in blacklist */
if (rk->rk_conf.topic_blacklist &&
@@ -1201,18 +1279,16 @@ rd_kafka_metadata_topic_match(rd_kafka_t *rk,
unmatched, match->elems[i].topic,
RD_KAFKA_PARTITION_UA);
- if (metadata->topics[ti].err) {
+ if (mdt->err) {
rd_kafka_topic_partition_list_add(
errored, topic, RD_KAFKA_PARTITION_UA)
- ->err = metadata->topics[ti].err;
+ ->err = mdt->err;
continue; /* Skip errored topics */
}
- rd_list_add(tinfos,
- rd_kafka_topic_info_new_with_rack(
- topic,
- metadata->topics[ti].partition_cnt,
- mdi->topics[ti].partitions));
+ rd_list_add(tinfos, rd_kafka_topic_info_new_with_rack(
+ topic, mdt->partition_cnt,
+ mdti->partitions));
cnt++;
}
@@ -1230,6 +1306,7 @@ rd_kafka_metadata_topic_match(rd_kafka_t *rk,
}
rd_kafka_topic_partition_list_destroy(unmatched);
+ rd_list_destroy(&cached_topics);
return cnt;
}
@@ -1347,6 +1424,7 @@ rd_kafka_metadata_refresh_topics(rd_kafka_t *rk,
rd_bool_t force,
rd_bool_t allow_auto_create,
rd_bool_t cgrp_update,
+ int32_t cgrp_subscription_version,
const char *reason) {
rd_list_t q_topics;
int destroy_rkb = 0;
@@ -1365,8 +1443,7 @@ rd_kafka_metadata_refresh_topics(rd_kafka_t *rk,
* these topics so that they will be included in
* a future all known_topics query. */
rd_kafka_metadata_cache_hint(rk, topics, NULL,
- RD_KAFKA_RESP_ERR__NOENT,
- 0 /*dont replace*/);
+ RD_KAFKA_RESP_ERR__NOENT);
rd_kafka_wrunlock(rk);
rd_kafka_dbg(rk, METADATA, "METADATA",
@@ -1387,8 +1464,7 @@ rd_kafka_metadata_refresh_topics(rd_kafka_t *rk,
* out any topics that are already being requested.
* q_topics will contain remaining topics to query. */
rd_kafka_metadata_cache_hint(rk, topics, &q_topics,
- RD_KAFKA_RESP_ERR__WAIT_CACHE,
- rd_false /*dont replace*/);
+ RD_KAFKA_RESP_ERR__WAIT_CACHE);
rd_kafka_wrunlock(rk);
if (rd_list_cnt(&q_topics) == 0) {
@@ -1413,9 +1489,9 @@ rd_kafka_metadata_refresh_topics(rd_kafka_t *rk,
"Requesting metadata for %d/%d topics: %s",
rd_list_cnt(&q_topics), rd_list_cnt(topics), reason);
- rd_kafka_MetadataRequest(rkb, &q_topics, NULL, reason,
- allow_auto_create, cgrp_update,
- rd_false /* force_racks */, NULL);
+ rd_kafka_MetadataRequest(
+ rkb, &q_topics, NULL, reason, allow_auto_create, cgrp_update,
+ cgrp_subscription_version, rd_false /* force_racks */, NULL);
rd_list_destroy(&q_topics);
@@ -1464,7 +1540,7 @@ rd_kafka_metadata_refresh_known_topics(rd_kafka_t *rk,
else
err = rd_kafka_metadata_refresh_topics(
rk, rkb, &topics, force, allow_auto_create_topics,
- rd_false /*!cgrp_update*/, reason);
+ rd_false /*!cgrp_update*/, -1, reason);
rd_list_destroy(&topics);
@@ -1504,7 +1580,8 @@ rd_kafka_metadata_refresh_consumer_topics(rd_kafka_t *rk,
rkcg = rk->rk_cgrp;
rd_assert(rkcg != NULL);
- if (rkcg->rkcg_flags & RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION) {
+ if (rkcg->rkcg_group_protocol == RD_KAFKA_GROUP_PROTOCOL_CLASSIC &&
+ rkcg->rkcg_flags & RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION) {
/* If there is a wildcard subscription we need to request
* all topics in the cluster so that we can perform
* regexp matching. */
@@ -1530,7 +1607,8 @@ rd_kafka_metadata_refresh_consumer_topics(rd_kafka_t *rk,
else
err = rd_kafka_metadata_refresh_topics(
rk, rkb, &topics, rd_true /*force*/,
- allow_auto_create_topics, rd_true /*cgrp_update*/, reason);
+ allow_auto_create_topics, rd_true /*cgrp_update*/,
+ rd_atomic32_get(&rkcg->rkcg_subscription_version), reason);
rd_list_destroy(&topics);
@@ -1557,8 +1635,9 @@ rd_kafka_resp_err_t rd_kafka_metadata_refresh_brokers(rd_kafka_t *rk,
const char *reason) {
return rd_kafka_metadata_request(rk, rkb, NULL /*brokers only*/,
rd_false /*!allow auto create topics*/,
- rd_false /*no cgrp update */, reason,
- NULL);
+ rd_false /*no cgrp update */,
+ -1 /* same subscription version */,
+ reason, NULL);
}
@@ -1592,7 +1671,8 @@ rd_kafka_resp_err_t rd_kafka_metadata_refresh_all(rd_kafka_t *rk,
rd_list_init(&topics, 0, NULL); /* empty list = all topics */
rd_kafka_MetadataRequest(
rkb, &topics, NULL, reason, rd_false /*no auto create*/,
- rd_true /*cgrp update*/, rd_false /* force_rack */, NULL);
+ rd_true /*cgrp update*/, -1 /* same subscription version */,
+ rd_false /* force_rack */, NULL);
rd_list_destroy(&topics);
if (destroy_rkb)
@@ -1618,6 +1698,7 @@ rd_kafka_metadata_request(rd_kafka_t *rk,
const rd_list_t *topics,
rd_bool_t allow_auto_create_topics,
rd_bool_t cgrp_update,
+ int32_t cgrp_subscription_version,
const char *reason,
rd_kafka_op_t *rko) {
int destroy_rkb = 0;
@@ -1629,9 +1710,9 @@ rd_kafka_metadata_request(rd_kafka_t *rk,
destroy_rkb = 1;
}
- rd_kafka_MetadataRequest(rkb, topics, NULL, reason,
- allow_auto_create_topics, cgrp_update,
- rd_false /* force racks */, rko);
+ rd_kafka_MetadataRequest(
+ rkb, topics, NULL, reason, allow_auto_create_topics, cgrp_update,
+ cgrp_subscription_version, rd_false /* force racks */, rko);
if (destroy_rkb)
rd_kafka_broker_destroy(rkb);
@@ -1695,7 +1776,7 @@ static void rd_kafka_metadata_leader_query_tmr_cb(rd_kafka_timers_t *rkts,
rd_kafka_metadata_refresh_topics(
rk, NULL, &topics, rd_true /*force*/,
rk->rk_conf.allow_auto_create_topics,
- rd_false /*!cgrp_update*/, "partition leader query");
+ rd_false /*!cgrp_update*/, -1, "partition leader query");
/* Back off next query exponentially till we reach
* the retry backoff max ms */
@@ -1716,16 +1797,19 @@ static void rd_kafka_metadata_leader_query_tmr_cb(rd_kafka_timers_t *rkts,
* exponentially increased intervals until no topics are missing
* leaders.
*
+ * @param force If true, run the query immediately without waiting for the
+ * interval.
+ *
* @locks none
* @locality any
*/
-void rd_kafka_metadata_fast_leader_query(rd_kafka_t *rk) {
+void rd_kafka_metadata_fast_leader_query(rd_kafka_t *rk, rd_bool_t force) {
rd_ts_t next;
- /* Restart the timer if it will speed things up. */
+ /* Restart the timer if it will speed things up, or if forced. */
next = rd_kafka_timer_next(
&rk->rk_timers, &rk->rk_metadata_cache.rkmc_query_tmr, 1 /*lock*/);
- if (next == -1 /* not started */ ||
+ if (force || next == -1 /* not started */ ||
next >
(rd_ts_t)rk->rk_conf.metadata_refresh_fast_interval_ms * 1000) {
rd_kafka_dbg(rk, METADATA | RD_KAFKA_DBG_TOPIC, "FASTQUERY",
@@ -2067,7 +2151,8 @@ rd_kafka_metadata_update_op(rd_kafka_t *rk, rd_kafka_metadata_internal_t *mdi) {
.partitions[part]
.leader_epoch;
- if (current_leader_epoch >= mdpi->leader_epoch) {
+ if (mdpi->leader_epoch != -1 &&
+ current_leader_epoch > mdpi->leader_epoch) {
rd_kafka_broker_destroy(rkb);
rd_kafka_dbg(
rk, METADATA, "METADATAUPDATE",
diff --git a/src/rdkafka_metadata.h b/src/rdkafka_metadata.h
index 9486a0050a..7916dcea77 100644
--- a/src/rdkafka_metadata.h
+++ b/src/rdkafka_metadata.h
@@ -147,6 +147,7 @@ rd_kafka_metadata_refresh_topics(rd_kafka_t *rk,
rd_bool_t force,
rd_bool_t allow_auto_create,
rd_bool_t cgrp_update,
+ int32_t cgrp_subscription_version,
const char *reason);
rd_kafka_resp_err_t
rd_kafka_metadata_refresh_known_topics(rd_kafka_t *rk,
@@ -170,6 +171,7 @@ rd_kafka_metadata_request(rd_kafka_t *rk,
const rd_list_t *topics,
rd_bool_t allow_auto_create_topics,
rd_bool_t cgrp_update,
+ int32_t cgrp_subscription_version,
const char *reason,
rd_kafka_op_t *rko);
@@ -281,9 +283,7 @@ int rd_kafka_metadata_cache_topic_update(
const rd_kafka_metadata_topic_internal_t *mdit,
rd_bool_t propagate,
rd_bool_t include_metadata,
- rd_kafka_metadata_broker_internal_t *brokers,
- size_t broker_cnt,
- rd_bool_t only_existing);
+ rd_bool_t has_reliable_leader_epochs);
void rd_kafka_metadata_cache_propagate_changes(rd_kafka_t *rk);
struct rd_kafka_metadata_cache_entry *
rd_kafka_metadata_cache_find(rd_kafka_t *rk, const char *topic, int valid);
@@ -298,21 +298,23 @@ void rd_kafka_metadata_cache_purge_hints_by_id(rd_kafka_t *rk,
int rd_kafka_metadata_cache_hint(rd_kafka_t *rk,
const rd_list_t *topics,
rd_list_t *dst,
- rd_kafka_resp_err_t err,
- rd_bool_t replace);
+ rd_kafka_resp_err_t err);
int rd_kafka_metadata_cache_hint_rktparlist(
rd_kafka_t *rk,
const rd_kafka_topic_partition_list_t *rktparlist,
- rd_list_t *dst,
- int replace);
+ rd_list_t *dst);
-const rd_kafka_metadata_topic_t *
-rd_kafka_metadata_cache_topic_get(rd_kafka_t *rk, const char *topic, int valid);
+const rd_kafka_metadata_topic_t *rd_kafka_metadata_cache_topic_get(
+ rd_kafka_t *rk,
+ const char *topic,
+ const rd_kafka_metadata_topic_internal_t **mdtip,
+ int valid);
int rd_kafka_metadata_cache_topic_partition_get(
rd_kafka_t *rk,
const rd_kafka_metadata_topic_t **mtopicp,
const rd_kafka_metadata_partition_t **mpartp,
+ const rd_kafka_metadata_partition_internal_t **mdpip,
const char *topic,
int32_t partition,
int valid);
@@ -321,7 +323,7 @@ int rd_kafka_metadata_cache_topics_count_exists(rd_kafka_t *rk,
const rd_list_t *topics,
int *metadata_agep);
-void rd_kafka_metadata_fast_leader_query(rd_kafka_t *rk);
+void rd_kafka_metadata_fast_leader_query(rd_kafka_t *rk, rd_bool_t force);
void rd_kafka_metadata_cache_init(rd_kafka_t *rk);
void rd_kafka_metadata_cache_destroy(rd_kafka_t *rk);
@@ -329,7 +331,9 @@ void rd_kafka_metadata_cache_purge(rd_kafka_t *rk, rd_bool_t purge_observers);
int rd_kafka_metadata_cache_wait_change(rd_kafka_t *rk, int timeout_ms);
void rd_kafka_metadata_cache_dump(FILE *fp, rd_kafka_t *rk);
-int rd_kafka_metadata_cache_topics_to_list(rd_kafka_t *rk, rd_list_t *topics);
+int rd_kafka_metadata_cache_topics_to_list(rd_kafka_t *rk,
+ rd_list_t *topics,
+ rd_bool_t exclude_valid);
void rd_kafka_metadata_cache_wait_state_change_async(
rd_kafka_t *rk,
diff --git a/src/rdkafka_metadata_cache.c b/src/rdkafka_metadata_cache.c
index 982983a7e5..157a90b20e 100644
--- a/src/rdkafka_metadata_cache.c
+++ b/src/rdkafka_metadata_cache.c
@@ -252,24 +252,20 @@ int rd_kafka_metadata_partition_id_cmp(const void *_a, const void *_b) {
return RD_CMP(a->id, b->id);
}
-
/**
- * @brief Add (and replace) cache entry for topic.
+ * @brief Creates a new metadata cache entry.
*
- * This makes a copy of \p topic
+ * @param mdt Topic to insert in the cache entry.
+ * @param mdti Topic to insert in the cache entry (internal structure).
+ * @param include_racks Include partition racks.
*
- * @locks_required rd_kafka_wrlock()
+ * @return The new metadata cache entry, to free with `rd_free`.
*/
-static struct rd_kafka_metadata_cache_entry *rd_kafka_metadata_cache_insert(
- rd_kafka_t *rk,
+static struct rd_kafka_metadata_cache_entry *rd_kafka_metadata_cache_entry_new(
const rd_kafka_metadata_topic_t *mtopic,
const rd_kafka_metadata_topic_internal_t *metadata_internal_topic,
- rd_ts_t now,
- rd_ts_t ts_expires,
- rd_bool_t include_racks,
- rd_kafka_metadata_broker_internal_t *brokers_internal,
- size_t broker_cnt) {
- struct rd_kafka_metadata_cache_entry *rkmce, *old, *old_by_id = NULL;
+ rd_bool_t include_racks) {
+ struct rd_kafka_metadata_cache_entry *rkmce;
rd_tmpabuf_t tbuf;
int i;
@@ -363,6 +359,21 @@ static struct rd_kafka_metadata_cache_entry *rd_kafka_metadata_cache_insert(
rkmce->rkmce_mtopic.partitions[i].isrs = NULL;
rkmce->rkmce_mtopic.partitions[i].isr_cnt = 0;
}
+
+ return rkmce;
+}
+
+/**
+ * @brief Add (and replace) cache entry for topic.
+ *
+ * @locks_required rd_kafka_wrlock()
+ */
+static struct rd_kafka_metadata_cache_entry *
+rd_kafka_metadata_cache_insert(rd_kafka_t *rk,
+ struct rd_kafka_metadata_cache_entry *rkmce,
+ rd_ts_t now,
+ rd_ts_t ts_expires) {
+ struct rd_kafka_metadata_cache_entry *old, *old_by_id = NULL;
TAILQ_INSERT_TAIL(&rk->rk_metadata_cache.rkmc_expiry, rkmce,
rkmce_link);
rk->rk_metadata_cache.rkmc_cnt++;
@@ -380,9 +391,13 @@ static struct rd_kafka_metadata_cache_entry *rd_kafka_metadata_cache_insert(
old_by_id = RD_AVL_INSERT(&rk->rk_metadata_cache.rkmc_avl_by_id,
rkmce, rkmce_avlnode_by_id);
}
- if (old && !RD_KAFKA_UUID_IS_ZERO(
- old->rkmce_metadata_internal_topic.topic_id)) {
- /* If it had a topic id, remove it from the tree */
+ if (old &&
+ !RD_KAFKA_UUID_IS_ZERO(
+ old->rkmce_metadata_internal_topic.topic_id) &&
+ rd_kafka_Uuid_cmp(rkmce->rkmce_metadata_internal_topic.topic_id,
+ old->rkmce_metadata_internal_topic.topic_id) !=
+ 0) {
+ /* If it had a different topic id, remove it from the tree */
RD_AVL_REMOVE_ELM(&rk->rk_metadata_cache.rkmc_avl_by_id, old);
}
if (old) {
@@ -390,10 +405,8 @@ static struct rd_kafka_metadata_cache_entry *rd_kafka_metadata_cache_insert(
rd_kafka_metadata_cache_delete(rk, old, 0);
}
if (old_by_id && old_by_id != old) {
- /* If there was a different cache entry in this tree,
- * remove and free it. */
- RD_AVL_REMOVE_ELM(&rk->rk_metadata_cache.rkmc_avl, old_by_id);
- rd_kafka_metadata_cache_delete(rk, old_by_id, 0);
+ rd_dassert(
+ !*"Different cache entries for topic name and topic id");
}
/* Explicitly not freeing the tmpabuf since rkmce points to its
@@ -401,6 +414,27 @@ static struct rd_kafka_metadata_cache_entry *rd_kafka_metadata_cache_insert(
return rkmce;
}
+/**
+ * @brief Add (and replace) cache entry for topic.
+ *
+ * This makes a copy of \p mtopic and \p metadata_internal_topic ,
+ *
+ * @locks_required rd_kafka_wrlock()
+ */
+static struct rd_kafka_metadata_cache_entry *rd_kafka_metadata_cache_insert_new(
+ rd_kafka_t *rk,
+ const rd_kafka_metadata_topic_t *mtopic,
+ const rd_kafka_metadata_topic_internal_t *metadata_internal_topic,
+ rd_ts_t now,
+ rd_ts_t ts_expires,
+ rd_bool_t include_racks) {
+ /* Create entry */
+ struct rd_kafka_metadata_cache_entry *rkmce =
+ rd_kafka_metadata_cache_entry_new(mtopic, metadata_internal_topic,
+ include_racks);
+ /* Insert/replace entry */
+ return rd_kafka_metadata_cache_insert(rk, rkmce, now, ts_expires);
+}
/**
* @brief Purge the metadata cache
@@ -441,6 +475,142 @@ void rd_kafka_metadata_cache_expiry_start(rd_kafka_t *rk) {
rd_kafka_metadata_cache_evict_tmr_cb, rk);
}
+#define rd_kafka_metadata_cache_topic_update_replace_partition( \
+ current_partition, new_partition, current_partition_cnt, \
+ new_partition_cnt, partition) \
+ ((partition) < (current_partition_cnt) && \
+ (partition) >= (new_partition_cnt) \
+ ? rd_false \
+ : (partition) < (new_partition_cnt) && \
+ (partition) >= (current_partition_cnt) \
+ ? rd_true \
+ : (new_partition).leader_epoch == -1 || \
+ (new_partition).leader_epoch >= \
+ (current_partition.leader_epoch));
+
+
+static struct rd_kafka_metadata_cache_entry *
+rd_kafka_metadata_cache_topic_update_merge_partitions(
+ rd_kafka_t *rk,
+ struct rd_kafka_metadata_cache_entry *rkmce_current,
+ const rd_kafka_metadata_topic_t *mdt,
+ const rd_kafka_metadata_topic_internal_t *mdti,
+ rd_bool_t include_racks,
+ rd_bool_t has_reliable_epochs) {
+ rd_tmpabuf_t tbuf;
+ struct rd_kafka_metadata_cache_entry *rkmce;
+ size_t i, current_partition_cnt, new_partition_cnt, partition_cnt;
+
+ if (!has_reliable_epochs || !rkmce_current ||
+ /* Different topic ids */
+ rd_kafka_Uuid_cmp(
+ mdti->topic_id,
+ rkmce_current->rkmce_metadata_internal_topic.topic_id) != 0) {
+ return rd_kafka_metadata_cache_entry_new(mdt, mdti,
+ include_racks);
+ }
+
+ current_partition_cnt = rkmce_current->rkmce_mtopic.partition_cnt;
+ new_partition_cnt = mdt->partition_cnt;
+ partition_cnt = RD_MAX(current_partition_cnt, new_partition_cnt);
+
+ rd_tmpabuf_new(&tbuf, sizeof(*rkmce), rd_true /*assert on fail*/);
+ rd_tmpabuf_add_alloc(&tbuf, sizeof(*rkmce));
+ rd_tmpabuf_add_alloc(&tbuf, strlen(mdt->topic) + 1);
+ rd_tmpabuf_add_alloc(&tbuf, partition_cnt * sizeof(*mdt->partitions));
+ rd_tmpabuf_add_alloc(&tbuf, partition_cnt * sizeof(*mdti->partitions));
+
+ for (i = 0; include_racks && i < partition_cnt; i++) {
+ size_t j;
+ rd_kafka_metadata_partition_internal_t *partition_internal;
+ rd_bool_t replace_partition =
+ rd_kafka_metadata_cache_topic_update_replace_partition(
+ rkmce_current->rkmce_metadata_internal_topic
+ .partitions[i],
+ mdti->partitions[i], current_partition_cnt,
+ new_partition_cnt, i);
+
+ partition_internal =
+ replace_partition
+ ? &mdti->partitions[i]
+ : &rkmce_current->rkmce_metadata_internal_topic
+ .partitions[i];
+ rd_tmpabuf_add_alloc(&tbuf, partition_internal->racks_cnt *
+ sizeof(char *));
+ for (j = 0; j < partition_internal->racks_cnt; j++) {
+ rd_tmpabuf_add_alloc(
+ &tbuf, strlen(partition_internal->racks[j]) + 1);
+ }
+ }
+
+ rd_tmpabuf_finalize(&tbuf);
+
+ rkmce = rd_tmpabuf_alloc(&tbuf, sizeof(*rkmce));
+
+ rkmce->rkmce_mtopic = *mdt;
+
+ rkmce->rkmce_metadata_internal_topic = *mdti;
+
+ /* Copy topic name */
+ rkmce->rkmce_mtopic.topic = rd_tmpabuf_write_str(&tbuf, mdt->topic);
+
+ /* Allocate partition array */
+ rkmce->rkmce_mtopic.partitions =
+ rd_tmpabuf_alloc(&tbuf, partition_cnt * sizeof(*mdt->partitions));
+
+ /* Allocate partition array (internal) */
+ rkmce->rkmce_metadata_internal_topic.partitions =
+ rd_tmpabuf_alloc(&tbuf, partition_cnt * sizeof(*mdti->partitions));
+
+ for (i = 0; i < partition_cnt; i++) {
+ struct rd_kafka_metadata_partition *partition;
+ rd_kafka_metadata_partition_internal_t *partition_internal;
+
+ rd_bool_t replace_partition =
+ rd_kafka_metadata_cache_topic_update_replace_partition(
+ rkmce_current->rkmce_metadata_internal_topic
+ .partitions[i],
+ mdti->partitions[i], current_partition_cnt,
+ new_partition_cnt, i);
+
+ if (replace_partition) {
+ partition = &mdt->partitions[i];
+ partition_internal = &mdti->partitions[i];
+ } else {
+ partition = &rkmce_current->rkmce_mtopic.partitions[i];
+ partition_internal =
+ &rkmce_current->rkmce_metadata_internal_topic
+ .partitions[i];
+ }
+
+ rkmce->rkmce_mtopic.partitions[i] = *partition;
+ rkmce->rkmce_metadata_internal_topic.partitions[i] =
+ *partition_internal;
+
+ if (include_racks) {
+ size_t j;
+ rkmce->rkmce_metadata_internal_topic.partitions[i]
+ .racks = rd_tmpabuf_alloc(
+ &tbuf,
+ partition_internal->racks_cnt * sizeof(char *));
+ rkmce->rkmce_metadata_internal_topic.partitions[i]
+ .racks_cnt = partition_internal->racks_cnt;
+ for (j = 0; j < partition_internal->racks_cnt; j++) {
+ rkmce->rkmce_metadata_internal_topic
+ .partitions[i]
+ .racks[j] = rd_tmpabuf_write_str(
+ &tbuf, partition_internal->racks[j]);
+ }
+ } else {
+ rkmce->rkmce_metadata_internal_topic.partitions[i]
+ .racks = NULL;
+ rkmce->rkmce_metadata_internal_topic.partitions[i]
+ .racks_cnt = 0;
+ }
+ }
+ return rkmce;
+}
+
/**
* @brief Update the metadata cache for a single topic
* with the provided metadata.
@@ -457,8 +627,12 @@ void rd_kafka_metadata_cache_expiry_start(rd_kafka_t *rk) {
* For permanent errors (authorization failures), we keep
* the entry cached for metadata.max.age.ms.
*
- * @param only_existing Update only existing metadata cache entries,
- * either valid or hinted.
+ * @param mdt Topic to insert in the cache entry.
+ * @param mdti Topic to insert in the cache entry (internal structure).
+ * @param propagate Propagate metadata cache changes now.
+ * @param include_racks Include partition racks.
+ * @param has_reliable_leader_epochs Comes from a broker with reliable leader
+ * epochs.
*
* @return 1 on metadata change, 0 when no change was applied
*
@@ -470,47 +644,64 @@ void rd_kafka_metadata_cache_expiry_start(rd_kafka_t *rk) {
int rd_kafka_metadata_cache_topic_update(
rd_kafka_t *rk,
const rd_kafka_metadata_topic_t *mdt,
- const rd_kafka_metadata_topic_internal_t *mdit,
+ const rd_kafka_metadata_topic_internal_t *mdti,
rd_bool_t propagate,
rd_bool_t include_racks,
- rd_kafka_metadata_broker_internal_t *brokers,
- size_t broker_cnt,
- rd_bool_t only_existing) {
- struct rd_kafka_metadata_cache_entry *rkmce = NULL;
- rd_ts_t now = rd_clock();
+ rd_bool_t has_reliable_leader_epochs) {
+ rd_ts_t now = rd_clock();
rd_ts_t ts_expires = now + (rk->rk_conf.metadata_max_age_ms * 1000);
- int changed = 1;
- if (only_existing) {
- if (likely(mdt->topic != NULL)) {
- rkmce = rd_kafka_metadata_cache_find(rk, mdt->topic, 0);
- } else {
- rkmce = rd_kafka_metadata_cache_find_by_id(
- rk, mdit->topic_id, 1);
- }
- if (!rkmce)
- return 0;
- }
+ int changed = 0;
if (likely(mdt->topic != NULL)) {
- /* Cache unknown topics for a short while (100ms) to allow the
- * cgrp logic to find negative cache hits. */
- if (mdt->err == RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART)
+ struct rd_kafka_metadata_cache_entry *rkmce,
+ *rkmce_current = NULL;
+ rd_kafka_metadata_topic_internal_t mdti_copy = *mdti;
+ switch (mdt->err) {
+ case RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART:
+ /* Cache unknown topics for metadata.propagation.max.ms
+ * to allow the cgrp logic to find negative cache hits.
+ * and to avoid false reappearances of the topic
+ * after deletion. */
ts_expires = RD_MIN(ts_expires, now + (100 * 1000));
- if (!mdt->err ||
- mdt->err == RD_KAFKA_RESP_ERR_TOPIC_AUTHORIZATION_FAILED ||
- mdt->err == RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART)
- rd_kafka_metadata_cache_insert(
- rk, mdt, mdit, now, ts_expires, include_racks,
- brokers, broker_cnt);
- else
- changed = rd_kafka_metadata_cache_delete_by_name(
- rk, mdt->topic);
+ /* Continue */
+ case RD_KAFKA_RESP_ERR_NO_ERROR:
+ rkmce_current =
+ rd_kafka_metadata_cache_find(rk, mdt->topic, 1);
+ if (mdt->err ==
+ RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART &&
+ rkmce_current &&
+ RD_KAFKA_UUID_IS_ZERO(mdti->topic_id) &&
+ !RD_KAFKA_UUID_IS_ZERO(
+ rkmce_current->rkmce_metadata_internal_topic
+ .topic_id)) {
+ /* Keep the existing topic id to detect
+ * if the same id is received again
+ * as existing */
+ mdti_copy.topic_id =
+ rkmce_current->rkmce_metadata_internal_topic
+ .topic_id;
+ }
+
+ /* Continue */
+ case RD_KAFKA_RESP_ERR_TOPIC_AUTHORIZATION_FAILED:
+ rkmce =
+ rd_kafka_metadata_cache_topic_update_merge_partitions(
+ rk, rkmce_current, mdt, &mdti_copy,
+ include_racks, has_reliable_leader_epochs);
+ /* Insert/replace entry */
+ rd_kafka_metadata_cache_insert(rk, rkmce, now,
+ ts_expires);
+ changed = 1;
+ break;
+ default:
+ break;
+ }
} else {
/* Cache entry found but no topic name:
* delete it. */
changed = rd_kafka_metadata_cache_delete_by_topic_id(
- rk, mdit->topic_id);
+ rk, mdti->topic_id);
}
if (changed && propagate)
@@ -604,7 +795,6 @@ void rd_kafka_metadata_cache_purge_hints_by_id(rd_kafka_t *rk,
* @param dst rd_list_t(char *topicname)
* @param err is the error to set on hint cache entries,
* typically ERR__WAIT_CACHE.
- * @param replace replace existing valid entries
*
* @returns the number of topic hints inserted.
*
@@ -613,8 +803,7 @@ void rd_kafka_metadata_cache_purge_hints_by_id(rd_kafka_t *rk,
int rd_kafka_metadata_cache_hint(rd_kafka_t *rk,
const rd_list_t *topics,
rd_list_t *dst,
- rd_kafka_resp_err_t err,
- rd_bool_t replace) {
+ rd_kafka_resp_err_t err) {
const char *topic;
rd_ts_t now = rd_clock();
rd_ts_t ts_expires = now + (rk->rk_conf.socket_timeout_ms * 1000);
@@ -628,9 +817,8 @@ int rd_kafka_metadata_cache_hint(rd_kafka_t *rk,
RD_ZERO_INIT;
/*const*/ struct rd_kafka_metadata_cache_entry *rkmce;
- /* !replace: Dont overwrite valid entries */
- if (!replace && (rkmce = rd_kafka_metadata_cache_find(
- rk, topic, 0 /*any*/))) {
+ if ((rkmce =
+ rd_kafka_metadata_cache_find(rk, topic, 0 /*any*/))) {
if (RD_KAFKA_METADATA_CACHE_VALID(rkmce) ||
(dst && rkmce->rkmce_mtopic.err !=
RD_KAFKA_RESP_ERR__NOENT))
@@ -639,9 +827,9 @@ int rd_kafka_metadata_cache_hint(rd_kafka_t *rk,
/* FALLTHRU */
}
- rd_kafka_metadata_cache_insert(rk, &mtopic,
- &metadata_internal_topic, now,
- ts_expires, rd_false, NULL, 0);
+ rd_kafka_metadata_cache_insert_new(rk, &mtopic,
+ &metadata_internal_topic,
+ now, ts_expires, rd_false);
cnt++;
if (dst)
@@ -666,8 +854,7 @@ int rd_kafka_metadata_cache_hint(rd_kafka_t *rk,
int rd_kafka_metadata_cache_hint_rktparlist(
rd_kafka_t *rk,
const rd_kafka_topic_partition_list_t *rktparlist,
- rd_list_t *dst,
- int replace) {
+ rd_list_t *dst) {
rd_list_t topics;
int r;
@@ -675,8 +862,8 @@ int rd_kafka_metadata_cache_hint_rktparlist(
rd_kafka_topic_partition_list_get_topic_names(rktparlist, &topics,
0 /*dont include regex*/);
rd_kafka_wrlock(rk);
- r = rd_kafka_metadata_cache_hint(
- rk, &topics, dst, RD_KAFKA_RESP_ERR__WAIT_CACHE, replace);
+ r = rd_kafka_metadata_cache_hint(rk, &topics, dst,
+ RD_KAFKA_RESP_ERR__WAIT_CACHE);
rd_kafka_wrunlock(rk);
rd_list_destroy(&topics);
@@ -812,20 +999,27 @@ void rd_kafka_metadata_cache_propagate_changes(rd_kafka_t *rk) {
}
/**
+ * @param mdtip If non NULL, it's set to a pointer to internal topic metadata,
+ * or to NULL if not found in cache.
* @returns the shared metadata for a topic, or NULL if not found in
* cache.
*
* @locks rd_kafka_*lock()
*/
-const rd_kafka_metadata_topic_t *
-rd_kafka_metadata_cache_topic_get(rd_kafka_t *rk,
- const char *topic,
- int valid) {
+const rd_kafka_metadata_topic_t *rd_kafka_metadata_cache_topic_get(
+ rd_kafka_t *rk,
+ const char *topic,
+ const rd_kafka_metadata_topic_internal_t **mdtip,
+ int valid) {
struct rd_kafka_metadata_cache_entry *rkmce;
- if (!(rkmce = rd_kafka_metadata_cache_find(rk, topic, valid)))
+ if (!(rkmce = rd_kafka_metadata_cache_find(rk, topic, valid))) {
+ if (mdtip)
+ *mdtip = NULL;
return NULL;
-
+ }
+ if (mdtip)
+ *mdtip = &rkmce->rkmce_metadata_internal_topic;
return &rkmce->rkmce_mtopic;
}
@@ -839,6 +1033,7 @@ rd_kafka_metadata_cache_topic_get(rd_kafka_t *rk,
*
* @param mtopicp: pointer to topic metadata
* @param mpartp: pointer to partition metadata
+ * @param mdpip: pointer to internal partition metadata
* @param valid: only return valid entries (no hints)
*
* @returns -1 if topic was not found in cache, 0 if topic was found
@@ -850,18 +1045,22 @@ int rd_kafka_metadata_cache_topic_partition_get(
rd_kafka_t *rk,
const rd_kafka_metadata_topic_t **mtopicp,
const rd_kafka_metadata_partition_t **mpartp,
+ const rd_kafka_metadata_partition_internal_t **mdpip,
const char *topic,
int32_t partition,
int valid) {
const rd_kafka_metadata_topic_t *mtopic;
+ const rd_kafka_metadata_topic_internal_t *mdti;
const rd_kafka_metadata_partition_t *mpart;
rd_kafka_metadata_partition_t skel = {.id = partition};
*mtopicp = NULL;
*mpartp = NULL;
+ *mdpip = NULL;
- if (!(mtopic = rd_kafka_metadata_cache_topic_get(rk, topic, valid)))
+ if (!(mtopic =
+ rd_kafka_metadata_cache_topic_get(rk, topic, &mdti, valid)))
return -1;
*mtopicp = mtopic;
@@ -878,6 +1077,8 @@ int rd_kafka_metadata_cache_topic_partition_get(
return 0;
*mpartp = mpart;
+ if (mdpip)
+ *mdpip = &mdti->partitions[mpart->id];
return 1;
}
@@ -924,17 +1125,21 @@ int rd_kafka_metadata_cache_topics_count_exists(rd_kafka_t *rk,
*
* Element type is (char *topic_name).
*
+ * @param exclude_valid Exclude topics that have up to date metadata info.
+ *
* @returns the number of elements added to \p topics
*
* @locks_required rd_kafka_*lock()
*/
-int rd_kafka_metadata_cache_topics_to_list(rd_kafka_t *rk, rd_list_t *topics) {
+int rd_kafka_metadata_cache_topics_to_list(rd_kafka_t *rk,
+ rd_list_t *topics,
+ rd_bool_t exclude_valid) {
const struct rd_kafka_metadata_cache_entry *rkmce;
int precnt = rd_list_cnt(topics);
TAILQ_FOREACH(rkmce, &rk->rk_metadata_cache.rkmc_expiry, rkmce_link) {
/* Ignore topics that have up to date metadata info */
- if (RD_KAFKA_METADATA_CACHE_VALID(rkmce))
+ if (exclude_valid && RD_KAFKA_METADATA_CACHE_VALID(rkmce))
continue;
if (rd_list_find(topics, rkmce->rkmce_mtopic.topic,
diff --git a/src/rdkafka_mock.c b/src/rdkafka_mock.c
index 48e1b03947..c8ca39e839 100644
--- a/src/rdkafka_mock.c
+++ b/src/rdkafka_mock.c
@@ -39,6 +39,8 @@
#include "rdkafka_mock_int.h"
#include "rdkafka_transport_int.h"
#include "rdkafka_mock.h"
+#include "rdunittest.h"
+
#include
typedef struct rd_kafka_mock_request_s rd_kafka_mock_request_t;
@@ -47,6 +49,8 @@ static void rd_kafka_mock_cluster_destroy0(rd_kafka_mock_cluster_t *mcluster);
static rd_kafka_mock_request_t *
rd_kafka_mock_request_new(int32_t id, int16_t api_key, int64_t timestamp_us);
static void rd_kafka_mock_request_free(void *element);
+static void rd_kafka_mock_coord_remove(rd_kafka_mock_cluster_t *mcluster,
+ int32_t broker_id);
static rd_kafka_mock_broker_t *
rd_kafka_mock_broker_find(const rd_kafka_mock_cluster_t *mcluster,
@@ -435,16 +439,22 @@ rd_kafka_mock_partition_assign_replicas(rd_kafka_mock_partition_t *mpart,
int replica_cnt = RD_MIN(replication_factor, mcluster->broker_cnt);
rd_kafka_mock_broker_t *mrkb;
int i = 0;
- int first_replica =
- (mpart->id * replication_factor) % mcluster->broker_cnt;
+ int first_replica;
int skipped = 0;
if (mpart->replicas)
rd_free(mpart->replicas);
- mpart->replicas = rd_calloc(replica_cnt, sizeof(*mpart->replicas));
+ mpart->replicas = replica_cnt
+ ? rd_calloc(replica_cnt, sizeof(*mpart->replicas))
+ : NULL;
mpart->replica_cnt = replica_cnt;
+ if (replica_cnt == 0) {
+ rd_kafka_mock_partition_set_leader0(mpart, NULL);
+ return;
+ }
+ first_replica = (mpart->id * replication_factor) % mcluster->broker_cnt;
/* Use a predictable, determininistic order on a per-topic basis.
*
@@ -918,6 +928,23 @@ static void rd_kafka_mock_cluster_io_add(rd_kafka_mock_cluster_t *mcluster,
mcluster->fd_cnt++;
}
+/**
+ * @brief Reassign partition replicas to broker, after deleting or
+ * adding a new one.
+ */
+static void
+rd_kafka_mock_cluster_reassign_partitions(rd_kafka_mock_cluster_t *mcluster) {
+ rd_kafka_mock_topic_t *mtopic;
+ TAILQ_FOREACH(mtopic, &mcluster->topics, link) {
+ int i;
+ for (i = 0; i < mtopic->partition_cnt; i++) {
+ rd_kafka_mock_partition_t *mpart =
+ &mtopic->partitions[i];
+ rd_kafka_mock_partition_assign_replicas(
+ mpart, mpart->replica_cnt);
+ }
+ }
+}
static void rd_kafka_mock_connection_close(rd_kafka_mock_connection_t *mconn,
const char *reason) {
@@ -1335,8 +1362,14 @@ static void rd_kafka_mock_connection_io(rd_kafka_mock_cluster_t *mcluster,
}
}
- if (events & (POLLERR | POLLHUP)) {
- rd_kafka_mock_connection_close(mconn, "Disconnected");
+ if (events & POLLERR) {
+ rd_kafka_mock_connection_close(mconn,
+ "Disconnected: "
+ "Error condition");
+ return;
+ }
+ if (events & POLLHUP) {
+ rd_kafka_mock_connection_close(mconn, "Disconnected: Hang up");
return;
}
@@ -1564,6 +1597,8 @@ static void rd_kafka_mock_broker_destroy(rd_kafka_mock_broker_t *mrkb) {
if (mrkb->rack)
rd_free(mrkb->rack);
+ rd_kafka_mock_coord_remove(mrkb->cluster, mrkb->id);
+
TAILQ_REMOVE(&mrkb->cluster->brokers, mrkb, link);
mrkb->cluster->broker_cnt--;
@@ -1571,6 +1606,30 @@ static void rd_kafka_mock_broker_destroy(rd_kafka_mock_broker_t *mrkb) {
}
+rd_kafka_resp_err_t
+rd_kafka_mock_broker_decommission(rd_kafka_mock_cluster_t *mcluster,
+ int32_t broker_id) {
+ rd_kafka_op_t *rko = rd_kafka_op_new(RD_KAFKA_OP_MOCK);
+
+ rko->rko_u.mock.broker_id = broker_id;
+ rko->rko_u.mock.cmd = RD_KAFKA_MOCK_CMD_BROKER_DECOMMISSION;
+
+ return rd_kafka_op_err_destroy(
+ rd_kafka_op_req(mcluster->ops, rko, RD_POLL_INFINITE));
+}
+
+rd_kafka_resp_err_t rd_kafka_mock_broker_add(rd_kafka_mock_cluster_t *mcluster,
+ int32_t broker_id) {
+ rd_kafka_op_t *rko = rd_kafka_op_new(RD_KAFKA_OP_MOCK);
+
+ rko->rko_u.mock.broker_id = broker_id;
+ rko->rko_u.mock.cmd = RD_KAFKA_MOCK_CMD_BROKER_ADD;
+
+ return rd_kafka_op_err_destroy(
+ rd_kafka_op_req(mcluster->ops, rko, RD_POLL_INFINITE));
+}
+
+
/**
* @brief Starts listening on the mock broker socket.
*
@@ -1664,16 +1723,28 @@ static int rd_kafka_mock_broker_new_listener(rd_kafka_mock_cluster_t *mcluster,
static rd_kafka_mock_broker_t *
-rd_kafka_mock_broker_new(rd_kafka_mock_cluster_t *mcluster, int32_t broker_id) {
+rd_kafka_mock_broker_new(rd_kafka_mock_cluster_t *mcluster,
+ int32_t broker_id,
+ rd_kafka_resp_err_t *err) {
rd_kafka_mock_broker_t *mrkb;
rd_socket_t listen_s;
struct sockaddr_in sin = {
.sin_family = AF_INET,
.sin_addr = {.s_addr = htonl(INADDR_LOOPBACK)}};
+ if (rd_kafka_mock_broker_find(mcluster, broker_id)) {
+ if (err)
+ *err = RD_KAFKA_RESP_ERR__INVALID_ARG;
+ /* A broker with this id already exists. */
+ return NULL;
+ }
+
listen_s = rd_kafka_mock_broker_new_listener(mcluster, &sin);
- if (listen_s == -1)
+ if (listen_s == -1) {
+ if (err)
+ *err = RD_KAFKA_RESP_ERR__TRANSPORT;
return NULL;
+ }
/*
* Create mock broker object
@@ -1698,6 +1769,8 @@ rd_kafka_mock_broker_new(rd_kafka_mock_cluster_t *mcluster, int32_t broker_id) {
if (rd_kafka_mock_broker_start_listener(mrkb) == -1) {
rd_kafka_mock_broker_destroy(mrkb);
+ if (err)
+ *err = RD_KAFKA_RESP_ERR__TRANSPORT;
return NULL;
}
@@ -1806,6 +1879,20 @@ rd_kafka_mock_coord_set(rd_kafka_mock_cluster_t *mcluster,
return mcoord;
}
+/**
+ * @brief Remove coordinator by broker id.
+ */
+void rd_kafka_mock_coord_remove(rd_kafka_mock_cluster_t *mcluster,
+ int32_t broker_id) {
+ rd_kafka_mock_coord_t *mcoord, *tmp;
+
+ TAILQ_FOREACH_SAFE(mcoord, &mcluster->coords, link, tmp) {
+ if (mcoord->broker_id == broker_id) {
+ rd_kafka_mock_coord_destroy(mcluster, mcoord);
+ }
+ }
+}
+
/**
* @brief Remove and return the next error, or RD_KAFKA_RESP_ERR_NO_ERROR
@@ -2205,6 +2292,31 @@ rd_kafka_mock_broker_set_rack(rd_kafka_mock_cluster_t *mcluster,
rd_kafka_op_req(mcluster->ops, rko, RD_POLL_INFINITE));
}
+void rd_kafka_mock_broker_set_host_port(rd_kafka_mock_cluster_t *cluster,
+ int32_t broker_id,
+ const char *host,
+ int port) {
+ rd_kafka_mock_broker_t *mrkb;
+
+ mtx_lock(&cluster->lock);
+ TAILQ_FOREACH(mrkb, &cluster->brokers, link) {
+ if (mrkb->id == broker_id) {
+ rd_kafka_dbg(
+ cluster->rk, MOCK, "MOCK",
+ "Broker %" PRId32
+ ": Setting advertised listener from %s:%d to %s:%d",
+ broker_id, mrkb->advertised_listener, mrkb->port,
+ host, port);
+ rd_snprintf(mrkb->advertised_listener,
+ sizeof(mrkb->advertised_listener), "%s",
+ host);
+ mrkb->port = port;
+ break;
+ }
+ }
+ mtx_unlock(&cluster->lock);
+}
+
rd_kafka_resp_err_t
rd_kafka_mock_coordinator_set(rd_kafka_mock_cluster_t *mcluster,
const char *key_type,
@@ -2326,6 +2438,11 @@ rd_kafka_mock_broker_cmd(rd_kafka_mock_cluster_t *mcluster,
mrkb->rack = NULL;
break;
+ case RD_KAFKA_MOCK_CMD_BROKER_DECOMMISSION:
+ rd_kafka_mock_broker_destroy(mrkb);
+ rd_kafka_mock_cluster_reassign_partitions(mcluster);
+ break;
+
default:
RD_BUG("Unhandled mock cmd %d", rko->rko_u.mock.cmd);
break;
@@ -2380,6 +2497,7 @@ rd_kafka_mock_cluster_cmd(rd_kafka_mock_cluster_t *mcluster,
rd_kafka_mock_partition_t *mpart;
rd_kafka_mock_broker_t *mrkb;
size_t i;
+ rd_kafka_resp_err_t err;
switch (rko->rko_u.mock.cmd) {
case RD_KAFKA_MOCK_CMD_TOPIC_CREATE:
@@ -2490,8 +2608,16 @@ rd_kafka_mock_cluster_cmd(rd_kafka_mock_cluster_t *mcluster,
case RD_KAFKA_MOCK_CMD_BROKER_SET_UPDOWN:
case RD_KAFKA_MOCK_CMD_BROKER_SET_RTT:
case RD_KAFKA_MOCK_CMD_BROKER_SET_RACK:
+ case RD_KAFKA_MOCK_CMD_BROKER_DECOMMISSION:
return rd_kafka_mock_brokers_cmd(mcluster, rko);
+ case RD_KAFKA_MOCK_CMD_BROKER_ADD:
+ if (!rd_kafka_mock_broker_new(mcluster,
+ rko->rko_u.mock.broker_id, &err))
+ return err;
+
+ rd_kafka_mock_cluster_reassign_partitions(mcluster);
+ break;
case RD_KAFKA_MOCK_CMD_COORD_SET:
if (!rd_kafka_mock_coord_set(mcluster, rko->rko_u.mock.name,
rko->rko_u.mock.str,
@@ -2534,6 +2660,14 @@ rd_kafka_mock_cluster_cmd(rd_kafka_mock_cluster_t *mcluster,
return RD_KAFKA_RESP_ERR_NO_ERROR;
}
+void rd_kafka_mock_group_initial_rebalance_delay_ms(
+ rd_kafka_mock_cluster_t *mcluster,
+ int32_t delay_ms) {
+ mtx_lock(&mcluster->lock);
+ mcluster->defaults.group_initial_rebalance_delay_ms = delay_ms;
+ mtx_unlock(&mcluster->lock);
+}
+
static rd_kafka_op_res_t
rd_kafka_mock_cluster_op_serve(rd_kafka_t *rk,
@@ -2570,7 +2704,8 @@ rd_kafka_mock_cluster_op_serve(rd_kafka_t *rk,
static void rd_kafka_mock_cluster_destroy0(rd_kafka_mock_cluster_t *mcluster) {
rd_kafka_mock_topic_t *mtopic;
rd_kafka_mock_broker_t *mrkb;
- rd_kafka_mock_cgrp_t *mcgrp;
+ rd_kafka_mock_cgrp_classic_t *mcgrp_classic;
+ rd_kafka_mock_cgrp_consumer_t *mcgrp_consumer;
rd_kafka_mock_coord_t *mcoord;
rd_kafka_mock_error_stack_t *errstack;
thrd_t dummy_rkb_thread;
@@ -2583,8 +2718,11 @@ static void rd_kafka_mock_cluster_destroy0(rd_kafka_mock_cluster_t *mcluster) {
while ((mrkb = TAILQ_FIRST(&mcluster->brokers)))
rd_kafka_mock_broker_destroy(mrkb);
- while ((mcgrp = TAILQ_FIRST(&mcluster->cgrps)))
- rd_kafka_mock_cgrp_destroy(mcgrp);
+ while ((mcgrp_classic = TAILQ_FIRST(&mcluster->cgrps_classic)))
+ rd_kafka_mock_cgrp_classic_destroy(mcgrp_classic);
+
+ while ((mcgrp_consumer = TAILQ_FIRST(&mcluster->cgrps_consumer)))
+ rd_kafka_mock_cgrp_consumer_destroy(mcgrp_consumer);
while ((mcoord = TAILQ_FIRST(&mcluster->coords)))
rd_kafka_mock_coord_destroy(mcluster, mcoord);
@@ -2598,16 +2736,16 @@ static void rd_kafka_mock_cluster_destroy0(rd_kafka_mock_cluster_t *mcluster) {
rd_list_destroy(&mcluster->request_list);
+ dummy_rkb_thread = mcluster->dummy_rkb->rkb_thread;
+
/*
- * Destroy dummy broker
+ * Destroy dummy broker.
+ * WARNING: This is last time we can read
+ * from dummy_rkb in this thread!
*/
rd_kafka_q_enq(mcluster->dummy_rkb->rkb_ops,
rd_kafka_op_new(RD_KAFKA_OP_TERMINATE));
- dummy_rkb_thread = mcluster->dummy_rkb->rkb_thread;
-
- rd_kafka_broker_destroy(mcluster->dummy_rkb);
-
if (thrd_join(dummy_rkb_thread, &ret) != thrd_success)
rd_assert(!*"failed to join mock dummy broker thread");
@@ -2680,7 +2818,7 @@ rd_kafka_mock_cluster_t *rd_kafka_mock_cluster_new(rd_kafka_t *rk,
TAILQ_INIT(&mcluster->brokers);
for (i = 1; i <= broker_cnt; i++) {
- if (!(mrkb = rd_kafka_mock_broker_new(mcluster, i))) {
+ if (!(mrkb = rd_kafka_mock_broker_new(mcluster, i, NULL))) {
rd_kafka_mock_cluster_destroy(mcluster);
return NULL;
}
@@ -2694,9 +2832,14 @@ rd_kafka_mock_cluster_t *rd_kafka_mock_cluster_new(rd_kafka_t *rk,
TAILQ_INIT(&mcluster->topics);
mcluster->defaults.partition_cnt = 4;
mcluster->defaults.replication_factor = RD_MIN(3, broker_cnt);
- mcluster->track_requests = rd_false;
+ mcluster->defaults.group_initial_rebalance_delay_ms = 3000;
+ mcluster->track_requests = rd_false;
+ mcluster->defaults.group_consumer_session_timeout_ms = 30000;
+ mcluster->defaults.group_consumer_heartbeat_interval_ms = 3000;
- TAILQ_INIT(&mcluster->cgrps);
+ TAILQ_INIT(&mcluster->cgrps_classic);
+
+ TAILQ_INIT(&mcluster->cgrps_consumer);
TAILQ_INIT(&mcluster->coords);
@@ -2874,3 +3017,645 @@ int16_t rd_kafka_mock_request_api_key(rd_kafka_mock_request_t *mreq) {
rd_ts_t rd_kafka_mock_request_timestamp(rd_kafka_mock_request_t *mreq) {
return mreq->timestamp;
}
+
+/* Unit tests */
+
+/**
+ * @brief Create a topic-partition list with vararg arguments.
+ *
+ * @param cnt Number of topic-partitions.
+ * @param ...vararg is a tuple of:
+ * const char *topic_name
+ * int32_t partition
+ *
+ * @remark The returned pointer ownership is transferred to the caller.
+ */
+static rd_kafka_topic_partition_list_t *ut_topic_partitions(int cnt, ...) {
+ va_list ap;
+ const char *topic_name;
+ int i = 0;
+
+ rd_kafka_topic_partition_list_t *rktparlist =
+ rd_kafka_topic_partition_list_new(cnt);
+ va_start(ap, cnt);
+ while (i < cnt) {
+ topic_name = va_arg(ap, const char *);
+ int32_t partition = va_arg(ap, int32_t);
+
+ rd_kafka_topic_partition_list_add(rktparlist, topic_name,
+ partition);
+ i++;
+ }
+ va_end(ap);
+
+ return rktparlist;
+}
+
+/**
+ * @brief Assert \p expected partition list is equal to \p actual.
+ *
+ * @param expected Expected partition list.
+ * @param actual Actual partition list.
+ * @return Comparation result.
+ */
+static int ut_assert_topic_partitions(rd_kafka_topic_partition_list_t *expected,
+ rd_kafka_topic_partition_list_t *actual) {
+ rd_bool_t equal;
+ char expected_str[256] = "";
+ char actual_str[256] = "";
+
+ if (expected)
+ RD_UT_ASSERT(actual, "list should be not-NULL, but it's NULL");
+ else
+ RD_UT_ASSERT(!actual, "list should be NULL, but it's not-NULL");
+
+
+ if (!expected)
+ return 0;
+
+ equal = !rd_kafka_topic_partition_list_cmp(
+ actual, expected, rd_kafka_topic_partition_cmp);
+
+ if (!equal) {
+ rd_kafka_topic_partition_list_str(expected, expected_str,
+ sizeof(expected_str),
+ RD_KAFKA_FMT_F_NO_ERR);
+ rd_kafka_topic_partition_list_str(actual, actual_str,
+ sizeof(actual_str),
+ RD_KAFKA_FMT_F_NO_ERR);
+ }
+
+ RD_UT_ASSERT(equal, "list should be equal. Expected: %s, got: %s",
+ expected_str, actual_str);
+ return 0;
+}
+
+/**
+ * @struct Fixture used for testing next assignment calculation.
+ */
+struct cgrp_consumer_member_next_assignment_fixture {
+ /** Current member epoch (after calling next assignment). */
+ int32_t current_member_epoch;
+ /** Current consumer assignment, if changed. */
+ rd_kafka_topic_partition_list_t *current_assignment;
+ /** Returned assignment, if expected. */
+ rd_kafka_topic_partition_list_t *returned_assignment;
+ /** Target assignment, if changed. */
+ rd_kafka_topic_partition_list_t *target_assignment;
+ /** Should simulate a disconnection and reconnection. */
+ rd_bool_t reconnected;
+ /** Should simulate a session time out. */
+ rd_bool_t session_timed_out;
+ /** Comment to log. */
+ const char *comment;
+};
+
+/**
+ * @brief Test next assignment calculation using passed \p fixtures.
+ * using a new cluster with a topic with name \p topic and
+ * \p partitions partitions.
+ *
+ * @param topic Topic name to create.
+ * @param partitions Topic partition.
+ * @param fixtures Array of fixtures for this test.
+ * @param fixtures_cnt Number of elements in \p fixtures.
+ * @return Number of occurred errors.
+ */
+static int ut_cgrp_consumer_member_next_assignment0(
+ const char *topic,
+ int partitions,
+ struct cgrp_consumer_member_next_assignment_fixture *fixtures,
+ size_t fixtures_cnt) {
+ int failures = 0;
+ int32_t current_member_epoch = 0;
+ size_t i;
+ rd_kafka_t *rk;
+ rd_kafka_mock_cluster_t *mcluster;
+ static rd_kafka_mock_topic_t *mtopic;
+ rd_kafka_mock_cgrp_consumer_t *mcgrp;
+ rd_kafka_mock_cgrp_consumer_member_t *member;
+ char errstr[512];
+ rd_kafkap_str_t GroupId = {.str = "group", .len = 5};
+ rd_kafkap_str_t MemberId = {.str = "A", .len = 1};
+ rd_kafkap_str_t InstanceId = {.len = -1};
+ rd_kafkap_str_t SubscribedTopic = {.str = topic, .len = strlen(topic)};
+ rd_kafkap_str_t SubscribedTopicRegex = RD_KAFKAP_STR_INITIALIZER_EMPTY;
+ struct rd_kafka_mock_connection_s *conn =
+ (struct rd_kafka_mock_connection_s
+ *)1; /* fake connection instance */
+
+ rk = rd_kafka_new(RD_KAFKA_CONSUMER, NULL, errstr, sizeof(errstr));
+ mcluster = rd_kafka_mock_cluster_new(rk, 1);
+ mcgrp = rd_kafka_mock_cgrp_consumer_get(mcluster, &GroupId);
+ member = rd_kafka_mock_cgrp_consumer_member_add(
+ mcgrp, conn, &MemberId, &InstanceId, &SubscribedTopic, 1,
+ &SubscribedTopicRegex);
+ mtopic = rd_kafka_mock_topic_new(mcluster, topic, partitions, 1);
+
+ for (i = 0; i < fixtures_cnt; i++) {
+ int j;
+ rd_kafka_topic_partition_list_t *current_assignment,
+ *member_target_assignment, *next_assignment,
+ *returned_assignment;
+
+ RD_UT_SAY("test fixture %" PRIusz ": %s", i,
+ fixtures[i].comment);
+
+ if (fixtures[i].session_timed_out) {
+ rd_kafka_mock_cgrp_consumer_member_leave(mcgrp, member,
+ rd_false);
+ member = rd_kafka_mock_cgrp_consumer_member_add(
+ mcgrp, conn, &MemberId, &InstanceId,
+ &SubscribedTopic, 1, &SubscribedTopicRegex);
+ }
+
+ if (fixtures[i].reconnected) {
+ rd_kafka_mock_cgrps_connection_closed(mcluster, conn);
+ conn++;
+ member = rd_kafka_mock_cgrp_consumer_member_add(
+ mcgrp, conn, &MemberId, &InstanceId,
+ &SubscribedTopic, 1, &SubscribedTopicRegex);
+ }
+
+ member_target_assignment = fixtures[i].target_assignment;
+ if (member_target_assignment) {
+ rd_kafka_mock_cgrp_consumer_target_assignment_t
+ *target_assignment;
+
+ target_assignment =
+ rd_kafka_mock_cgrp_consumer_target_assignment_new(
+ (char **)&MemberId.str, 1,
+ &member_target_assignment);
+
+ rd_kafka_mock_cgrp_consumer_target_assignment(
+ mcluster, GroupId.str, target_assignment);
+ rd_kafka_mock_cgrp_consumer_target_assignment_destroy(
+ target_assignment);
+ rd_kafka_topic_partition_list_destroy(
+ member_target_assignment);
+ }
+
+ current_assignment = fixtures[i].current_assignment;
+ if (current_assignment) {
+ /* Set topic id */
+ for (j = 0; j < current_assignment->cnt; j++) {
+ rd_kafka_topic_partition_set_topic_id(
+ ¤t_assignment->elems[j], mtopic->id);
+ }
+ }
+
+ next_assignment =
+ rd_kafka_mock_cgrp_consumer_member_next_assignment(
+ member, current_assignment, ¤t_member_epoch);
+ RD_IF_FREE(current_assignment,
+ rd_kafka_topic_partition_list_destroy);
+ RD_UT_ASSERT(
+ current_member_epoch == fixtures[i].current_member_epoch,
+ "current member epoch after call. Expected: %" PRId32
+ ", got: %" PRId32,
+ fixtures[i].current_member_epoch, current_member_epoch);
+
+ returned_assignment = fixtures[i].returned_assignment;
+ failures += ut_assert_topic_partitions(returned_assignment,
+ next_assignment);
+
+ RD_IF_FREE(next_assignment,
+ rd_kafka_topic_partition_list_destroy);
+ RD_IF_FREE(returned_assignment,
+ rd_kafka_topic_partition_list_destroy);
+ }
+
+ rd_kafka_mock_cluster_destroy(mcluster);
+ rd_kafka_destroy(rk);
+ return failures;
+}
+
+/**
+ * @brief Test case where multiple revocations are acked.
+ * Only when they're acked member epoch is bumped
+ * and a new partition is returned to the member.
+ *
+ * @return Number of occurred errors.
+ */
+static int ut_cgrp_consumer_member_next_assignment1(void) {
+ RD_UT_SAY("Case 1: multiple revocations acked");
+
+ const char *topic = "topic";
+ struct cgrp_consumer_member_next_assignment_fixture fixtures[] = {
+ {
+ .comment = "Target+Returned assignment 0,1,2. Epoch 0 -> 3",
+ .current_member_epoch = 3,
+ .current_assignment = NULL,
+ .target_assignment =
+ ut_topic_partitions(3, topic, 0, topic, 1, topic, 2),
+ .returned_assignment =
+ ut_topic_partitions(3, topic, 0, topic, 1, topic, 2),
+ },
+ {
+ .comment = "Current assignment empty",
+ .current_member_epoch = 3,
+ .current_assignment = ut_topic_partitions(0),
+ .returned_assignment = NULL,
+ },
+ {
+ .comment = "Empty heartbeat",
+ .current_member_epoch = 3,
+ .current_assignment = NULL,
+ .returned_assignment = NULL,
+ },
+ {
+ .comment = "Current assignment 0",
+ .current_member_epoch = 3,
+ .current_assignment = ut_topic_partitions(1, topic, 0),
+ .returned_assignment = NULL,
+ },
+ {
+ .comment = "Empty heartbeat",
+ .current_member_epoch = 3,
+ .current_assignment = NULL,
+ .returned_assignment = NULL,
+ },
+ {
+ .comment = "Current assignment 0,1",
+ .current_member_epoch = 3,
+ .current_assignment =
+ ut_topic_partitions(2, topic, 0, topic, 1),
+ .returned_assignment = NULL,
+ },
+ {
+ .comment = "Empty heartbeat",
+ .current_member_epoch = 3,
+ .current_assignment = NULL,
+ .returned_assignment = NULL,
+ },
+ {
+ .comment = "Current assignment 0,1,2",
+ .current_member_epoch = 3,
+ .current_assignment =
+ ut_topic_partitions(3, topic, 0, topic, 1, topic, 2),
+ .returned_assignment = NULL,
+ },
+ {
+ .comment = "Target assignment 0,1,3. Returned assignment 0,1",
+ .current_member_epoch = 3,
+ .target_assignment =
+ ut_topic_partitions(3, topic, 0, topic, 1, topic, 3),
+ .current_assignment = NULL,
+ .returned_assignment =
+ ut_topic_partitions(2, topic, 0, topic, 1),
+ },
+ {
+ .comment = "Target assignment 0,3. Returned assignment 0",
+ .current_member_epoch = 3,
+ .target_assignment = ut_topic_partitions(2, topic, 0, topic, 3),
+ .current_assignment = NULL,
+ .returned_assignment = ut_topic_partitions(1, topic, 0),
+ },
+ {
+ .comment = "Empty heartbeat",
+ .current_member_epoch = 3,
+ .current_assignment = NULL,
+ .returned_assignment = NULL,
+ },
+ {
+ .comment = "Current assignment 0,1",
+ .current_member_epoch = 3,
+ .current_assignment =
+ ut_topic_partitions(2, topic, 0, topic, 1),
+ .returned_assignment = NULL,
+ },
+ {
+ .comment = "Empty heartbeat",
+ .current_member_epoch = 3,
+ .current_assignment = NULL,
+ .returned_assignment = NULL,
+ },
+ {
+ .comment = "Current assignment 0. Returned assignment 0,3. "
+ "Epoch 3 -> 5",
+ .current_member_epoch = 5,
+ .current_assignment = ut_topic_partitions(1, topic, 0),
+ .returned_assignment =
+ ut_topic_partitions(2, topic, 0, topic, 3),
+ },
+ {
+ .comment = "Empty heartbeat",
+ .current_member_epoch = 5,
+ .current_assignment = NULL,
+ .returned_assignment = NULL,
+ },
+ {
+ .comment = "Current assignment 0,3",
+ .current_member_epoch = 5,
+ .current_assignment =
+ ut_topic_partitions(2, topic, 0, topic, 3),
+ .returned_assignment = NULL,
+ },
+ };
+ return ut_cgrp_consumer_member_next_assignment0(
+ topic, 4, fixtures, RD_ARRAY_SIZE(fixtures));
+}
+
+/**
+ * @brief Test case where multiple revocations happen.
+ * Only the first revocation is acked and after that
+ * there's a reassignment and epoch bump.
+ *
+ * @return Number of occurred errors.
+ */
+static int ut_cgrp_consumer_member_next_assignment2(void) {
+ RD_UT_SAY(
+ "Case 2: reassignment of revoked partition, partial revocation "
+ "acknowledge");
+
+ const char *topic = "topic";
+ struct cgrp_consumer_member_next_assignment_fixture fixtures[] = {
+ {
+ .comment = "Target+Returned assignment 0,1,2. Epoch 0 -> 3",
+ .current_member_epoch = 3,
+ .current_assignment = NULL,
+ .target_assignment =
+ ut_topic_partitions(3, topic, 0, topic, 1, topic, 2),
+ .returned_assignment =
+ ut_topic_partitions(3, topic, 0, topic, 1, topic, 2),
+ },
+ {
+ .comment = "Current assignment 0,1,2",
+ .current_member_epoch = 3,
+ .current_assignment =
+ ut_topic_partitions(3, topic, 0, topic, 1, topic, 2),
+ .returned_assignment = NULL,
+ },
+ {
+ .comment = "Target assignment 0,1,3. Returned assignment 0,1",
+ .current_member_epoch = 3,
+ .target_assignment =
+ ut_topic_partitions(3, topic, 0, topic, 1, topic, 3),
+ .current_assignment = NULL,
+ .returned_assignment =
+ ut_topic_partitions(2, topic, 0, topic, 1),
+ },
+ {
+ .comment = "Target assignment 0,3. Returned assignment 0",
+ .current_member_epoch = 3,
+ .target_assignment = ut_topic_partitions(2, topic, 0, topic, 3),
+ .current_assignment = NULL,
+ .returned_assignment = ut_topic_partitions(1, topic, 0),
+ },
+ {
+ .comment = "Empty heartbeat",
+ .current_member_epoch = 3,
+ .current_assignment = NULL,
+ .returned_assignment = NULL,
+ },
+ {
+ .comment = "Current assignment 0,1",
+ .current_member_epoch = 3,
+ .current_assignment =
+ ut_topic_partitions(2, topic, 0, topic, 1),
+ .returned_assignment = NULL,
+ },
+ {
+ .comment = "Empty heartbeat",
+ .current_member_epoch = 3,
+ .current_assignment = NULL,
+ .returned_assignment = NULL,
+ },
+ {
+ .comment = "Target+Returned assignment 0,1,3. Epoch 3 -> 6",
+ .current_member_epoch = 6,
+ .target_assignment =
+ ut_topic_partitions(3, topic, 0, topic, 1, topic, 3),
+ .current_assignment = NULL,
+ .returned_assignment =
+ ut_topic_partitions(3, topic, 0, topic, 1, topic, 3),
+ },
+ {
+ .comment = "Empty heartbeat",
+ .current_member_epoch = 6,
+ .current_assignment = NULL,
+ .returned_assignment = NULL,
+ },
+ {
+ .comment = "Current assignment 0,1,3",
+ .current_member_epoch = 6,
+ .current_assignment =
+ ut_topic_partitions(3, topic, 0, topic, 1, topic, 3),
+ .returned_assignment = NULL,
+ },
+ };
+ return ut_cgrp_consumer_member_next_assignment0(
+ topic, 4, fixtures, RD_ARRAY_SIZE(fixtures));
+}
+
+/**
+ * @brief Test case where multiple revocations happen.
+ * They aren't acked but then a
+ * reassignment of all the revoked partition happens, bumping the epoch.
+ *
+ * @return Number of occurred errors.
+ */
+static int ut_cgrp_consumer_member_next_assignment3(void) {
+ RD_UT_SAY(
+ "Case 3: reassignment of revoked partition and new partition, no "
+ "revocation acknowledge");
+
+ const char *topic = "topic";
+ struct cgrp_consumer_member_next_assignment_fixture fixtures[] = {
+ {
+ .comment = "Target+Returned assignment 0,1,2. Epoch 0 -> 3",
+ .current_member_epoch = 3,
+ .current_assignment = NULL,
+ .target_assignment =
+ ut_topic_partitions(3, topic, 0, topic, 1, topic, 2),
+ .returned_assignment =
+ ut_topic_partitions(3, topic, 0, topic, 1, topic, 2),
+ },
+ {
+ .comment = "Current assignment 0,1,2",
+ .current_member_epoch = 3,
+ .current_assignment =
+ ut_topic_partitions(3, topic, 0, topic, 1, topic, 2),
+ .returned_assignment = NULL,
+ },
+ {
+ .comment = "Target assignment 0,1,3. Returned assignment 0,1",
+ .current_member_epoch = 3,
+ .target_assignment =
+ ut_topic_partitions(3, topic, 0, topic, 1, topic, 3),
+ .current_assignment = NULL,
+ .returned_assignment =
+ ut_topic_partitions(2, topic, 0, topic, 1),
+ },
+ {
+ .comment = "Target assignment 0,3. Returned assignment 0",
+ .current_member_epoch = 3,
+ .target_assignment = ut_topic_partitions(2, topic, 0, topic, 3),
+ .current_assignment = NULL,
+ .returned_assignment = ut_topic_partitions(1, topic, 0),
+ },
+ {
+ .comment = "Empty heartbeat",
+ .current_member_epoch = 3,
+ .current_assignment = NULL,
+ .returned_assignment = NULL,
+ },
+ {
+ .comment = "Target+Returned assignment 0,1,2,3. Epoch 3 -> 6",
+ .current_member_epoch = 6,
+ .target_assignment = ut_topic_partitions(
+ 3, topic, 0, topic, 1, topic, 2, topic, 3, NULL),
+ .returned_assignment = ut_topic_partitions(
+ 3, topic, 0, topic, 1, topic, 2, topic, 3, NULL),
+ },
+ {
+ .comment = "Empty heartbeat",
+ .current_member_epoch = 6,
+ .current_assignment = NULL,
+ .returned_assignment = NULL,
+ },
+ {
+ .comment = "Current assignment 0,1,2,3",
+ .current_member_epoch = 6,
+ .current_assignment = ut_topic_partitions(
+ 3, topic, 0, topic, 1, topic, 2, topic, 3, NULL),
+ .returned_assignment = NULL,
+ },
+ };
+ return ut_cgrp_consumer_member_next_assignment0(
+ topic, 4, fixtures, RD_ARRAY_SIZE(fixtures));
+}
+
+/**
+ * @brief Test case where a disconnection happens and after that
+ * the client send its assignment again, with same member epoch,
+ * and receives back the returned assignment, even if the same.
+ *
+ * @return Number of occurred errors.
+ */
+static int ut_cgrp_consumer_member_next_assignment4(void) {
+ RD_UT_SAY("Case 4: reconciliation after disconnection");
+
+ const char *topic = "topic";
+ struct cgrp_consumer_member_next_assignment_fixture fixtures[] = {
+ {
+ .comment = "Target+Returned assignment 0,1,2. Epoch 0 -> 3",
+ .current_member_epoch = 3,
+ .current_assignment = NULL,
+ .target_assignment =
+ ut_topic_partitions(3, topic, 0, topic, 1, topic, 2),
+ .returned_assignment =
+ ut_topic_partitions(3, topic, 0, topic, 1, topic, 2),
+ },
+ {
+ .comment = "Current assignment empty",
+ .current_member_epoch = 3,
+ .current_assignment = ut_topic_partitions(0),
+ .returned_assignment = NULL,
+ },
+ {
+ .comment = "Disconnected, resends current assignment. Returns "
+ "assignment again",
+ .reconnected = rd_true,
+ .current_member_epoch = 3,
+ .current_assignment = ut_topic_partitions(0),
+ .returned_assignment =
+ ut_topic_partitions(3, topic, 0, topic, 1, topic, 2),
+ },
+ {
+ .comment = "Empty heartbeat",
+ .current_member_epoch = 3,
+ .current_assignment = NULL,
+ .returned_assignment = NULL,
+ },
+ {
+ .comment = "Current assignment 0,1,2",
+ .current_member_epoch = 3,
+ .current_assignment =
+ ut_topic_partitions(3, topic, 0, topic, 1, topic, 2),
+ .returned_assignment = NULL,
+ },
+ };
+ return ut_cgrp_consumer_member_next_assignment0(
+ topic, 3, fixtures, RD_ARRAY_SIZE(fixtures));
+}
+
+/**
+ * @brief Test case where a session timeout happens and then
+ * the client receives a FENCED_MEMBER_EPOCH error,
+ * revokes all of its partitions and rejoins with epoch 0.
+ *
+ * @return Number of occurred errors.
+ */
+static int ut_cgrp_consumer_member_next_assignment5(void) {
+ RD_UT_SAY("Case 5: fenced consumer");
+
+ const char *topic = "topic";
+ struct cgrp_consumer_member_next_assignment_fixture fixtures[] = {
+ {
+ .comment = "Target+Returned assignment 0,1,2. Epoch 0 -> 3",
+ .current_member_epoch = 3,
+ .current_assignment = NULL,
+ .target_assignment =
+ ut_topic_partitions(3, topic, 0, topic, 1, topic, 2),
+ .returned_assignment =
+ ut_topic_partitions(3, topic, 0, topic, 1, topic, 2),
+ },
+ {
+ .comment = "Session times out, receives FENCED_MEMBER_EPOCH. "
+ "Epoch 3 -> 0",
+ .session_timed_out = rd_true,
+ .current_member_epoch = -1,
+ .current_assignment = NULL,
+ .returned_assignment = NULL,
+ },
+ {
+ .comment = "Target+Returned assignment 0,1,2. Epoch 0 -> 6",
+ .target_assignment =
+ ut_topic_partitions(3, topic, 0, topic, 1, topic, 2),
+ .current_member_epoch = 4,
+ .current_assignment = NULL,
+ .returned_assignment =
+ ut_topic_partitions(3, topic, 0, topic, 1, topic, 2),
+ },
+ {
+ .comment = "Current assignment 0,1,2",
+ .current_member_epoch = 4,
+ .current_assignment =
+ ut_topic_partitions(3, topic, 0, topic, 1, topic, 2),
+ .returned_assignment = NULL,
+ },
+ };
+ return ut_cgrp_consumer_member_next_assignment0(
+ topic, 3, fixtures, RD_ARRAY_SIZE(fixtures));
+}
+
+/**
+ * @brief Test all next assignment calculation cases,
+ * for KIP-848 consumer group type and collect
+ * number of errors.
+ *
+ * @return Number of occurred errors.
+ */
+static int ut_cgrp_consumer_member_next_assignment(void) {
+ RD_UT_BEGIN();
+ int failures = 0;
+
+ failures += ut_cgrp_consumer_member_next_assignment1();
+ failures += ut_cgrp_consumer_member_next_assignment2();
+ failures += ut_cgrp_consumer_member_next_assignment3();
+ failures += ut_cgrp_consumer_member_next_assignment4();
+ failures += ut_cgrp_consumer_member_next_assignment5();
+
+ RD_UT_ASSERT(!failures, "some tests failed");
+ RD_UT_PASS();
+}
+
+/**
+ * @brief Mock cluster unit tests
+ */
+int unittest_mock_cluster(void) {
+ int fails = 0;
+ fails += ut_cgrp_consumer_member_next_assignment();
+ return fails;
+}
diff --git a/src/rdkafka_mock.h b/src/rdkafka_mock.h
index e13d7d5e9e..0b81b312ef 100644
--- a/src/rdkafka_mock.h
+++ b/src/rdkafka_mock.h
@@ -168,6 +168,15 @@ rd_kafka_mock_push_request_errors_array(rd_kafka_mock_cluster_t *mcluster,
const rd_kafka_resp_err_t *errors);
+/**
+ * @brief Apply broker configuration group.initial.rebalance.delay.ms
+ * to the whole \p mcluster.
+ */
+RD_EXPORT void rd_kafka_mock_group_initial_rebalance_delay_ms(
+ rd_kafka_mock_cluster_t *mcluster,
+ int32_t delay_ms);
+
+
/**
* @brief Push \p cnt errors and RTT tuples in the \p ... va-arg list onto
* the broker's error stack for the given \p ApiKey.
@@ -313,6 +322,22 @@ RD_EXPORT rd_kafka_resp_err_t
rd_kafka_mock_broker_set_down(rd_kafka_mock_cluster_t *mcluster,
int32_t broker_id);
+/**
+ * @brief Sets a new \p host and \p port for a given broker identified by
+ * \p broker_id.
+ *
+ * @param mcluster Mock cluster instance.
+ * @param broker_id The id of the broker to modify.
+ * @param host The new hostname.
+ * @param port The new port.
+ */
+RD_EXPORT void
+rd_kafka_mock_broker_set_host_port(rd_kafka_mock_cluster_t *mcluster,
+ int32_t broker_id,
+ const char *host,
+ int port);
+
+
/**
* @brief Makes the broker accept connections again.
* This does NOT trigger leader change.
@@ -349,6 +374,33 @@ rd_kafka_mock_broker_set_rack(rd_kafka_mock_cluster_t *mcluster,
+/**
+ * @brief Remove and delete a mock broker from a cluster.
+ * All partitions assigned to that broker will be
+ * reassigned to other brokers.
+ *
+ * @param cluster The mock cluster containing the broker
+ * @param broker_id The broker to delete
+ * @returns 0 on success or -1 on error
+ */
+RD_EXPORT rd_kafka_resp_err_t
+rd_kafka_mock_broker_decommission(rd_kafka_mock_cluster_t *cluster,
+ int32_t broker_id);
+
+/**
+ * @brief Add a new broker to the cluster.
+ * Cluster partition will be reassigned to use the new broker
+ * as well.
+ *
+ * @param mcluster The mock cluster
+ * @param broker_id The id of the broker to add
+ *
+ * @returns Error value or 0 if no error occurred
+ */
+RD_EXPORT rd_kafka_resp_err_t
+rd_kafka_mock_broker_add(rd_kafka_mock_cluster_t *mcluster, int32_t broker_id);
+
+
/**
* @brief Explicitly sets the coordinator. If this API is not a standard
* hashing scheme will be used.
@@ -474,6 +526,82 @@ rd_kafka_mock_telemetry_set_requested_metrics(rd_kafka_mock_cluster_t *mcluster,
RD_EXPORT rd_kafka_resp_err_t
rd_kafka_mock_telemetry_set_push_interval(rd_kafka_mock_cluster_t *mcluster,
int64_t push_interval_ms);
+
+typedef struct rd_kafka_mock_cgrp_consumer_target_assignment_s
+ rd_kafka_mock_cgrp_consumer_target_assignment_t;
+
+/**
+ * @brief Create a new target assignment for \p member_cnt members
+ * given a member id and a member assignment for each member `i`,
+ * specified in \p member_ids[i] and \p assignment[i].
+ *
+ * @remark used for mocking target assignment
+ * in KIP-848 consumer group protocol.
+ *
+ * @param member_ids Array of member ids of size \p member_cnt.
+ * @param member_cnt Number of members.
+ * @param assignment Array of (rd_kafka_topic_partition_list_t *) of size \p
+ * member_cnt.
+ */
+RD_EXPORT rd_kafka_mock_cgrp_consumer_target_assignment_t *
+rd_kafka_mock_cgrp_consumer_target_assignment_new(
+ char **member_ids,
+ int member_cnt,
+ rd_kafka_topic_partition_list_t **assignment);
+
+/**
+ * @brief Destroy target assignment \p target_assignment .
+ */
+RD_EXPORT void rd_kafka_mock_cgrp_consumer_target_assignment_destroy(
+ rd_kafka_mock_cgrp_consumer_target_assignment_t *target_assignment);
+
+/**
+ * @brief Sets next target assignment for the group
+ * identified by \p group_id to the
+ * target assignment contained in \p target_assignment,
+ * in the cluster \p mcluster.
+ *
+ * @remark used for mocking target assignment
+ * in KIP-848 consumer group protocol.
+ *
+ * @param mcluster Mock cluster instance.
+ * @param group_id Group id.
+ * @param target_assignment Target assignment for all the members.
+ */
+RD_EXPORT void rd_kafka_mock_cgrp_consumer_target_assignment(
+ rd_kafka_mock_cluster_t *mcluster,
+ const char *group_id,
+ rd_kafka_mock_cgrp_consumer_target_assignment_t *target_assignment);
+
+/**
+ * @brief Sets group.consumer.session.timeout.ms
+ * for the cluster \p mcluster to \p group_consumer_session_timeout_ms.
+ *
+ * @remark used in KIP-848 consumer group protocol.
+ *
+ * @param mcluster Mock cluster instance.
+ * @param group_consumer_session_timeout_ms Session timeout in milliseconds.
+ */
+RD_EXPORT void rd_kafka_mock_set_group_consumer_session_timeout_ms(
+ rd_kafka_mock_cluster_t *mcluster,
+ int group_consumer_session_timeout_ms);
+
+/**
+ * @brief Sets group.consumer.heartbeat.interval.ms
+ * for the cluster \p mcluster to \p
+ * group_consumer_heartbeat_interval_ms.
+ *
+ * @remark used in KIP-848 consumer group protocol.
+ *
+ * @param mcluster Mock cluster instance.
+ * @param group_consumer_heartbeat_interval_ms Heartbeat interval in
+ * milliseconds.
+ */
+RD_EXPORT void rd_kafka_mock_set_group_consumer_heartbeat_interval_ms(
+ rd_kafka_mock_cluster_t *mcluster,
+ int group_consumer_heartbeat_interval_ms);
+
+
/**@}*/
#ifdef __cplusplus
diff --git a/src/rdkafka_mock_cgrp.c b/src/rdkafka_mock_cgrp.c
index 60b3aa1567..0c75e003e5 100644
--- a/src/rdkafka_mock_cgrp.c
+++ b/src/rdkafka_mock_cgrp.c
@@ -37,19 +37,21 @@
#include "rdkafka_mock_int.h"
-static const char *rd_kafka_mock_cgrp_state_names[] = {
+static const char *rd_kafka_mock_cgrp_classic_state_names[] = {
"Empty", "Joining", "Syncing", "Rebalancing", "Up"};
-static void rd_kafka_mock_cgrp_rebalance(rd_kafka_mock_cgrp_t *mcgrp,
- const char *reason);
static void
-rd_kafka_mock_cgrp_member_destroy(rd_kafka_mock_cgrp_t *mcgrp,
- rd_kafka_mock_cgrp_member_t *member);
+rd_kafka_mock_cgrp_classic_rebalance(rd_kafka_mock_cgrp_classic_t *mcgrp,
+ const char *reason);
+static void rd_kafka_mock_cgrp_classic_member_destroy(
+ rd_kafka_mock_cgrp_classic_t *mcgrp,
+ rd_kafka_mock_cgrp_classic_member_t *member);
-static void rd_kafka_mock_cgrp_set_state(rd_kafka_mock_cgrp_t *mcgrp,
- unsigned int new_state,
- const char *reason) {
+static void
+rd_kafka_mock_cgrp_classic_set_state(rd_kafka_mock_cgrp_classic_t *mcgrp,
+ unsigned int new_state,
+ const char *reason) {
if (mcgrp->state == new_state)
return;
@@ -57,8 +59,8 @@ static void rd_kafka_mock_cgrp_set_state(rd_kafka_mock_cgrp_t *mcgrp,
"Mock consumer group %s with %d member(s) "
"changing state %s -> %s: %s",
mcgrp->id, mcgrp->member_cnt,
- rd_kafka_mock_cgrp_state_names[mcgrp->state],
- rd_kafka_mock_cgrp_state_names[new_state], reason);
+ rd_kafka_mock_cgrp_classic_state_names[mcgrp->state],
+ rd_kafka_mock_cgrp_classic_state_names[new_state], reason);
mcgrp->state = new_state;
}
@@ -67,8 +69,9 @@ static void rd_kafka_mock_cgrp_set_state(rd_kafka_mock_cgrp_t *mcgrp,
/**
* @brief Mark member as active (restart session timer)
*/
-void rd_kafka_mock_cgrp_member_active(rd_kafka_mock_cgrp_t *mcgrp,
- rd_kafka_mock_cgrp_member_t *member) {
+void rd_kafka_mock_cgrp_classic_member_active(
+ rd_kafka_mock_cgrp_classic_t *mcgrp,
+ rd_kafka_mock_cgrp_classic_member_t *member) {
rd_kafka_dbg(mcgrp->cluster->rk, MOCK, "MOCK",
"Marking mock consumer group member %s as active",
member->id);
@@ -81,11 +84,11 @@ void rd_kafka_mock_cgrp_member_active(rd_kafka_mock_cgrp_t *mcgrp,
*
* @param member may be NULL.
*/
-rd_kafka_resp_err_t
-rd_kafka_mock_cgrp_check_state(rd_kafka_mock_cgrp_t *mcgrp,
- rd_kafka_mock_cgrp_member_t *member,
- const rd_kafka_buf_t *request,
- int32_t generation_id) {
+rd_kafka_resp_err_t rd_kafka_mock_cgrp_classic_check_state(
+ rd_kafka_mock_cgrp_classic_t *mcgrp,
+ rd_kafka_mock_cgrp_classic_member_t *member,
+ const rd_kafka_buf_t *request,
+ int32_t generation_id) {
int16_t ApiKey = request->rkbuf_reqhdr.ApiKey;
rd_bool_t has_generation_id = ApiKey == RD_KAFKAP_SyncGroup ||
ApiKey == RD_KAFKAP_Heartbeat ||
@@ -142,9 +145,9 @@ rd_kafka_mock_cgrp_check_state(rd_kafka_mock_cgrp_t *mcgrp,
/**
* @brief Set a member's assignment (from leader's SyncGroupRequest)
*/
-void rd_kafka_mock_cgrp_member_assignment_set(
- rd_kafka_mock_cgrp_t *mcgrp,
- rd_kafka_mock_cgrp_member_t *member,
+void rd_kafka_mock_cgrp_classic_member_assignment_set(
+ rd_kafka_mock_cgrp_classic_t *mcgrp,
+ rd_kafka_mock_cgrp_classic_member_t *member,
const rd_kafkap_bytes_t *Metadata) {
if (member->assignment) {
rd_assert(mcgrp->assignment_cnt > 0);
@@ -163,9 +166,10 @@ void rd_kafka_mock_cgrp_member_assignment_set(
/**
* @brief Sync done (successfully) or failed, send responses back to members.
*/
-static void rd_kafka_mock_cgrp_sync_done(rd_kafka_mock_cgrp_t *mcgrp,
- rd_kafka_resp_err_t err) {
- rd_kafka_mock_cgrp_member_t *member;
+static void
+rd_kafka_mock_cgrp_classic_sync_done(rd_kafka_mock_cgrp_classic_t *mcgrp,
+ rd_kafka_resp_err_t err) {
+ rd_kafka_mock_cgrp_classic_member_t *member;
TAILQ_FOREACH(member, &mcgrp->members, link) {
rd_kafka_buf_t *resp;
@@ -181,7 +185,8 @@ static void rd_kafka_mock_cgrp_sync_done(rd_kafka_mock_cgrp_t *mcgrp,
resp, !err ? member->assignment : NULL);
}
- rd_kafka_mock_cgrp_member_assignment_set(mcgrp, member, NULL);
+ rd_kafka_mock_cgrp_classic_member_assignment_set(mcgrp, member,
+ NULL);
if (member->conn) {
rd_kafka_mock_connection_set_blocking(member->conn,
@@ -201,20 +206,21 @@ static void rd_kafka_mock_cgrp_sync_done(rd_kafka_mock_cgrp_t *mcgrp,
* @brief Check if all members have sent SyncGroupRequests, if so, propagate
* assignment to members.
*/
-static void rd_kafka_mock_cgrp_sync_check(rd_kafka_mock_cgrp_t *mcgrp) {
+static void
+rd_kafka_mock_cgrp_classic_sync_check(rd_kafka_mock_cgrp_classic_t *mcgrp) {
rd_kafka_dbg(mcgrp->cluster->rk, MOCK, "MOCK",
"Mock consumer group %s: awaiting %d/%d syncing members "
"in state %s",
mcgrp->id, mcgrp->assignment_cnt, mcgrp->member_cnt,
- rd_kafka_mock_cgrp_state_names[mcgrp->state]);
+ rd_kafka_mock_cgrp_classic_state_names[mcgrp->state]);
if (mcgrp->assignment_cnt < mcgrp->member_cnt)
return;
- rd_kafka_mock_cgrp_sync_done(mcgrp, RD_KAFKA_RESP_ERR_NO_ERROR);
- rd_kafka_mock_cgrp_set_state(mcgrp, RD_KAFKA_MOCK_CGRP_STATE_UP,
- "all members synced");
+ rd_kafka_mock_cgrp_classic_sync_done(mcgrp, RD_KAFKA_RESP_ERR_NO_ERROR);
+ rd_kafka_mock_cgrp_classic_set_state(mcgrp, RD_KAFKA_MOCK_CGRP_STATE_UP,
+ "all members synced");
}
@@ -223,16 +229,16 @@ static void rd_kafka_mock_cgrp_sync_check(rd_kafka_mock_cgrp_t *mcgrp) {
* which will be sent when the all group member SyncGroupRequest are
* received.
*/
-rd_kafka_resp_err_t
-rd_kafka_mock_cgrp_member_sync_set(rd_kafka_mock_cgrp_t *mcgrp,
- rd_kafka_mock_cgrp_member_t *member,
- rd_kafka_mock_connection_t *mconn,
- rd_kafka_buf_t *resp) {
+rd_kafka_resp_err_t rd_kafka_mock_cgrp_classic_member_sync_set(
+ rd_kafka_mock_cgrp_classic_t *mcgrp,
+ rd_kafka_mock_cgrp_classic_member_t *member,
+ rd_kafka_mock_connection_t *mconn,
+ rd_kafka_buf_t *resp) {
if (mcgrp->state != RD_KAFKA_MOCK_CGRP_STATE_SYNCING)
return RD_KAFKA_RESP_ERR_REBALANCE_IN_PROGRESS; /* FIXME */
- rd_kafka_mock_cgrp_member_active(mcgrp, member);
+ rd_kafka_mock_cgrp_classic_member_active(mcgrp, member);
rd_assert(!member->resp);
@@ -241,7 +247,7 @@ rd_kafka_mock_cgrp_member_sync_set(rd_kafka_mock_cgrp_t *mcgrp,
rd_kafka_mock_connection_set_blocking(member->conn, rd_true);
/* Check if all members now have an assignment, if so, send responses */
- rd_kafka_mock_cgrp_sync_check(mcgrp);
+ rd_kafka_mock_cgrp_classic_sync_check(mcgrp);
return RD_KAFKA_RESP_ERR_NO_ERROR;
}
@@ -250,16 +256,16 @@ rd_kafka_mock_cgrp_member_sync_set(rd_kafka_mock_cgrp_t *mcgrp,
/**
* @brief Member is explicitly leaving the group (through LeaveGroupRequest)
*/
-rd_kafka_resp_err_t
-rd_kafka_mock_cgrp_member_leave(rd_kafka_mock_cgrp_t *mcgrp,
- rd_kafka_mock_cgrp_member_t *member) {
+rd_kafka_resp_err_t rd_kafka_mock_cgrp_classic_member_leave(
+ rd_kafka_mock_cgrp_classic_t *mcgrp,
+ rd_kafka_mock_cgrp_classic_member_t *member) {
rd_kafka_dbg(mcgrp->cluster->rk, MOCK, "MOCK",
"Member %s is leaving group %s", member->id, mcgrp->id);
- rd_kafka_mock_cgrp_member_destroy(mcgrp, member);
+ rd_kafka_mock_cgrp_classic_member_destroy(mcgrp, member);
- rd_kafka_mock_cgrp_rebalance(mcgrp, "explicit member leave");
+ rd_kafka_mock_cgrp_classic_rebalance(mcgrp, "explicit member leave");
return RD_KAFKA_RESP_ERR_NO_ERROR;
}
@@ -267,8 +273,9 @@ rd_kafka_mock_cgrp_member_leave(rd_kafka_mock_cgrp_t *mcgrp,
/**
* @brief Destroys/frees an array of protocols, including the array itself.
*/
-void rd_kafka_mock_cgrp_protos_destroy(rd_kafka_mock_cgrp_proto_t *protos,
- int proto_cnt) {
+void rd_kafka_mock_cgrp_classic_protos_destroy(
+ rd_kafka_mock_cgrp_classic_proto_t *protos,
+ int proto_cnt) {
int i;
for (i = 0; i < proto_cnt; i++) {
@@ -280,15 +287,16 @@ void rd_kafka_mock_cgrp_protos_destroy(rd_kafka_mock_cgrp_proto_t *protos,
rd_free(protos);
}
-static void
-rd_kafka_mock_cgrp_rebalance_timer_restart(rd_kafka_mock_cgrp_t *mcgrp,
- int timeout_ms);
+static void rd_kafka_mock_cgrp_classic_rebalance_timer_restart(
+ rd_kafka_mock_cgrp_classic_t *mcgrp,
+ int timeout_ms);
/**
* @brief Elect consumer group leader and send JoinGroup responses
*/
-static void rd_kafka_mock_cgrp_elect_leader(rd_kafka_mock_cgrp_t *mcgrp) {
- rd_kafka_mock_cgrp_member_t *member;
+static void
+rd_kafka_mock_cgrp_classic_elect_leader(rd_kafka_mock_cgrp_classic_t *mcgrp) {
+ rd_kafka_mock_cgrp_classic_member_t *member;
rd_assert(mcgrp->state == RD_KAFKA_MOCK_CGRP_STATE_JOINING);
rd_assert(!TAILQ_EMPTY(&mcgrp->members));
@@ -331,7 +339,7 @@ static void rd_kafka_mock_cgrp_elect_leader(rd_kafka_mock_cgrp_t *mcgrp) {
rd_bool_t is_leader = member == mcgrp->leader;
int member_cnt = is_leader ? mcgrp->member_cnt : 0;
rd_kafka_buf_t *resp;
- rd_kafka_mock_cgrp_member_t *member2;
+ rd_kafka_mock_cgrp_classic_member_t *member2;
rd_kafka_mock_connection_t *mconn;
/* Member connection has been closed, it will eventually
@@ -373,7 +381,7 @@ static void rd_kafka_mock_cgrp_elect_leader(rd_kafka_mock_cgrp_t *mcgrp) {
/* Mark each member as active to avoid them timing out
* at the same time as a JoinGroup handler that blocks
* session.timeout.ms to elect a leader. */
- rd_kafka_mock_cgrp_member_active(mcgrp, member);
+ rd_kafka_mock_cgrp_classic_member_active(mcgrp, member);
rd_kafka_mock_connection_set_blocking(mconn, rd_false);
rd_kafka_mock_connection_send_response(mconn, resp);
@@ -381,28 +389,32 @@ static void rd_kafka_mock_cgrp_elect_leader(rd_kafka_mock_cgrp_t *mcgrp) {
mcgrp->last_member_cnt = mcgrp->member_cnt;
- rd_kafka_mock_cgrp_set_state(mcgrp, RD_KAFKA_MOCK_CGRP_STATE_SYNCING,
- "leader elected, waiting for all "
- "members to sync");
+ rd_kafka_mock_cgrp_classic_set_state(mcgrp,
+ RD_KAFKA_MOCK_CGRP_STATE_SYNCING,
+ "leader elected, waiting for all "
+ "members to sync");
- rd_kafka_mock_cgrp_rebalance_timer_restart(mcgrp,
- mcgrp->session_timeout_ms);
+ rd_kafka_mock_cgrp_classic_rebalance_timer_restart(
+ mcgrp, mcgrp->session_timeout_ms);
}
/**
* @brief Trigger group rebalance.
*/
-static void rd_kafka_mock_cgrp_rebalance(rd_kafka_mock_cgrp_t *mcgrp,
- const char *reason) {
+static void
+rd_kafka_mock_cgrp_classic_rebalance(rd_kafka_mock_cgrp_classic_t *mcgrp,
+ const char *reason) {
int timeout_ms;
if (mcgrp->state == RD_KAFKA_MOCK_CGRP_STATE_JOINING)
return; /* Do nothing, group is already rebalancing. */
else if (mcgrp->state == RD_KAFKA_MOCK_CGRP_STATE_EMPTY)
- timeout_ms = 3000; /* First join, low timeout.
- * Same as group.initial.rebalance.delay.ms
- * on the broker. */
+ /* First join, low timeout.
+ * Same as group.initial.rebalance.delay.ms
+ * on the broker. */
+ timeout_ms =
+ mcgrp->cluster->defaults.group_initial_rebalance_delay_ms;
else if (mcgrp->state == RD_KAFKA_MOCK_CGRP_STATE_REBALANCING &&
mcgrp->member_cnt == mcgrp->last_member_cnt)
timeout_ms = 100; /* All members rejoined, quickly transition
@@ -416,21 +428,23 @@ static void rd_kafka_mock_cgrp_rebalance(rd_kafka_mock_cgrp_t *mcgrp,
if (mcgrp->state == RD_KAFKA_MOCK_CGRP_STATE_SYNCING)
/* Abort current Syncing state */
- rd_kafka_mock_cgrp_sync_done(
+ rd_kafka_mock_cgrp_classic_sync_done(
mcgrp, RD_KAFKA_RESP_ERR_REBALANCE_IN_PROGRESS);
- rd_kafka_mock_cgrp_set_state(mcgrp, RD_KAFKA_MOCK_CGRP_STATE_JOINING,
- reason);
- rd_kafka_mock_cgrp_rebalance_timer_restart(mcgrp, timeout_ms);
+ rd_kafka_mock_cgrp_classic_set_state(
+ mcgrp, RD_KAFKA_MOCK_CGRP_STATE_JOINING, reason);
+ rd_kafka_mock_cgrp_classic_rebalance_timer_restart(mcgrp, timeout_ms);
}
/**
* @brief Consumer group state machine triggered by timer events.
*/
-static void rd_kafka_mock_cgrp_fsm_timeout(rd_kafka_mock_cgrp_t *mcgrp) {
+static void
+rd_kafka_mock_cgrp_classic_fsm_timeout(rd_kafka_mock_cgrp_classic_t *mcgrp) {
rd_kafka_dbg(mcgrp->cluster->rk, MOCK, "MOCK",
"Mock consumer group %s FSM timeout in state %s",
- mcgrp->id, rd_kafka_mock_cgrp_state_names[mcgrp->state]);
+ mcgrp->id,
+ rd_kafka_mock_cgrp_classic_state_names[mcgrp->state]);
switch (mcgrp->state) {
case RD_KAFKA_MOCK_CGRP_STATE_EMPTY:
@@ -439,9 +453,9 @@ static void rd_kafka_mock_cgrp_fsm_timeout(rd_kafka_mock_cgrp_t *mcgrp) {
case RD_KAFKA_MOCK_CGRP_STATE_JOINING:
/* Timed out waiting for more members, elect a leader */
if (mcgrp->member_cnt > 0)
- rd_kafka_mock_cgrp_elect_leader(mcgrp);
+ rd_kafka_mock_cgrp_classic_elect_leader(mcgrp);
else
- rd_kafka_mock_cgrp_set_state(
+ rd_kafka_mock_cgrp_classic_set_state(
mcgrp, RD_KAFKA_MOCK_CGRP_STATE_EMPTY,
"no members joined");
break;
@@ -450,20 +464,20 @@ static void rd_kafka_mock_cgrp_fsm_timeout(rd_kafka_mock_cgrp_t *mcgrp) {
/* Timed out waiting for all members to sync */
/* Send error response to all waiting members */
- rd_kafka_mock_cgrp_sync_done(
+ rd_kafka_mock_cgrp_classic_sync_done(
mcgrp, RD_KAFKA_RESP_ERR_REBALANCE_IN_PROGRESS /* FIXME */);
- rd_kafka_mock_cgrp_set_state(
+ rd_kafka_mock_cgrp_classic_set_state(
mcgrp, RD_KAFKA_MOCK_CGRP_STATE_REBALANCING,
"timed out waiting for all members to synchronize");
break;
case RD_KAFKA_MOCK_CGRP_STATE_REBALANCING:
/* Timed out waiting for all members to Leave or re-Join */
- rd_kafka_mock_cgrp_set_state(mcgrp,
- RD_KAFKA_MOCK_CGRP_STATE_JOINING,
- "timed out waiting for all "
- "members to re-Join or Leave");
+ rd_kafka_mock_cgrp_classic_set_state(
+ mcgrp, RD_KAFKA_MOCK_CGRP_STATE_JOINING,
+ "timed out waiting for all "
+ "members to re-Join or Leave");
break;
case RD_KAFKA_MOCK_CGRP_STATE_UP:
@@ -475,27 +489,27 @@ static void rd_kafka_mock_cgrp_fsm_timeout(rd_kafka_mock_cgrp_t *mcgrp) {
static void rd_kafka_mcgrp_rebalance_timer_cb(rd_kafka_timers_t *rkts,
void *arg) {
- rd_kafka_mock_cgrp_t *mcgrp = arg;
+ rd_kafka_mock_cgrp_classic_t *mcgrp = arg;
- rd_kafka_mock_cgrp_fsm_timeout(mcgrp);
+ rd_kafka_mock_cgrp_classic_fsm_timeout(mcgrp);
}
/**
* @brief Restart the rebalance timer, postponing leader election.
*/
-static void
-rd_kafka_mock_cgrp_rebalance_timer_restart(rd_kafka_mock_cgrp_t *mcgrp,
- int timeout_ms) {
+static void rd_kafka_mock_cgrp_classic_rebalance_timer_restart(
+ rd_kafka_mock_cgrp_classic_t *mcgrp,
+ int timeout_ms) {
rd_kafka_timer_start_oneshot(
&mcgrp->cluster->timers, &mcgrp->rebalance_tmr, rd_true,
timeout_ms * 1000, rd_kafka_mcgrp_rebalance_timer_cb, mcgrp);
}
-static void
-rd_kafka_mock_cgrp_member_destroy(rd_kafka_mock_cgrp_t *mcgrp,
- rd_kafka_mock_cgrp_member_t *member) {
+static void rd_kafka_mock_cgrp_classic_member_destroy(
+ rd_kafka_mock_cgrp_classic_t *mcgrp,
+ rd_kafka_mock_cgrp_classic_member_t *member) {
rd_assert(mcgrp->member_cnt > 0);
TAILQ_REMOVE(&mcgrp->members, member, link);
mcgrp->member_cnt--;
@@ -508,9 +522,10 @@ rd_kafka_mock_cgrp_member_destroy(rd_kafka_mock_cgrp_t *mcgrp,
if (member->group_instance_id)
rd_free(member->group_instance_id);
- rd_kafka_mock_cgrp_member_assignment_set(mcgrp, member, NULL);
+ rd_kafka_mock_cgrp_classic_member_assignment_set(mcgrp, member, NULL);
- rd_kafka_mock_cgrp_protos_destroy(member->protos, member->proto_cnt);
+ rd_kafka_mock_cgrp_classic_protos_destroy(member->protos,
+ member->proto_cnt);
rd_free(member);
}
@@ -519,13 +534,13 @@ rd_kafka_mock_cgrp_member_destroy(rd_kafka_mock_cgrp_t *mcgrp,
/**
* @brief Find member in group.
*/
-rd_kafka_mock_cgrp_member_t *
-rd_kafka_mock_cgrp_member_find(const rd_kafka_mock_cgrp_t *mcgrp,
- const rd_kafkap_str_t *MemberId) {
- const rd_kafka_mock_cgrp_member_t *member;
+rd_kafka_mock_cgrp_classic_member_t *rd_kafka_mock_cgrp_classic_member_find(
+ const rd_kafka_mock_cgrp_classic_t *mcgrp,
+ const rd_kafkap_str_t *MemberId) {
+ const rd_kafka_mock_cgrp_classic_member_t *member;
TAILQ_FOREACH(member, &mcgrp->members, link) {
if (!rd_kafkap_str_cmp_str(MemberId, member->id))
- return (rd_kafka_mock_cgrp_member_t *)member;
+ return (rd_kafka_mock_cgrp_classic_member_t *)member;
}
return NULL;
@@ -535,25 +550,25 @@ rd_kafka_mock_cgrp_member_find(const rd_kafka_mock_cgrp_t *mcgrp,
/**
* @brief Update or add member to consumer group
*/
-rd_kafka_resp_err_t
-rd_kafka_mock_cgrp_member_add(rd_kafka_mock_cgrp_t *mcgrp,
- rd_kafka_mock_connection_t *mconn,
- rd_kafka_buf_t *resp,
- const rd_kafkap_str_t *MemberId,
- const rd_kafkap_str_t *ProtocolType,
- const rd_kafkap_str_t *GroupInstanceId,
- rd_kafka_mock_cgrp_proto_t *protos,
- int proto_cnt,
- int session_timeout_ms) {
- rd_kafka_mock_cgrp_member_t *member;
+rd_kafka_resp_err_t rd_kafka_mock_cgrp_classic_member_add(
+ rd_kafka_mock_cgrp_classic_t *mcgrp,
+ rd_kafka_mock_connection_t *mconn,
+ rd_kafka_buf_t *resp,
+ const rd_kafkap_str_t *MemberId,
+ const rd_kafkap_str_t *ProtocolType,
+ const rd_kafkap_str_t *GroupInstanceId,
+ rd_kafka_mock_cgrp_classic_proto_t *protos,
+ int proto_cnt,
+ int session_timeout_ms) {
+ rd_kafka_mock_cgrp_classic_member_t *member;
rd_kafka_resp_err_t err;
- err = rd_kafka_mock_cgrp_check_state(mcgrp, NULL, resp, -1);
+ err = rd_kafka_mock_cgrp_classic_check_state(mcgrp, NULL, resp, -1);
if (err)
return err;
/* Find member */
- member = rd_kafka_mock_cgrp_member_find(mcgrp, MemberId);
+ member = rd_kafka_mock_cgrp_classic_member_find(mcgrp, MemberId);
if (!member) {
/* Not found, add member */
member = rd_calloc(1, sizeof(*member));
@@ -575,20 +590,20 @@ rd_kafka_mock_cgrp_member_add(rd_kafka_mock_cgrp_t *mcgrp,
}
if (mcgrp->state != RD_KAFKA_MOCK_CGRP_STATE_JOINING)
- rd_kafka_mock_cgrp_rebalance(mcgrp, "member join");
+ rd_kafka_mock_cgrp_classic_rebalance(mcgrp, "member join");
mcgrp->session_timeout_ms = session_timeout_ms;
if (member->protos)
- rd_kafka_mock_cgrp_protos_destroy(member->protos,
- member->proto_cnt);
+ rd_kafka_mock_cgrp_classic_protos_destroy(member->protos,
+ member->proto_cnt);
member->protos = protos;
member->proto_cnt = proto_cnt;
rd_assert(!member->resp);
member->resp = resp;
member->conn = mconn;
- rd_kafka_mock_cgrp_member_active(mcgrp, member);
+ rd_kafka_mock_cgrp_classic_member_active(mcgrp, member);
return RD_KAFKA_RESP_ERR_NO_ERROR;
}
@@ -596,10 +611,10 @@ rd_kafka_mock_cgrp_member_add(rd_kafka_mock_cgrp_t *mcgrp,
/**
* @brief Check if any members have exceeded the session timeout.
*/
-static void rd_kafka_mock_cgrp_session_tmr_cb(rd_kafka_timers_t *rkts,
- void *arg) {
- rd_kafka_mock_cgrp_t *mcgrp = arg;
- rd_kafka_mock_cgrp_member_t *member, *tmp;
+static void rd_kafka_mock_cgrp_classic_session_tmr_cb(rd_kafka_timers_t *rkts,
+ void *arg) {
+ rd_kafka_mock_cgrp_classic_t *mcgrp = arg;
+ rd_kafka_mock_cgrp_classic_member_t *member, *tmp;
rd_ts_t now = rd_clock();
int timeout_cnt = 0;
@@ -613,19 +628,19 @@ static void rd_kafka_mock_cgrp_session_tmr_cb(rd_kafka_timers_t *rkts,
"Member %s session timed out for group %s",
member->id, mcgrp->id);
- rd_kafka_mock_cgrp_member_destroy(mcgrp, member);
+ rd_kafka_mock_cgrp_classic_member_destroy(mcgrp, member);
timeout_cnt++;
}
if (timeout_cnt)
- rd_kafka_mock_cgrp_rebalance(mcgrp, "member timeout");
+ rd_kafka_mock_cgrp_classic_rebalance(mcgrp, "member timeout");
}
-void rd_kafka_mock_cgrp_destroy(rd_kafka_mock_cgrp_t *mcgrp) {
- rd_kafka_mock_cgrp_member_t *member;
+void rd_kafka_mock_cgrp_classic_destroy(rd_kafka_mock_cgrp_classic_t *mcgrp) {
+ rd_kafka_mock_cgrp_classic_member_t *member;
- TAILQ_REMOVE(&mcgrp->cluster->cgrps, mcgrp, link);
+ TAILQ_REMOVE(&mcgrp->cluster->cgrps_classic, mcgrp, link);
rd_kafka_timer_stop(&mcgrp->cluster->timers, &mcgrp->rebalance_tmr,
rd_true);
@@ -636,15 +651,16 @@ void rd_kafka_mock_cgrp_destroy(rd_kafka_mock_cgrp_t *mcgrp) {
if (mcgrp->protocol_name)
rd_free(mcgrp->protocol_name);
while ((member = TAILQ_FIRST(&mcgrp->members)))
- rd_kafka_mock_cgrp_member_destroy(mcgrp, member);
+ rd_kafka_mock_cgrp_classic_member_destroy(mcgrp, member);
rd_free(mcgrp);
}
-rd_kafka_mock_cgrp_t *rd_kafka_mock_cgrp_find(rd_kafka_mock_cluster_t *mcluster,
- const rd_kafkap_str_t *GroupId) {
- rd_kafka_mock_cgrp_t *mcgrp;
- TAILQ_FOREACH(mcgrp, &mcluster->cgrps, link) {
+rd_kafka_mock_cgrp_classic_t *
+rd_kafka_mock_cgrp_classic_find(rd_kafka_mock_cluster_t *mcluster,
+ const rd_kafkap_str_t *GroupId) {
+ rd_kafka_mock_cgrp_classic_t *mcgrp;
+ TAILQ_FOREACH(mcgrp, &mcluster->cgrps_classic, link) {
if (!rd_kafkap_str_cmp_str(GroupId, mcgrp->id))
return mcgrp;
}
@@ -654,15 +670,15 @@ rd_kafka_mock_cgrp_t *rd_kafka_mock_cgrp_find(rd_kafka_mock_cluster_t *mcluster,
/**
- * @brief Find or create a consumer group
+ * @brief Find or create a classic consumer group
*/
-rd_kafka_mock_cgrp_t *
-rd_kafka_mock_cgrp_get(rd_kafka_mock_cluster_t *mcluster,
- const rd_kafkap_str_t *GroupId,
- const rd_kafkap_str_t *ProtocolType) {
- rd_kafka_mock_cgrp_t *mcgrp;
+rd_kafka_mock_cgrp_classic_t *
+rd_kafka_mock_cgrp_classic_get(rd_kafka_mock_cluster_t *mcluster,
+ const rd_kafkap_str_t *GroupId,
+ const rd_kafkap_str_t *ProtocolType) {
+ rd_kafka_mock_cgrp_classic_t *mcgrp;
- mcgrp = rd_kafka_mock_cgrp_find(mcluster, GroupId);
+ mcgrp = rd_kafka_mock_cgrp_classic_find(mcluster, GroupId);
if (mcgrp)
return mcgrp;
@@ -677,24 +693,25 @@ rd_kafka_mock_cgrp_get(rd_kafka_mock_cluster_t *mcluster,
TAILQ_INIT(&mcgrp->members);
rd_kafka_timer_start(&mcluster->timers, &mcgrp->session_tmr,
1000 * 1000 /*1s*/,
- rd_kafka_mock_cgrp_session_tmr_cb, mcgrp);
+ rd_kafka_mock_cgrp_classic_session_tmr_cb, mcgrp);
- TAILQ_INSERT_TAIL(&mcluster->cgrps, mcgrp, link);
+ TAILQ_INSERT_TAIL(&mcluster->cgrps_classic, mcgrp, link);
return mcgrp;
}
/**
- * @brief A client connection closed, check if any cgrp has any state
+ * @brief A client connection closed, check if any classic cgrp has any state
* for this connection that needs to be cleared.
*/
-void rd_kafka_mock_cgrps_connection_closed(rd_kafka_mock_cluster_t *mcluster,
- rd_kafka_mock_connection_t *mconn) {
- rd_kafka_mock_cgrp_t *mcgrp;
+void rd_kafka_mock_cgrps_classic_connection_closed(
+ rd_kafka_mock_cluster_t *mcluster,
+ rd_kafka_mock_connection_t *mconn) {
+ rd_kafka_mock_cgrp_classic_t *mcgrp;
- TAILQ_FOREACH(mcgrp, &mcluster->cgrps, link) {
- rd_kafka_mock_cgrp_member_t *member, *tmp;
+ TAILQ_FOREACH(mcgrp, &mcluster->cgrps_classic, link) {
+ rd_kafka_mock_cgrp_classic_member_t *member, *tmp;
TAILQ_FOREACH_SAFE(member, &mcgrp->members, link, tmp) {
if (member->conn == mconn) {
member->conn = NULL;
@@ -706,3 +723,1154 @@ void rd_kafka_mock_cgrps_connection_closed(rd_kafka_mock_cluster_t *mcluster,
}
}
}
+
+/**
+ * @struct Target assignment for a consumer group.
+ * `member_ids` and `assignment` are in the same order
+ * and have the same count.
+ */
+typedef struct rd_kafka_mock_cgrp_consumer_target_assignment_s {
+ rd_list_t *member_ids; /**< Member id list (char *). */
+ rd_list_t *assignment; /**< Assingment list
+ (rd_kafka_topic_partition_list_t *). */
+} rd_kafka_mock_cgrp_consumer_target_assignment_t;
+
+static rd_kafka_mock_cgrp_consumer_target_assignment_t *
+rd_kafka_mock_cgrp_consumer_target_assignment_new0(rd_list_t *member_ids,
+ rd_list_t *assignment) {
+ rd_assert(member_ids->rl_cnt == assignment->rl_cnt);
+ rd_kafka_mock_cgrp_consumer_target_assignment_t *target_assignment =
+ rd_calloc(1, sizeof(*target_assignment));
+ target_assignment->member_ids =
+ rd_list_copy(member_ids, rd_list_string_copy, NULL);
+ target_assignment->assignment = rd_list_copy(
+ assignment, rd_kafka_topic_partition_list_copy_opaque, NULL);
+ return target_assignment;
+}
+
+rd_kafka_mock_cgrp_consumer_target_assignment_t *
+rd_kafka_mock_cgrp_consumer_target_assignment_new(
+ char **member_ids,
+ int member_cnt,
+ rd_kafka_topic_partition_list_t **assignment) {
+ int i;
+ rd_list_t *member_id_list, *assignment_list;
+ rd_kafka_mock_cgrp_consumer_target_assignment_t *ret;
+
+ member_id_list = rd_list_new(member_cnt, rd_free);
+ assignment_list =
+ rd_list_new(member_cnt, rd_kafka_topic_partition_list_destroy_free);
+ for (i = 0; i < member_cnt; i++) {
+ rd_list_add(member_id_list, rd_strdup(member_ids[i]));
+ rd_list_add(assignment_list,
+ rd_kafka_topic_partition_list_copy(assignment[i]));
+ }
+
+ ret = rd_kafka_mock_cgrp_consumer_target_assignment_new0(
+ member_id_list, assignment_list);
+ rd_list_destroy(member_id_list);
+ rd_list_destroy(assignment_list);
+ return ret;
+}
+
+void rd_kafka_mock_cgrp_consumer_target_assignment_destroy(
+ rd_kafka_mock_cgrp_consumer_target_assignment_t *target_assignment) {
+ rd_list_destroy(target_assignment->member_ids);
+ rd_list_destroy(target_assignment->assignment);
+ rd_free(target_assignment);
+}
+
+/**
+ * @brief Sets next target assignment and member epoch for \p member
+ * to a copy of partition list \p rktparlist,
+ * filling its topic ids if not provided, using \p cgrp cluster topics.
+ *
+ * @param mcgrp The consumer group containing the member.
+ * @param member A consumer group member.
+ * @param target_member_epoch New member epoch.
+ * @param rktparlist Next target assignment.
+ *
+ * @locks mcluster->lock MUST be held.
+ */
+static void rd_kafka_mock_cgrp_consumer_member_target_assignment_set(
+ rd_kafka_mock_cgrp_consumer_t *mcgrp,
+ rd_kafka_mock_cgrp_consumer_member_t *member,
+ int target_member_epoch,
+ const rd_kafka_topic_partition_list_t *rktparlist) {
+ rd_kafka_topic_partition_t *rktpar;
+ if (member->target_assignment) {
+ rd_kafka_topic_partition_list_destroy(
+ member->target_assignment);
+ }
+ member->target_member_epoch = target_member_epoch;
+ member->target_assignment =
+ rd_kafka_topic_partition_list_copy(rktparlist);
+
+ /* If not present, fill topic ids using names */
+ RD_KAFKA_TPLIST_FOREACH(rktpar, member->target_assignment) {
+ rd_kafka_Uuid_t topic_id =
+ rd_kafka_topic_partition_get_topic_id(rktpar);
+ if (!rd_kafka_Uuid_cmp(topic_id, RD_KAFKA_UUID_ZERO)) {
+ rd_kafka_mock_topic_t *mtopic =
+ rd_kafka_mock_topic_find(mcgrp->cluster,
+ rktpar->topic);
+ if (mtopic)
+ rd_kafka_topic_partition_set_topic_id(
+ rktpar, mtopic->id);
+ }
+ }
+}
+
+/**
+ * @brief Sets next target assignment for group \p mcgrp
+ * to a copy of \p target_assignment partition lists.
+ *
+ * @param mcgrp The consumer group.
+ * @param target_assignment Target assignment for all members.
+ *
+ * @locks mcluster->lock MUST be held.
+ */
+static void rd_kafka_mock_cgrp_consumer_target_assignment_set(
+ rd_kafka_mock_cgrp_consumer_t *mcgrp,
+ rd_kafka_mock_cgrp_consumer_target_assignment_t *target_assignment) {
+ int i = 0;
+ int32_t new_target_member_epoch;
+ const char *member_id;
+ rd_kafka_mock_cgrp_consumer_member_t *member;
+
+ mcgrp->group_epoch++;
+ new_target_member_epoch = mcgrp->group_epoch;
+ RD_LIST_FOREACH(member_id, target_assignment->member_ids, i) {
+ rd_kafkap_str_t *member_id_str =
+ rd_kafkap_str_new(member_id, strlen(member_id));
+ rd_kafka_topic_partition_list_t *member_assignment =
+ rd_list_elem(target_assignment->assignment, i);
+ member = rd_kafka_mock_cgrp_consumer_member_find(mcgrp,
+ member_id_str);
+ rd_kafkap_str_destroy(member_id_str);
+
+ if (!member)
+ continue;
+
+ rd_kafka_mock_cgrp_consumer_member_target_assignment_set(
+ mcgrp, member, new_target_member_epoch, member_assignment);
+ }
+}
+
+typedef RD_MAP_TYPE(const char *, rd_list_t *) map_str_list;
+typedef RD_MAP_TYPE(const char *, int *) map_str_int;
+
+/**
+ * @brief Calculate a simple range target assignment for the consumer group \p
+ * mcgrp. This isn't replicating any given broker assignor but is used
+ * when the test doesn't need a specific type of assignment.
+ *
+ * If the test needs it, instead of replicating same conditions with all the
+ * members, one can mock the assignment directly with
+ * `rd_kafka_mock_cgrp_consumer_target_assignment`.
+ */
+static rd_kafka_mock_cgrp_consumer_target_assignment_t *
+rd_kafka_mock_cgrp_consumer_target_assignment_calculate_range(
+ const rd_kafka_mock_cgrp_consumer_t *mcgrp) {
+ int i, *i_pointer;
+ const char *topic;
+ rd_list_t *members;
+ rd_kafka_mock_cgrp_consumer_member_t *member;
+ rd_kafka_mock_cluster_t *mcluster = mcgrp->cluster;
+ /* List of member ids (char *) */
+ rd_list_t *member_ids = rd_list_new(mcgrp->member_cnt, rd_free);
+ /* List of member assignment (rd_kafka_topic_partition_list_t *) */
+ rd_list_t *assignment = rd_list_new(
+ mcgrp->member_cnt, rd_kafka_topic_partition_list_destroy_free);
+ /* Map from topic name to list of members */
+ map_str_list topic_members =
+ RD_MAP_INITIALIZER(mcgrp->member_cnt, rd_map_str_cmp,
+ rd_map_str_hash, NULL, rd_list_destroy_free);
+ /* Map from member id to index in the members and assignment lists. */
+ map_str_int member_idx = RD_MAP_INITIALIZER(
+ mcgrp->member_cnt, rd_map_str_cmp, rd_map_str_hash, NULL, rd_free);
+
+ i = 0;
+
+ /* First create a map with topics associated to the list of members
+ * and save the member idx in the `member_idx` map. */
+ TAILQ_FOREACH(member, &mcgrp->members, link) {
+ int j;
+ rd_list_add(member_ids, rd_strdup(member->id));
+ rd_list_add(assignment, rd_kafka_topic_partition_list_new(0));
+
+ RD_LIST_FOREACH(topic, member->subscribed_topics, j) {
+ if (!RD_MAP_GET(&topic_members, topic)) {
+ members = rd_list_new(0, NULL);
+ RD_MAP_SET(&topic_members, topic, members);
+ } else
+ members = RD_MAP_GET(&topic_members, topic);
+ rd_list_add(members, member);
+ }
+ i_pointer = rd_calloc(1, sizeof(*i_pointer));
+ *i_pointer = i;
+ RD_MAP_SET(&member_idx, member->id, i_pointer);
+ i++;
+ }
+
+ /* For each topic to a range assignment and add the
+ * corresponding partitions to the assignment for that member.
+ * Finds the list index using the `member_idx` map. */
+ RD_MAP_FOREACH(topic, members, &topic_members) {
+ rd_kafka_Uuid_t topic_id;
+ rd_kafka_topic_partition_list_t *member_assignment;
+ int members_cnt = rd_list_cnt(members);
+ int common, one_more, assigned = 0;
+ rd_kafkap_str_t Topic = {.str = topic, .len = strlen(topic)};
+ rd_kafka_mock_topic_t *mock_topic =
+ rd_kafka_mock_topic_find_by_kstr(mcluster, &Topic);
+ if (!mock_topic)
+ continue;
+
+ topic_id = mock_topic->id;
+
+ /* Assign one partition more
+ * to the first mock_topic->partition_cnt % members_cnt
+ * members. */
+ common = mock_topic->partition_cnt / members_cnt;
+ one_more = mock_topic->partition_cnt % members_cnt;
+
+ RD_LIST_FOREACH(member, members, i) {
+ int j, num_partitions = common;
+ int idx = *RD_MAP_GET(&member_idx, member->id);
+ member_assignment = rd_list_elem(assignment, idx);
+ if (idx < one_more)
+ num_partitions++;
+ for (j = 0; j < num_partitions; j++) {
+ rd_kafka_topic_partition_t *rktpar =
+ rd_kafka_topic_partition_list_add(
+ member_assignment, topic, assigned + j);
+ rd_kafka_topic_partition_set_topic_id(rktpar,
+ topic_id);
+ }
+ assigned += num_partitions;
+ }
+ }
+
+ rd_kafka_mock_cgrp_consumer_target_assignment_t *ret =
+ rd_kafka_mock_cgrp_consumer_target_assignment_new0(member_ids,
+ assignment);
+
+ RD_MAP_DESTROY(&topic_members);
+ RD_MAP_DESTROY(&member_idx);
+
+ rd_list_destroy(member_ids);
+ rd_list_destroy(assignment);
+
+ return ret;
+}
+
+/**
+ * @brief Recalculate and set a target assignment for \p mcgrp
+ * only if `mcgrp->manual_assignment` isn't set.
+ *
+ * @locks mcluster->lock MUST be held.
+ */
+static void rd_kafka_mock_cgrp_consumer_target_assignment_recalculate(
+ rd_kafka_mock_cgrp_consumer_t *mcgrp) {
+ if (mcgrp->manual_assignment)
+ return;
+
+ rd_kafka_mock_cgrp_consumer_target_assignment_t *target_assignment =
+ rd_kafka_mock_cgrp_consumer_target_assignment_calculate_range(
+ mcgrp);
+ rd_kafka_mock_cgrp_consumer_target_assignment_set(mcgrp,
+ target_assignment);
+ rd_kafka_mock_cgrp_consumer_target_assignment_destroy(
+ target_assignment);
+}
+
+/**
+ * @brief Set manual target assignment \p target_assignment
+ * to the consumer group \p mcgrp .
+ *
+ * @param mcgrp Consumer group
+ * @param target_assignment Target assignment to set.
+ * Pass NULL to return to automatic assignment.
+ *
+ * @locks mcluster->lock MUST be held.
+ */
+static void rd_kafka_mock_cgrp_consumer_target_assignment_set_manual(
+ rd_kafka_mock_cgrp_consumer_t *mcgrp,
+ rd_kafka_mock_cgrp_consumer_target_assignment_t *target_assignment) {
+ if (!target_assignment) {
+ mcgrp->manual_assignment = rd_false;
+ rd_kafka_mock_cgrp_consumer_target_assignment_recalculate(
+ mcgrp);
+ return;
+ }
+
+ mcgrp->manual_assignment = rd_true;
+
+ rd_kafka_mock_cgrp_consumer_target_assignment_set(mcgrp,
+ target_assignment);
+}
+
+/**
+ * @brief Sets \p member current assignment to a copy of
+ * \p current_assignment.
+ *
+ * @param member A consumer group member.
+ * @param current_assignment Current assignment to set.
+ *
+ * @locks mcluster->lock MUST be held.
+ */
+static void rd_kafka_mock_cgrp_consumer_member_current_assignment_set(
+ rd_kafka_mock_cgrp_consumer_member_t *member,
+ const rd_kafka_topic_partition_list_t *current_assignment) {
+ if (member->current_assignment) {
+ rd_kafka_topic_partition_list_destroy(
+ member->current_assignment);
+ }
+
+ member->current_assignment =
+ current_assignment
+ ? rd_kafka_topic_partition_list_copy(current_assignment)
+ : NULL;
+}
+
+/**
+ * @brief Sets \p member returned assignment to a
+ * copy of \p returned_assignment.
+ *
+ * @param member A consumer group member.
+ * @param returned_assignment Returned assignment to set.
+ *
+ * @locks mcluster->lock MUST be held.
+ */
+static void rd_kafka_mock_cgrp_consumer_member_returned_assignment_set(
+ rd_kafka_mock_cgrp_consumer_member_t *member,
+ const rd_kafka_topic_partition_list_t *returned_assignment) {
+ if (member->returned_assignment) {
+ rd_kafka_topic_partition_list_destroy(
+ member->returned_assignment);
+ }
+ member->returned_assignment =
+ returned_assignment
+ ? rd_kafka_topic_partition_list_copy(returned_assignment)
+ : NULL;
+}
+
+/**
+ * @brief Returns a copy of \p member target assignment containing only
+ * partitions that can be assignment, whose topic id is non-zero.
+ *
+ * @param member The group member.
+ *
+ * @remark The returned pointer ownership is transferred to the caller.
+ *
+ * @locks mcluster->lock MUST be held.
+ */
+static rd_kafka_topic_partition_list_t *
+rd_kafka_mock_cgrp_consumer_member_target_assignment_assignable(
+ rd_kafka_mock_cgrp_consumer_member_t *member) {
+ rd_kafka_topic_partition_list_t *assignment = member->target_assignment;
+ rd_kafka_topic_partition_t *rktpar;
+ rd_kafka_topic_partition_list_t *ret =
+ rd_kafka_topic_partition_list_new(assignment->cnt);
+
+ RD_KAFKA_TPLIST_FOREACH(rktpar, assignment) {
+ rd_kafka_Uuid_t topic_id =
+ rd_kafka_topic_partition_get_topic_id(rktpar);
+ if (rd_kafka_Uuid_cmp(topic_id, RD_KAFKA_UUID_ZERO)) {
+ rd_kafka_topic_partition_list_add_copy(ret, rktpar);
+ }
+ }
+
+ return ret;
+}
+
+/**
+ * Returns true iff \p new_assignment doesn't have any intersection with any
+ * other member current assignment.
+ *
+ * If there's an intersection, it means we cannot bump the epoch at the moment,
+ * because some of these partitions are held by a different member. They have
+ * to be revoked from that member before it's possible to increase the epoch
+ * and assign additional partitions to this member.
+ */
+rd_bool_t rd_kafka_mock_cgrp_consumer_member_next_assignment_can_bump_epoch(
+ rd_kafka_mock_cgrp_consumer_member_t *member,
+ rd_kafka_topic_partition_list_t *new_assignment) {
+ rd_kafka_topic_partition_list_t *double_assignment,
+ *assigned_partitions = rd_kafka_topic_partition_list_new(0);
+ rd_kafka_mock_cgrp_consumer_member_t *other_member;
+ rd_kafka_mock_cgrp_consumer_t *mcgrp = member->mcgrp;
+ rd_bool_t ret;
+
+ TAILQ_FOREACH(other_member, &mcgrp->members, link) {
+ int other_current_assignment_cnt = 0,
+ other_returned_assignment_cnt = 0;
+ if (member == other_member)
+ continue;
+ if (other_member->current_assignment)
+ other_current_assignment_cnt =
+ other_member->current_assignment->cnt;
+ if (other_member->returned_assignment)
+ other_returned_assignment_cnt =
+ other_member->returned_assignment->cnt;
+
+ if (other_current_assignment_cnt > 0 &&
+ other_current_assignment_cnt >
+ other_returned_assignment_cnt) {
+ /* This is the case where we're revoking
+ * some partitions.
+ * returned_assignment < current_assignment. */
+ rd_kafka_topic_partition_list_add_list(
+ assigned_partitions,
+ other_member->current_assignment);
+ } else if (other_returned_assignment_cnt > 0) {
+ /* This is the case where we're assigning
+ * some partitions.
+ * returned_assignment >= current_assignment. */
+ rd_kafka_topic_partition_list_add_list(
+ assigned_partitions,
+ other_member->returned_assignment);
+ }
+ }
+ double_assignment = rd_kafka_topic_partition_list_intersection_by_id(
+ new_assignment, assigned_partitions);
+ ret = double_assignment->cnt == 0;
+
+ rd_kafka_topic_partition_list_destroy(assigned_partitions);
+ rd_kafka_topic_partition_list_destroy(double_assignment);
+ return ret;
+}
+
+/**
+ * @brief Calculates if \p member,
+ * needs a revocation, that is if its current assignment
+ * isn't a subset of its target assignment.
+ * In case it needs a revocation, it returns
+ * the intersection between the two assignments,
+ * that is the remaining partitions after revocation
+ * of those not included in target assignment.
+ *
+ * @param member The group member.
+ *
+ * @return The remaining set of partitions, or NULL in case no revocation
+ * is needed.
+ *
+ * @remark The returned pointer ownership is transferred to the caller.
+ *
+ * @locks mcluster->lock MUST be held.
+ */
+static rd_kafka_topic_partition_list_t *
+rd_kafka_mock_cgrp_consumer_member_needs_revocation(
+ rd_kafka_mock_cgrp_consumer_member_t *member) {
+ rd_kafka_topic_partition_list_t *intersection;
+ rd_bool_t needs_revocation;
+
+ if (member->current_assignment)
+ /* If we have a current assignment we
+ * calculate the intersection with
+ * target assignment. */
+ intersection = rd_kafka_topic_partition_list_intersection_by_id(
+ member->current_assignment, member->target_assignment);
+ else
+ /* Otherwise intersection is empty. */
+ intersection = rd_kafka_topic_partition_list_new(0);
+
+ needs_revocation = member->current_assignment &&
+ intersection->cnt < member->current_assignment->cnt;
+ if (needs_revocation) {
+ return intersection;
+ }
+
+ rd_kafka_topic_partition_list_destroy(intersection);
+ return NULL;
+}
+
+/**
+ * @brief Calculates if \p member,
+ * can receive new partitions, given revocation is completed.
+ * In case new partitions aren't held by other members it
+ * returns the assignable target assignment and bumps current
+ * member epoch, otherwise it returns NULL and
+ * doesn't change current member epoch.
+ *
+ * @param member The group member.
+ *
+ * @return The assignable set of partitions, or NULL in case new partitions
+ * cannot be assigned yet.
+ *
+ * @remark The returned pointer ownership is transferred to the caller.
+ *
+ * @locks mcluster->lock MUST be held.
+ */
+static rd_kafka_topic_partition_list_t *
+rd_kafka_mock_cgrp_consumer_member_needs_assignment(
+ rd_kafka_mock_cgrp_consumer_member_t *member) {
+ rd_kafka_topic_partition_list_t *returned_assignment =
+ rd_kafka_mock_cgrp_consumer_member_target_assignment_assignable(
+ member);
+
+ if (!rd_kafka_mock_cgrp_consumer_member_next_assignment_can_bump_epoch(
+ member, returned_assignment)) {
+ /* We can't bump the epoch still,
+ * there are some partitions held by other members.
+ * We have to return NULL. */
+ rd_kafka_topic_partition_list_destroy(returned_assignment);
+ return NULL;
+ }
+
+ /* No partitions to remove, return
+ * target assignment and reconcile the
+ * epochs */
+ member->current_member_epoch = member->target_member_epoch;
+ return returned_assignment;
+}
+
+/**
+ * @brief Calculates next assignment and member epoch for a \p member,
+ * given \p current_assignment.
+ *
+ * @param member The group member.
+ * @param current_assignment The assignment sent by the member, or NULL if it
+ * didn't change. Must be NULL if *member_epoch is 0.
+ * @param member_epoch Pointer to client reported member epoch. Can be updated.
+ *
+ * @return The new assignment to return to the member.
+ *
+ * @remark The returned pointer ownership is transferred to the caller.
+ *
+ * @locks mcluster->lock MUST be held.
+ */
+rd_kafka_topic_partition_list_t *
+rd_kafka_mock_cgrp_consumer_member_next_assignment(
+ rd_kafka_mock_cgrp_consumer_member_t *member,
+ rd_kafka_topic_partition_list_t *current_assignment,
+ int *member_epoch) {
+ rd_kafka_topic_partition_list_t *assignment_to_return = NULL;
+
+ if (current_assignment) {
+ /* Update current assignment to reflect what is provided
+ * by the client. */
+ rd_kafka_mock_cgrp_consumer_member_current_assignment_set(
+ member, current_assignment);
+ }
+
+ if (*member_epoch > 0 &&
+ member->current_member_epoch != *member_epoch) {
+ /* Member epoch is different from the one we expect,
+ * that means we have to fence the member. */
+ *member_epoch = -1; /* FENCED_MEMBER_EPOCH */
+ return NULL;
+ }
+
+ if (member->target_assignment) {
+ /* We have a target assignment,
+ * let's check if we can assign it. */
+
+ if (*member_epoch != member->current_member_epoch ||
+ member->current_member_epoch !=
+ member->target_member_epoch) {
+ /* Epochs are different, that means we have to bump the
+ * epoch immediately or do some revocations
+ * before that. */
+
+ assignment_to_return =
+ rd_kafka_mock_cgrp_consumer_member_needs_revocation(
+ member);
+ if (!assignment_to_return) {
+ /* After revocation we only have to
+ * add new partitions.
+ * In case these new partitions are held
+ * by other members we still cannot do it. */
+ assignment_to_return =
+ rd_kafka_mock_cgrp_consumer_member_needs_assignment(
+ member);
+ }
+ } else if (!member->returned_assignment) {
+ /* If all the epochs are the same, the only case
+ * where we have to return the assignment is
+ * after a disconnection, when returned_assignment has
+ * been reset to NULL. */
+ assignment_to_return =
+ rd_kafka_mock_cgrp_consumer_member_target_assignment_assignable(
+ member);
+ }
+ }
+
+ *member_epoch = member->current_member_epoch;
+ if (assignment_to_return) {
+ /* Compare assignment_to_return with last returned_assignment.
+ * If equal, return NULL, otherwise return assignment_to_return
+ * and update last returned_assignment. */
+ rd_bool_t same_returned_assignment =
+ member->returned_assignment &&
+ !rd_kafka_topic_partition_list_cmp(
+ member->returned_assignment, assignment_to_return,
+ rd_kafka_topic_partition_by_id_cmp);
+
+ if (same_returned_assignment) {
+ /* Returned assignment is the same as previous
+ * one, we return NULL instead to show no change. */
+ rd_kafka_topic_partition_list_destroy(
+ assignment_to_return);
+ assignment_to_return = NULL;
+ } else {
+ /* We store returned assignment
+ * for later comparison. */
+ rd_kafka_mock_cgrp_consumer_member_returned_assignment_set(
+ member, assignment_to_return);
+ }
+ }
+ return assignment_to_return;
+}
+
+/**
+ * @brief Mark member as active (restart session timer).
+ *
+ * @param mcgrp Member's consumer group.
+ * @param member Member to set as active.
+ *
+ * @locks mcluster->lock MUST be held.
+ */
+void rd_kafka_mock_cgrp_consumer_member_active(
+ rd_kafka_mock_cgrp_consumer_t *mcgrp,
+ rd_kafka_mock_cgrp_consumer_member_t *member) {
+ rd_kafka_dbg(mcgrp->cluster->rk, MOCK, "MOCK",
+ "Marking mock consumer group member %s as active",
+ member->id);
+ member->ts_last_activity = rd_clock();
+}
+
+/**
+ * @brief Finds a member in consumer group \p mcgrp by \p MemberId.
+ *
+ * @param mcgrp Consumer group to search.
+ * @param MemberId Member id to look for.
+ * @return Found member or NULL.
+ *
+ * @locks mcluster->lock MUST be held.
+ */
+rd_kafka_mock_cgrp_consumer_member_t *rd_kafka_mock_cgrp_consumer_member_find(
+ const rd_kafka_mock_cgrp_consumer_t *mcgrp,
+ const rd_kafkap_str_t *MemberId) {
+ const rd_kafka_mock_cgrp_consumer_member_t *member;
+ TAILQ_FOREACH(member, &mcgrp->members, link) {
+ if (!rd_kafkap_str_cmp_str(MemberId, member->id))
+ return (rd_kafka_mock_cgrp_consumer_member_t *)member;
+ }
+
+ return NULL;
+}
+
+/**
+ * @brief Finds a member in consumer group \p mcgrp by \p InstanceId.
+ *
+ * @param mcgrp Consumer group to search.
+ * @param InstanceId Instance id to look for.
+ * @return Found member or NULL.
+ *
+ * @locks mcluster->lock MUST be held.
+ */
+rd_kafka_mock_cgrp_consumer_member_t *
+rd_kafka_mock_cgrp_consumer_member_find_by_instance_id(
+ const rd_kafka_mock_cgrp_consumer_t *mcgrp,
+ const rd_kafkap_str_t *InstanceId) {
+ if (RD_KAFKAP_STR_IS_NULL(InstanceId))
+ return NULL;
+
+ const rd_kafka_mock_cgrp_consumer_member_t *member;
+ TAILQ_FOREACH(member, &mcgrp->members, link) {
+ if (!member->instance_id)
+ continue;
+
+ if (!rd_kafkap_str_cmp_str(InstanceId, member->instance_id))
+ return (rd_kafka_mock_cgrp_consumer_member_t *)member;
+ }
+
+ return NULL;
+}
+
+static void validate_subscription(const rd_kafkap_str_t *SubscribedTopicNames,
+ int32_t SubscribedTopicNamesCnt,
+ const rd_kafkap_str_t *SubscribedTopicRegex) {
+ /* Either they are both NULL
+ * or both non-NULL. */
+ rd_assert((SubscribedTopicNames == NULL) ==
+ RD_KAFKAP_STR_IS_NULL(SubscribedTopicRegex));
+ /* If they're not NULL at least one should be non-empty */
+ rd_assert(SubscribedTopicNames == NULL || SubscribedTopicNamesCnt > 0 ||
+ RD_KAFKAP_STR_LEN(SubscribedTopicRegex) > 0);
+}
+
+/**
+ * @brief Set the subscribed topics for the member \p member based on \p
+ * SubscribedTopicNames and \p SubscribedTopicRegex. Deduplicates the list after
+ * sorting it.
+ * @return `rd_true` if the subscription was changed, that happens
+ * if it's set and different from previous one.
+ *
+ * @locks mcluster->lock MUST be held.
+ */
+static rd_bool_t rd_kafka_mock_cgrp_consumer_member_subscribed_topic_names_set(
+ rd_kafka_mock_cgrp_consumer_member_t *member,
+ rd_kafkap_str_t *SubscribedTopicNames,
+ int32_t SubscribedTopicNamesCnt,
+ const rd_kafkap_str_t *SubscribedTopicRegex) {
+ rd_bool_t changed = rd_false;
+ rd_list_t *new_subscription;
+ int32_t i;
+
+ validate_subscription(SubscribedTopicNames, SubscribedTopicNamesCnt,
+ SubscribedTopicRegex);
+
+ if (!SubscribedTopicNames &&
+ RD_KAFKAP_STR_IS_NULL(SubscribedTopicRegex) &&
+ !member->subscribed_topic_regex) {
+ /* When client is sending NULL for SubscribedTopicNames and
+ * SubscribedTopicRegex, its subscription didn't change. If we
+ * already had a regex, we need to compute the regex again. */
+ return changed;
+ }
+
+ if (SubscribedTopicNames) {
+ RD_IF_FREE(member->subscribed_topic_names, rd_list_destroy);
+ member->subscribed_topic_names =
+ rd_list_new(SubscribedTopicNamesCnt, rd_free);
+ for (i = 0; i < SubscribedTopicNamesCnt; i++) {
+ rd_list_add(
+ member->subscribed_topic_names,
+ RD_KAFKAP_STR_DUP(&SubscribedTopicNames[i]));
+ }
+ }
+
+ if (!RD_KAFKAP_STR_IS_NULL(SubscribedTopicRegex)) {
+ RD_IF_FREE(member->subscribed_topic_regex, rd_free);
+ member->subscribed_topic_regex =
+ RD_KAFKAP_STR_DUP(SubscribedTopicRegex);
+ }
+
+ new_subscription =
+ rd_list_new(rd_list_cnt(member->subscribed_topic_names), rd_free);
+
+ rd_list_copy_to(new_subscription, member->subscribed_topic_names,
+ rd_list_string_copy, NULL);
+
+ if (member->subscribed_topic_regex[0]) {
+ rd_kafka_mock_cluster_t *mcluster = member->mcgrp->cluster;
+ rd_kafka_mock_topic_t *mtopic;
+ char errstr[1];
+ rd_regex_t *re = rd_regex_comp(member->subscribed_topic_regex,
+ errstr, sizeof(errstr));
+
+ TAILQ_FOREACH(mtopic, &mcluster->topics, link) {
+ if (rd_regex_exec(re, mtopic->name))
+ rd_list_add(new_subscription,
+ rd_strdup(mtopic->name));
+ }
+
+ rd_regex_destroy(re);
+ }
+
+ rd_list_deduplicate(&new_subscription, rd_strcmp2);
+
+ if (!member->subscribed_topics ||
+ rd_list_cmp(new_subscription, member->subscribed_topics,
+ rd_list_cmp_str)) {
+ if (member->subscribed_topics)
+ rd_list_destroy(member->subscribed_topics);
+ member->subscribed_topics =
+ rd_list_copy(new_subscription, rd_list_string_copy, NULL);
+ changed = rd_true;
+ }
+ rd_list_destroy(new_subscription);
+ return changed;
+}
+
+static void rd_kafka_mock_cgrp_consumer_member_topic_id_set(
+ rd_kafka_mock_cgrp_consumer_member_t *member,
+ const rd_kafkap_str_t *MemberId) {
+ /* KIP 1082: MemberId is generated by the client */
+ rd_assert(RD_KAFKAP_STR_LEN(MemberId) > 0);
+ RD_IF_FREE(member->id, rd_free);
+ member->id = RD_KAFKAP_STR_DUP(MemberId);
+}
+
+/**
+ * @brief Adds a member to consumer group \p mcgrp. If member with same
+ * \p MemberId is already present, only updates the connection and
+ * sets it as active.
+ *
+ * @param mcgrp Consumer group to add the member to.
+ * @param conn Member connection.
+ * @param MemberId Member id.
+ * @param InstanceId Group instance id (optional).
+ * @param session_timeout_ms Session timeout to use.
+ * @param SubscribedTopicNames Array of subscribed topics.
+ * Mandatory if the member is a new one.
+ * @param SubscribedTopicNamesCnt Number of elements in \p SubscribedTopicNames.
+ * @param SubscribedTopicRegex Subscribed topic regex.
+ *
+ * @return New or existing member, NULL if the member cannot be added.
+ *
+ * @locks mcluster->lock MUST be held.
+ */
+rd_kafka_mock_cgrp_consumer_member_t *rd_kafka_mock_cgrp_consumer_member_add(
+ rd_kafka_mock_cgrp_consumer_t *mcgrp,
+ struct rd_kafka_mock_connection_s *conn,
+ const rd_kafkap_str_t *MemberId,
+ const rd_kafkap_str_t *InstanceId,
+ rd_kafkap_str_t *SubscribedTopicNames,
+ int32_t SubscribedTopicNamesCnt,
+ const rd_kafkap_str_t *SubscribedTopicRegex) {
+ rd_kafka_mock_cgrp_consumer_member_t *member = NULL;
+ rd_bool_t changed = rd_false;
+
+ /* Find member */
+ member = rd_kafka_mock_cgrp_consumer_member_find(mcgrp, MemberId);
+ if (!member) {
+ member = rd_kafka_mock_cgrp_consumer_member_find_by_instance_id(
+ mcgrp, InstanceId);
+
+ if (member) {
+ if (!member->left_static_membership) {
+ /* Old member still active,
+ * fence this one */
+ return NULL;
+ }
+
+ if (rd_kafkap_str_cmp_str(MemberId, member->id) != 0) {
+ /* Member is a new instance and is rejoining
+ * with a new MemberId. */
+ rd_kafka_mock_cgrp_consumer_member_topic_id_set(
+ member, MemberId);
+ }
+ member->left_static_membership = rd_false;
+ }
+ } else {
+ member->left_static_membership = rd_false;
+ }
+
+ if (!member) {
+ validate_subscription(SubscribedTopicNames,
+ SubscribedTopicNamesCnt,
+ SubscribedTopicRegex);
+
+ /* In case of session timeout
+ * where the member isn't aware it's been fenced. */
+ if (SubscribedTopicNames == NULL)
+ return NULL;
+
+ /* Not found, add member */
+ member = rd_calloc(1, sizeof(*member));
+ member->mcgrp = mcgrp;
+
+ rd_kafka_mock_cgrp_consumer_member_topic_id_set(member,
+ MemberId);
+
+ if (!RD_KAFKAP_STR_IS_NULL(InstanceId))
+ member->instance_id = RD_KAFKAP_STR_DUP(InstanceId);
+
+ TAILQ_INSERT_TAIL(&mcgrp->members, member, link);
+ mcgrp->member_cnt++;
+ changed = rd_true;
+ member->target_member_epoch = mcgrp->group_epoch;
+ }
+
+ changed |=
+ rd_kafka_mock_cgrp_consumer_member_subscribed_topic_names_set(
+ member, SubscribedTopicNames, SubscribedTopicNamesCnt,
+ SubscribedTopicRegex);
+
+ mcgrp->session_timeout_ms =
+ mcgrp->cluster->defaults.group_consumer_session_timeout_ms;
+ mcgrp->heartbeat_interval_ms =
+ mcgrp->cluster->defaults.group_consumer_heartbeat_interval_ms;
+
+ member->conn = conn;
+
+ rd_kafka_mock_cgrp_consumer_member_active(mcgrp, member);
+
+ if (changed)
+ rd_kafka_mock_cgrp_consumer_target_assignment_recalculate(
+ mcgrp);
+
+ return member;
+}
+
+/**
+ * @brief Destroys a consumer group member, removing from its consumer group.
+ *
+ * @param mcgrp Member consumer group.
+ * @param member Member to destroy.
+ *
+ * @locks mcluster->lock MUST be held.
+ */
+static void rd_kafka_mock_cgrp_consumer_member_destroy(
+ rd_kafka_mock_cgrp_consumer_t *mcgrp,
+ rd_kafka_mock_cgrp_consumer_member_t *member) {
+ rd_assert(mcgrp->member_cnt > 0);
+ TAILQ_REMOVE(&mcgrp->members, member, link);
+ mcgrp->member_cnt--;
+
+ rd_kafka_mock_cgrp_consumer_target_assignment_recalculate(mcgrp);
+
+ rd_free(member->id);
+
+ if (member->instance_id)
+ rd_free(member->instance_id);
+
+ RD_IF_FREE(member->target_assignment,
+ rd_kafka_topic_partition_list_destroy);
+ RD_IF_FREE(member->current_assignment,
+ rd_kafka_topic_partition_list_destroy);
+ RD_IF_FREE(member->returned_assignment,
+ rd_kafka_topic_partition_list_destroy);
+ RD_IF_FREE(member->subscribed_topics, rd_list_destroy_free);
+
+ RD_IF_FREE(member->subscribed_topic_names, rd_list_destroy_free);
+
+ RD_IF_FREE(member->subscribed_topic_regex, rd_free);
+
+ rd_free(member);
+}
+
+static void rd_kafka_mock_cgrp_consumer_member_leave_static(
+ rd_kafka_mock_cgrp_consumer_member_t *member) {
+ member->left_static_membership = rd_true;
+ rd_kafka_mock_cgrp_consumer_member_returned_assignment_set(member,
+ NULL);
+}
+
+
+/**
+ * @brief Called when a member must leave a consumer group.
+ *
+ * @param mcgrp Consumer group to leave.
+ * @param member Member that leaves.
+ * @param leave_static If true, the member is leaving with static group
+ * membership.
+ *
+ * @locks mcluster->lock MUST be held.
+ */
+void rd_kafka_mock_cgrp_consumer_member_leave(
+ rd_kafka_mock_cgrp_consumer_t *mcgrp,
+ rd_kafka_mock_cgrp_consumer_member_t *member,
+ rd_bool_t leave_static) {
+ rd_bool_t is_static = member->instance_id != NULL;
+
+ rd_kafka_dbg(mcgrp->cluster->rk, MOCK, "MOCK",
+ "Member %s is leaving group %s, is static: %s, "
+ "static leave: %s",
+ member->id, mcgrp->id, RD_STR_ToF(is_static),
+ RD_STR_ToF(leave_static));
+ if (!is_static || !leave_static)
+ rd_kafka_mock_cgrp_consumer_member_destroy(mcgrp, member);
+ else
+ rd_kafka_mock_cgrp_consumer_member_leave_static(member);
+}
+
+/**
+ * @brief Called when a member is fenced from a consumer group.
+ *
+ * @param mcgrp Consumer group.
+ * @param member Member to fence.
+ *
+ * @locks mcluster->lock MUST be held.
+ */
+void rd_kafka_mock_cgrp_consumer_member_fenced(
+ rd_kafka_mock_cgrp_consumer_t *mcgrp,
+ rd_kafka_mock_cgrp_consumer_member_t *member) {
+
+ rd_kafka_dbg(mcgrp->cluster->rk, MOCK, "MOCK",
+ "Member %s is fenced from group %s", member->id,
+ mcgrp->id);
+
+ rd_kafka_mock_cgrp_consumer_member_destroy(mcgrp, member);
+}
+
+/**
+ * @brief Find a consumer group in cluster \p mcluster by \p GroupId.
+ *
+ * @param mcluster Cluster to search in.
+ * @param GroupId Group id to search.
+ * @return Found group or NULL.
+ *
+ * @locks mcluster->lock MUST be held.
+ */
+rd_kafka_mock_cgrp_consumer_t *
+rd_kafka_mock_cgrp_consumer_find(const rd_kafka_mock_cluster_t *mcluster,
+ const rd_kafkap_str_t *GroupId) {
+ rd_kafka_mock_cgrp_consumer_t *mcgrp;
+ TAILQ_FOREACH(mcgrp, &mcluster->cgrps_consumer, link) {
+ if (!rd_kafkap_str_cmp_str(GroupId, mcgrp->id))
+ return mcgrp;
+ }
+
+ return NULL;
+}
+
+/**
+ * @brief Check if any members have exceeded the session timeout.
+ *
+ * @param rkts Timers.
+ * @param arg Consumer group.
+ *
+ * @locks mcluster->lock is acquired and released.
+ */
+static void rd_kafka_mock_cgrp_consumer_session_tmr_cb(rd_kafka_timers_t *rkts,
+ void *arg) {
+ rd_kafka_mock_cgrp_consumer_t *mcgrp = arg;
+ rd_kafka_mock_cgrp_consumer_member_t *member, *tmp;
+ rd_ts_t now = rd_clock();
+ rd_kafka_mock_cluster_t *mcluster = mcgrp->cluster;
+
+ mtx_unlock(&mcluster->lock);
+ TAILQ_FOREACH_SAFE(member, &mcgrp->members, link, tmp) {
+ if (member->ts_last_activity +
+ (mcgrp->session_timeout_ms * 1000) >
+ now)
+ continue;
+
+ rd_kafka_dbg(mcgrp->cluster->rk, MOCK, "MOCK",
+ "Member %s session timed out for group %s",
+ member->id, mcgrp->id);
+
+ rd_kafka_mock_cgrp_consumer_member_fenced(mcgrp, member);
+ }
+ mtx_unlock(&mcluster->lock);
+}
+
+
+/**
+ * @brief Find or create a "consumer" consumer group.
+ *
+ * @param mcluster Cluster to search in.
+ * @param GroupId Group id to look for.
+ * @return Found or new consumer group.
+ *
+ * @locks mcluster->lock MUST be held.
+ */
+rd_kafka_mock_cgrp_consumer_t *
+rd_kafka_mock_cgrp_consumer_get(rd_kafka_mock_cluster_t *mcluster,
+ const rd_kafkap_str_t *GroupId) {
+ rd_kafka_mock_cgrp_consumer_t *mcgrp;
+
+ mcgrp = rd_kafka_mock_cgrp_consumer_find(mcluster, GroupId);
+ if (mcgrp)
+ return mcgrp;
+
+ mcgrp = rd_calloc(1, sizeof(*mcgrp));
+ mcgrp->cluster = mcluster;
+ mcgrp->id = RD_KAFKAP_STR_DUP(GroupId);
+ mcgrp->group_epoch = 1;
+ TAILQ_INIT(&mcgrp->members);
+ rd_kafka_timer_start(&mcluster->timers, &mcgrp->session_tmr,
+ 1000 * 1000 /*1s*/,
+ rd_kafka_mock_cgrp_consumer_session_tmr_cb, mcgrp);
+
+ TAILQ_INSERT_TAIL(&mcluster->cgrps_consumer, mcgrp, link);
+
+ return mcgrp;
+}
+
+
+void rd_kafka_mock_cgrp_consumer_target_assignment(
+ rd_kafka_mock_cluster_t *mcluster,
+ const char *group_id,
+ rd_kafka_mock_cgrp_consumer_target_assignment_t *target_assignment) {
+ rd_kafka_mock_cgrp_consumer_t *mcgrp;
+ rd_kafkap_str_t *group_id_str =
+ rd_kafkap_str_new(group_id, strlen(group_id));
+
+ mtx_lock(&mcluster->lock);
+
+ mcgrp = rd_kafka_mock_cgrp_consumer_find(mcluster, group_id_str);
+ if (!mcgrp)
+ goto destroy;
+
+ rd_kafka_mock_cgrp_consumer_target_assignment_set_manual(
+ mcgrp, target_assignment);
+
+destroy:
+ rd_kafkap_str_destroy(group_id_str);
+ mtx_unlock(&mcluster->lock);
+}
+
+void rd_kafka_mock_set_group_consumer_session_timeout_ms(
+ rd_kafka_mock_cluster_t *mcluster,
+ int group_consumer_session_timeout_ms) {
+ mtx_lock(&mcluster->lock);
+ mcluster->defaults.group_consumer_session_timeout_ms =
+ group_consumer_session_timeout_ms;
+ mtx_unlock(&mcluster->lock);
+}
+
+void rd_kafka_mock_set_group_consumer_heartbeat_interval_ms(
+ rd_kafka_mock_cluster_t *mcluster,
+ int group_consumer_heartbeat_interval_ms) {
+ mtx_lock(&mcluster->lock);
+ mcluster->defaults.group_consumer_heartbeat_interval_ms =
+ group_consumer_heartbeat_interval_ms;
+ mtx_unlock(&mcluster->lock);
+}
+
+/**
+ * @brief A client connection closed, check if any consumer cgrp has any state
+ * for this connection that needs to be cleared.
+ *
+ * @param mcluster Cluster to search in.
+ * @param mconn Connection that was closed.
+ *
+ * @locks mcluster->lock MUST be held.
+ */
+void rd_kafka_mock_cgrps_consumer_connection_closed(
+ rd_kafka_mock_cluster_t *mcluster,
+ rd_kafka_mock_connection_t *mconn) {
+ rd_kafka_mock_cgrp_consumer_t *mcgrp;
+
+ TAILQ_FOREACH(mcgrp, &mcluster->cgrps_consumer, link) {
+ rd_kafka_mock_cgrp_consumer_member_t *member, *tmp;
+ TAILQ_FOREACH_SAFE(member, &mcgrp->members, link, tmp) {
+ if (member->conn == mconn) {
+ member->conn = NULL;
+ rd_kafka_mock_cgrp_consumer_member_returned_assignment_set(
+ member, NULL);
+ rd_kafka_mock_cgrp_consumer_member_current_assignment_set(
+ member, NULL);
+ }
+ }
+ }
+}
+
+/**
+ * @brief Destroys consumer group \p mcgrp and all of its members.
+ *
+ * @param mcgrp Consumer group to destroy.
+ *
+ * @locks mcluster->lock MUST be held.
+ */
+void rd_kafka_mock_cgrp_consumer_destroy(rd_kafka_mock_cgrp_consumer_t *mcgrp) {
+ rd_kafka_mock_cgrp_consumer_member_t *member;
+
+ TAILQ_REMOVE(&mcgrp->cluster->cgrps_consumer, mcgrp, link);
+
+ rd_kafka_timer_stop(&mcgrp->cluster->timers, &mcgrp->session_tmr,
+ rd_true);
+ rd_free(mcgrp->id);
+ while ((member = TAILQ_FIRST(&mcgrp->members)))
+ rd_kafka_mock_cgrp_consumer_member_destroy(mcgrp, member);
+ rd_free(mcgrp);
+}
+
+/**
+ * @brief A client connection closed, check if any cgrp has any state
+ * for this connection that needs to be cleared.
+ *
+ * @param mcluster Mock cluster.
+ * @param mconn Connection that was closed.
+ */
+void rd_kafka_mock_cgrps_connection_closed(rd_kafka_mock_cluster_t *mcluster,
+ rd_kafka_mock_connection_t *mconn) {
+ rd_kafka_mock_cgrps_classic_connection_closed(mcluster, mconn);
+ rd_kafka_mock_cgrps_consumer_connection_closed(mcluster, mconn);
+}
diff --git a/src/rdkafka_mock_handlers.c b/src/rdkafka_mock_handlers.c
index 45626b5381..ad509ecceb 100644
--- a/src/rdkafka_mock_handlers.c
+++ b/src/rdkafka_mock_handlers.c
@@ -290,13 +290,26 @@ void rd_kafka_mock_Fetch_reply_tags_partition_write(
rd_kafka_mock_partition_t *mpart) {
switch (tagtype) {
case 1: /* CurrentLeader */
+ {
+ int32_t leader_id = mpart->leader->id,
+ leader_epoch = mpart->leader_epoch;
+ rd_kafka_mock_partition_leader_t *mpart_leader =
+ rd_kafka_mock_partition_next_leader_response(mpart);
+ if (mpart_leader) {
+ leader_id = mpart_leader->leader_id;
+ leader_epoch = mpart_leader->leader_epoch;
+ rd_kafka_mock_partition_leader_destroy(mpart,
+ mpart_leader);
+ }
+
/* Leader id */
- rd_kafka_buf_write_i32(rkbuf, mpart->leader->id);
+ rd_kafka_buf_write_i32(rkbuf, leader_id);
/* Leader epoch */
- rd_kafka_buf_write_i32(rkbuf, mpart->leader_epoch);
+ rd_kafka_buf_write_i32(rkbuf, leader_epoch);
/* Field tags */
rd_kafka_buf_write_tags_empty(rkbuf);
break;
+ }
default:
break;
}
@@ -989,7 +1002,7 @@ static int rd_kafka_mock_handle_OffsetCommit(rd_kafka_mock_connection_t *mconn,
rd_kafka_buf_t *resp = rd_kafka_mock_buf_new_response(rkbuf);
rd_kafka_mock_broker_t *mrkb;
rd_kafka_resp_err_t all_err;
- int32_t GenerationId = -1, TopicsCnt;
+ int32_t GenerationIdOrMemberEpoch = -1, TopicsCnt;
rd_kafkap_str_t GroupId, MemberId, GroupInstanceId;
if (rkbuf->rkbuf_reqhdr.ApiVersion >= 3) {
@@ -1000,7 +1013,7 @@ static int rd_kafka_mock_handle_OffsetCommit(rd_kafka_mock_connection_t *mconn,
rd_kafka_buf_read_str(rkbuf, &GroupId);
if (rkbuf->rkbuf_reqhdr.ApiVersion >= 1) {
- rd_kafka_buf_read_i32(rkbuf, &GenerationId);
+ rd_kafka_buf_read_i32(rkbuf, &GenerationIdOrMemberEpoch);
rd_kafka_buf_read_str(rkbuf, &MemberId);
}
@@ -1024,24 +1037,50 @@ static int rd_kafka_mock_handle_OffsetCommit(rd_kafka_mock_connection_t *mconn,
if (!all_err) {
- rd_kafka_mock_cgrp_t *mcgrp;
+ rd_kafka_mock_cgrp_classic_t *mcgrp_classic;
- mcgrp = rd_kafka_mock_cgrp_find(mcluster, &GroupId);
- if (mcgrp) {
- rd_kafka_mock_cgrp_member_t *member = NULL;
+ mcgrp_classic =
+ rd_kafka_mock_cgrp_classic_find(mcluster, &GroupId);
+ if (mcgrp_classic) {
+ rd_kafka_mock_cgrp_classic_member_t *member = NULL;
if (!RD_KAFKAP_STR_IS_NULL(&MemberId))
- member = rd_kafka_mock_cgrp_member_find(
- mcgrp, &MemberId);
+ member = rd_kafka_mock_cgrp_classic_member_find(
+ mcgrp_classic, &MemberId);
if (!member)
all_err = RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID;
else
- all_err = rd_kafka_mock_cgrp_check_state(
- mcgrp, member, rkbuf, GenerationId);
+ all_err =
+ rd_kafka_mock_cgrp_classic_check_state(
+ mcgrp_classic, member, rkbuf,
+ GenerationIdOrMemberEpoch);
+ } else {
+ rd_kafka_mock_cgrp_consumer_t *mcgrp_consumer;
+ rd_kafka_mock_cgrp_consumer_member_t *member = NULL;
+
+ mcgrp_consumer = rd_kafka_mock_cgrp_consumer_find(
+ mcluster, &GroupId);
+ if (mcgrp_consumer) {
+ if (!RD_KAFKAP_STR_IS_NULL(&MemberId))
+ member =
+ rd_kafka_mock_cgrp_consumer_member_find(
+ mcgrp_consumer, &MemberId);
+
+ if (!member)
+ all_err =
+ RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID;
+ else
+ all_err =
+ GenerationIdOrMemberEpoch !=
+ member->current_member_epoch
+ ? RD_KAFKA_RESP_ERR_STALE_MEMBER_EPOCH
+ : RD_KAFKA_RESP_ERR_NO_ERROR;
+ }
}
- /* FIXME: also check that partitions are assigned to member */
+ /* As happens here, a real broker doesn't check that partitions
+ * are assigned to the member, but only the GenerationId. */
}
rd_kafka_buf_read_arraycnt(rkbuf, &TopicsCnt, RD_KAFKAP_TOPICS_MAX);
@@ -1258,6 +1297,8 @@ static int rd_kafka_mock_handle_Metadata(rd_kafka_mock_connection_t *mconn,
rd_bool_t list_all_topics = rd_false;
int32_t TopicsCnt;
int i;
+ size_t of_Brokers_cnt;
+ int32_t response_Brokers_cnt = 0;
if (rkbuf->rkbuf_reqhdr.ApiVersion >= 3) {
/* Response: ThrottleTime */
@@ -1265,9 +1306,11 @@ static int rd_kafka_mock_handle_Metadata(rd_kafka_mock_connection_t *mconn,
}
/* Response: #Brokers */
- rd_kafka_buf_write_arraycnt(resp, mcluster->broker_cnt);
+ of_Brokers_cnt = rd_kafka_buf_write_arraycnt_pos(resp);
TAILQ_FOREACH(mrkb, &mcluster->brokers, link) {
+ if (!mrkb->up)
+ continue;
/* Response: Brokers.Nodeid */
rd_kafka_buf_write_i32(resp, mrkb->id);
/* Response: Brokers.Host */
@@ -1279,7 +1322,10 @@ static int rd_kafka_mock_handle_Metadata(rd_kafka_mock_connection_t *mconn,
rd_kafka_buf_write_str(resp, mrkb->rack, -1);
}
rd_kafka_buf_write_tags_empty(resp);
+ response_Brokers_cnt++;
}
+ rd_kafka_buf_finalize_arraycnt(resp, of_Brokers_cnt,
+ response_Brokers_cnt);
if (rkbuf->rkbuf_reqhdr.ApiVersion >= 2) {
/* Response: ClusterId */
@@ -1524,8 +1570,8 @@ static int rd_kafka_mock_handle_JoinGroup(rd_kafka_mock_connection_t *mconn,
int32_t ProtocolCnt = 0;
int32_t i;
rd_kafka_resp_err_t err;
- rd_kafka_mock_cgrp_t *mcgrp;
- rd_kafka_mock_cgrp_proto_t *protos = NULL;
+ rd_kafka_mock_cgrp_classic_t *mcgrp;
+ rd_kafka_mock_cgrp_classic_proto_t *protos = NULL;
rd_kafka_buf_read_str(rkbuf, &GroupId);
rd_kafka_buf_read_i32(rkbuf, &SessionTimeoutMs);
@@ -1578,13 +1624,13 @@ static int rd_kafka_mock_handle_JoinGroup(rd_kafka_mock_connection_t *mconn,
}
if (!err) {
- mcgrp =
- rd_kafka_mock_cgrp_get(mcluster, &GroupId, &ProtocolType);
+ mcgrp = rd_kafka_mock_cgrp_classic_get(mcluster, &GroupId,
+ &ProtocolType);
rd_assert(mcgrp);
/* This triggers an async rebalance, the response will be
* sent later. */
- err = rd_kafka_mock_cgrp_member_add(
+ err = rd_kafka_mock_cgrp_classic_member_add(
mcgrp, mconn, resp, &MemberId, &ProtocolType,
&GroupInstanceId, protos, ProtocolCnt, SessionTimeoutMs);
if (!err) {
@@ -1595,7 +1641,7 @@ static int rd_kafka_mock_handle_JoinGroup(rd_kafka_mock_connection_t *mconn,
}
}
- rd_kafka_mock_cgrp_protos_destroy(protos, ProtocolCnt);
+ rd_kafka_mock_cgrp_classic_protos_destroy(protos, ProtocolCnt);
/* Error case */
rd_kafka_buf_write_i16(resp, err); /* ErrorCode */
@@ -1612,7 +1658,7 @@ static int rd_kafka_mock_handle_JoinGroup(rd_kafka_mock_connection_t *mconn,
err_parse:
rd_kafka_buf_destroy(resp);
if (protos)
- rd_kafka_mock_cgrp_protos_destroy(protos, ProtocolCnt);
+ rd_kafka_mock_cgrp_classic_protos_destroy(protos, ProtocolCnt);
return -1;
}
@@ -1630,8 +1676,8 @@ static int rd_kafka_mock_handle_Heartbeat(rd_kafka_mock_connection_t *mconn,
rd_kafkap_str_t GroupInstanceId = RD_KAFKAP_STR_INITIALIZER;
int32_t GenerationId;
rd_kafka_resp_err_t err;
- rd_kafka_mock_cgrp_t *mcgrp;
- rd_kafka_mock_cgrp_member_t *member = NULL;
+ rd_kafka_mock_cgrp_classic_t *mcgrp;
+ rd_kafka_mock_cgrp_classic_member_t *member = NULL;
rd_kafka_buf_read_str(rkbuf, &GroupId);
rd_kafka_buf_read_i32(rkbuf, &GenerationId);
@@ -1660,23 +1706,24 @@ static int rd_kafka_mock_handle_Heartbeat(rd_kafka_mock_connection_t *mconn,
}
if (!err) {
- mcgrp = rd_kafka_mock_cgrp_find(mcluster, &GroupId);
+ mcgrp = rd_kafka_mock_cgrp_classic_find(mcluster, &GroupId);
if (!mcgrp)
err = RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND;
}
if (!err) {
- member = rd_kafka_mock_cgrp_member_find(mcgrp, &MemberId);
+ member =
+ rd_kafka_mock_cgrp_classic_member_find(mcgrp, &MemberId);
if (!member)
err = RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID;
}
if (!err)
- err = rd_kafka_mock_cgrp_check_state(mcgrp, member, rkbuf,
- GenerationId);
+ err = rd_kafka_mock_cgrp_classic_check_state(
+ mcgrp, member, rkbuf, GenerationId);
if (!err)
- rd_kafka_mock_cgrp_member_active(mcgrp, member);
+ rd_kafka_mock_cgrp_classic_member_active(mcgrp, member);
rd_kafka_buf_write_i16(resp, err); /* ErrorCode */
@@ -1701,8 +1748,8 @@ static int rd_kafka_mock_handle_LeaveGroup(rd_kafka_mock_connection_t *mconn,
rd_kafka_buf_t *resp = rd_kafka_mock_buf_new_response(rkbuf);
rd_kafkap_str_t GroupId, MemberId;
rd_kafka_resp_err_t err;
- rd_kafka_mock_cgrp_t *mcgrp;
- rd_kafka_mock_cgrp_member_t *member = NULL;
+ rd_kafka_mock_cgrp_classic_t *mcgrp;
+ rd_kafka_mock_cgrp_classic_member_t *member = NULL;
rd_kafka_buf_read_str(rkbuf, &GroupId);
rd_kafka_buf_read_str(rkbuf, &MemberId);
@@ -1729,22 +1776,24 @@ static int rd_kafka_mock_handle_LeaveGroup(rd_kafka_mock_connection_t *mconn,
}
if (!err) {
- mcgrp = rd_kafka_mock_cgrp_find(mcluster, &GroupId);
+ mcgrp = rd_kafka_mock_cgrp_classic_find(mcluster, &GroupId);
if (!mcgrp)
err = RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND;
}
if (!err) {
- member = rd_kafka_mock_cgrp_member_find(mcgrp, &MemberId);
+ member =
+ rd_kafka_mock_cgrp_classic_member_find(mcgrp, &MemberId);
if (!member)
err = RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID;
}
if (!err)
- err = rd_kafka_mock_cgrp_check_state(mcgrp, member, rkbuf, -1);
+ err = rd_kafka_mock_cgrp_classic_check_state(mcgrp, member,
+ rkbuf, -1);
if (!err)
- rd_kafka_mock_cgrp_member_leave(mcgrp, member);
+ rd_kafka_mock_cgrp_classic_member_leave(mcgrp, member);
rd_kafka_buf_write_i16(resp, err); /* ErrorCode */
@@ -1773,8 +1822,8 @@ static int rd_kafka_mock_handle_SyncGroup(rd_kafka_mock_connection_t *mconn,
int32_t GenerationId, AssignmentCnt;
int32_t i;
rd_kafka_resp_err_t err;
- rd_kafka_mock_cgrp_t *mcgrp = NULL;
- rd_kafka_mock_cgrp_member_t *member = NULL;
+ rd_kafka_mock_cgrp_classic_t *mcgrp = NULL;
+ rd_kafka_mock_cgrp_classic_member_t *member = NULL;
rd_kafka_buf_read_str(rkbuf, &GroupId);
rd_kafka_buf_read_i32(rkbuf, &GenerationId);
@@ -1804,23 +1853,24 @@ static int rd_kafka_mock_handle_SyncGroup(rd_kafka_mock_connection_t *mconn,
}
if (!err) {
- mcgrp = rd_kafka_mock_cgrp_find(mcluster, &GroupId);
+ mcgrp = rd_kafka_mock_cgrp_classic_find(mcluster, &GroupId);
if (!mcgrp)
err = RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND;
}
if (!err) {
- member = rd_kafka_mock_cgrp_member_find(mcgrp, &MemberId);
+ member =
+ rd_kafka_mock_cgrp_classic_member_find(mcgrp, &MemberId);
if (!member)
err = RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID;
}
if (!err)
- err = rd_kafka_mock_cgrp_check_state(mcgrp, member, rkbuf,
- GenerationId);
+ err = rd_kafka_mock_cgrp_classic_check_state(
+ mcgrp, member, rkbuf, GenerationId);
if (!err)
- rd_kafka_mock_cgrp_member_active(mcgrp, member);
+ rd_kafka_mock_cgrp_classic_member_active(mcgrp, member);
if (!err) {
rd_bool_t is_leader = mcgrp->leader && mcgrp->leader == member;
@@ -1836,7 +1886,7 @@ static int rd_kafka_mock_handle_SyncGroup(rd_kafka_mock_connection_t *mconn,
for (i = 0; i < AssignmentCnt; i++) {
rd_kafkap_str_t MemberId2;
rd_kafkap_bytes_t Metadata;
- rd_kafka_mock_cgrp_member_t *member2;
+ rd_kafka_mock_cgrp_classic_member_t *member2;
rd_kafka_buf_read_str(rkbuf, &MemberId2);
rd_kafka_buf_read_kbytes(rkbuf, &Metadata);
@@ -1845,17 +1895,18 @@ static int rd_kafka_mock_handle_SyncGroup(rd_kafka_mock_connection_t *mconn,
continue;
/* Find member */
- member2 = rd_kafka_mock_cgrp_member_find(mcgrp, &MemberId2);
+ member2 =
+ rd_kafka_mock_cgrp_classic_member_find(mcgrp, &MemberId2);
if (!member2)
continue;
- rd_kafka_mock_cgrp_member_assignment_set(mcgrp, member2,
- &Metadata);
+ rd_kafka_mock_cgrp_classic_member_assignment_set(mcgrp, member2,
+ &Metadata);
}
if (!err) {
- err = rd_kafka_mock_cgrp_member_sync_set(mcgrp, member, mconn,
- resp);
+ err = rd_kafka_mock_cgrp_classic_member_sync_set(mcgrp, member,
+ mconn, resp);
/* .._sync_set() assumes ownership of resp */
if (!err)
return 0; /* Response will be sent when all members
@@ -2645,6 +2696,7 @@ static int rd_kafka_mock_handle_PushTelemetry(rd_kafka_mock_connection_t *mconn,
rd_kafka_Uuid_t ClientInstanceId;
int32_t SubscriptionId;
rd_bool_t terminating;
+ int8_t CompressionType;
rd_kafka_compression_t compression_type = RD_KAFKA_COMPRESSION_NONE;
rd_kafkap_bytes_t metrics;
rd_kafka_resp_err_t err;
@@ -2652,11 +2704,13 @@ static int rd_kafka_mock_handle_PushTelemetry(rd_kafka_mock_connection_t *mconn,
rd_kafka_buf_read_uuid(rkbuf, &ClientInstanceId);
rd_kafka_buf_read_i32(rkbuf, &SubscriptionId);
rd_kafka_buf_read_bool(rkbuf, &terminating);
- rd_kafka_buf_read_i8(rkbuf, &compression_type);
+ rd_kafka_buf_read_i8(rkbuf, &CompressionType);
+ compression_type = CompressionType;
rd_kafka_buf_read_kbytes(rkbuf, &metrics);
void *uncompressed_payload = NULL;
size_t uncompressed_payload_len = 0;
+ rd_assert(metrics.data != NULL);
if (compression_type != RD_KAFKA_COMPRESSION_NONE) {
rd_rkb_log(rkb, LOG_DEBUG, "MOCKTELEMETRY",
@@ -2678,6 +2732,7 @@ static int rd_kafka_mock_handle_PushTelemetry(rd_kafka_mock_connection_t *mconn,
uncompressed_payload_len = metrics.len;
}
+ rd_assert(uncompressed_payload != NULL);
rd_kafka_mock_handle_PushTelemetry_payload(rkb, uncompressed_payload,
uncompressed_payload_len);
if (compression_type != RD_KAFKA_COMPRESSION_NONE)
@@ -2698,6 +2753,245 @@ static int rd_kafka_mock_handle_PushTelemetry(rd_kafka_mock_connection_t *mconn,
rd_kafka_buf_destroy(resp);
return -1;
}
+static void rd_kafka_mock_handle_ConsumerGroupHeartbeat_write_TopicPartitions(
+ rd_kafka_buf_t *rkbuf,
+ rd_kafka_topic_partition_list_t *rktparlist) {
+ const rd_kafka_topic_partition_field_t fields[] = {
+ RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION,
+ RD_KAFKA_TOPIC_PARTITION_FIELD_END};
+ rd_kafka_topic_partition_list_sort_by_topic_id(rktparlist);
+ rd_kafka_buf_write_topic_partitions(
+ rkbuf, rktparlist, rd_false /*don't skip invalid offsets*/,
+ rd_false /*any offset*/, rd_true /* use_topic id */,
+ rd_false /* don't use topic name */, fields);
+}
+
+static int
+rd_kafka_mock_handle_ConsumerGroupHeartbeat(rd_kafka_mock_connection_t *mconn,
+ rd_kafka_buf_t *rkbuf) {
+ const rd_bool_t log_decode_errors = rd_true;
+ rd_bool_t sent_assignment_parse_err = rd_false;
+ rd_kafka_mock_cluster_t *mcluster = mconn->broker->cluster;
+ rd_kafka_buf_t *resp = rd_kafka_mock_buf_new_response(rkbuf);
+ rd_kafka_topic_partition_list_t *sent_assignment = NULL,
+ *existing_assignment = NULL,
+ *next_assignment = NULL;
+ rd_kafka_topic_partition_t *rktpar;
+ rd_kafkap_str_t GroupId, MemberId, InstanceId, RackId, ServerAssignor,
+ SubscribedTopicRegex;
+ rd_kafkap_str_t *SubscribedTopicNames = NULL;
+ int32_t MemberEpoch, RebalanceTimeoutMs, SubscribedTopicNamesCnt;
+ int32_t i;
+ rd_kafka_resp_err_t err;
+ rd_kafka_mock_cgrp_consumer_t *mcgrp = NULL;
+ rd_kafka_mock_broker_t *mrkb = NULL;
+ rd_kafka_mock_cgrp_consumer_member_t *member = NULL;
+
+ /* GroupId */
+ rd_kafka_buf_read_str(rkbuf, &GroupId);
+ rd_assert(!RD_KAFKAP_STR_IS_NULL(&GroupId));
+
+ /* MemberId */
+ rd_kafka_buf_read_str(rkbuf, &MemberId);
+ rd_assert(!RD_KAFKAP_STR_IS_NULL(&MemberId));
+
+ /* MemberEpoch */
+ rd_kafka_buf_read_i32(rkbuf, &MemberEpoch);
+ rd_assert(MemberEpoch >= -2);
+
+ /* InstanceId */
+ rd_kafka_buf_read_str(rkbuf, &InstanceId);
+
+ /* RackId */
+ rd_kafka_buf_read_str(rkbuf, &RackId);
+
+ /* RebalanceTimeoutMs */
+ rd_kafka_buf_read_i32(rkbuf, &RebalanceTimeoutMs);
+ rd_assert(RebalanceTimeoutMs >= -1);
+
+ /* #SubscribedTopicNames */
+ rd_kafka_buf_read_arraycnt(rkbuf, &SubscribedTopicNamesCnt,
+ RD_KAFKAP_TOPICS_MAX);
+ if (SubscribedTopicNamesCnt >= 0) {
+ SubscribedTopicNames = rd_calloc(
+ SubscribedTopicNamesCnt > 0 ? SubscribedTopicNamesCnt : 1,
+ sizeof(rd_kafkap_str_t));
+ for (i = 0; i < SubscribedTopicNamesCnt; i++) {
+ /* SubscribedTopicNames[i] */
+ rd_kafka_buf_read_str(rkbuf, &SubscribedTopicNames[i]);
+ }
+ }
+
+ rd_kafka_buf_read_str(rkbuf, &SubscribedTopicRegex);
+
+ /* ServerAssignor */
+ rd_kafka_buf_read_str(rkbuf, &ServerAssignor);
+
+ /* #TopicPartitions */
+ const rd_kafka_topic_partition_field_t sent_assignment_fields[] = {
+ RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION,
+ RD_KAFKA_TOPIC_PARTITION_FIELD_END};
+ sent_assignment = rd_kafka_buf_read_topic_partitions_nullable(
+ rkbuf, rd_true, rd_false, 0, sent_assignment_fields,
+ &sent_assignment_parse_err);
+ if (sent_assignment_parse_err)
+ goto err_parse;
+
+ if (sent_assignment) {
+ rd_kafka_Uuid_t last_topic_id = RD_KAFKA_UUID_ZERO;
+ rd_kafka_mock_topic_t *mtopic = NULL;
+ existing_assignment =
+ rd_kafka_topic_partition_list_new(sent_assignment->cnt);
+ RD_KAFKA_TPLIST_FOREACH(rktpar, sent_assignment) {
+ rd_kafka_Uuid_t current_topic_id =
+ rd_kafka_topic_partition_get_topic_id(rktpar);
+
+ if (rd_kafka_Uuid_cmp(current_topic_id,
+ last_topic_id) != 0) {
+ last_topic_id = current_topic_id;
+ mtopic = rd_kafka_mock_topic_find_by_id(
+ mcluster, current_topic_id);
+ }
+
+ if (mtopic) {
+ rd_kafka_topic_partition_t *added =
+ rd_kafka_topic_partition_list_add(
+ existing_assignment, "",
+ rktpar->partition);
+ rd_kafka_topic_partition_set_topic_id(
+ added, last_topic_id);
+ }
+ }
+ }
+
+ /* Inject error, if any */
+ err = rd_kafka_mock_next_request_error(mconn, resp);
+
+ if (!err) {
+ mrkb = rd_kafka_mock_cluster_get_coord(
+ mcluster, RD_KAFKA_COORD_GROUP, &GroupId);
+
+ if (!mrkb)
+ err = RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE;
+ else if (mrkb != mconn->broker)
+ err = RD_KAFKA_RESP_ERR_NOT_COORDINATOR;
+ }
+
+ if (!err) {
+ mtx_lock(&mcluster->lock);
+ mcgrp = rd_kafka_mock_cgrp_consumer_get(mcluster, &GroupId);
+ rd_assert(mcgrp);
+
+ member = rd_kafka_mock_cgrp_consumer_member_add(
+ mcgrp, mconn, &MemberId, &InstanceId, SubscribedTopicNames,
+ SubscribedTopicNamesCnt, &SubscribedTopicRegex);
+
+ if (member) {
+ if (MemberEpoch >= 0) {
+ next_assignment =
+ rd_kafka_mock_cgrp_consumer_member_next_assignment(
+ member, existing_assignment,
+ &MemberEpoch);
+ if (MemberEpoch < 0) {
+ err =
+ RD_KAFKA_RESP_ERR_FENCED_MEMBER_EPOCH;
+ }
+ } else {
+ rd_kafka_mock_cgrp_consumer_member_leave(
+ mcgrp, member, MemberEpoch == -2);
+ member = NULL;
+ }
+ } else {
+ err = RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID;
+ }
+ mtx_unlock(&mcluster->lock);
+ } else {
+ switch (err) {
+ case RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID:
+ case RD_KAFKA_RESP_ERR_FENCED_MEMBER_EPOCH:
+ /* In case the error was set
+ * by `rd_kafka_mock_next_request_error`. */
+ MemberEpoch = -1;
+ mtx_lock(&mcluster->lock);
+ mcgrp = rd_kafka_mock_cgrp_consumer_find(mcluster,
+ &GroupId);
+ if (!mcgrp) {
+ mtx_unlock(&mcluster->lock);
+ break;
+ }
+
+ rd_kafka_mock_cgrp_consumer_member_t *member =
+ rd_kafka_mock_cgrp_consumer_member_find(mcgrp,
+ &MemberId);
+ if (member) {
+ rd_kafka_mock_cgrp_consumer_member_fenced(
+ mcgrp, member);
+ member = NULL;
+ }
+ mtx_unlock(&mcluster->lock);
+ default:
+ break;
+ }
+ }
+
+ /*
+ * Construct response
+ */
+ /* Response: Throttle */
+ rd_kafka_buf_write_i32(resp, 0);
+
+ /* Response: ErrorCode */
+ rd_kafka_buf_write_i16(resp, err);
+
+ /* Response: ErrorMessage */
+ rd_kafka_buf_write_str(resp, rd_kafka_err2str(err), -1);
+
+ /* Response: MemberId */
+ if (!err && member)
+ rd_kafka_buf_write_str(resp, member->id, -1);
+ else
+ rd_kafka_buf_write_str(resp, NULL, -1);
+
+ /* Response: MemberEpoch */
+ rd_kafka_buf_write_i32(resp, MemberEpoch);
+
+ /* Response: HeartbeatIntervalMs */
+ if (mcgrp) {
+ rd_kafka_buf_write_i32(resp, mcgrp->heartbeat_interval_ms);
+ } else {
+ rd_kafka_buf_write_i32(resp, 0);
+ }
+
+ if (next_assignment) {
+ /* Response: Assignment */
+ rd_kafka_buf_write_i8(resp, 1);
+
+ /* Response: TopicPartitions */
+ rd_kafka_mock_handle_ConsumerGroupHeartbeat_write_TopicPartitions(
+ resp, next_assignment);
+
+ rd_kafka_buf_write_tags_empty(resp);
+ } else {
+ /* Response: Assignment */
+ rd_kafka_buf_write_i8(resp, -1);
+ }
+
+ rd_kafka_mock_connection_send_response(mconn, resp);
+
+ rd_free(SubscribedTopicNames);
+ RD_IF_FREE(sent_assignment, rd_kafka_topic_partition_list_destroy);
+ RD_IF_FREE(existing_assignment, rd_kafka_topic_partition_list_destroy);
+ RD_IF_FREE(next_assignment, rd_kafka_topic_partition_list_destroy);
+ return 0;
+
+err_parse:
+ RD_IF_FREE(SubscribedTopicNames, rd_free);
+ RD_IF_FREE(sent_assignment, rd_kafka_topic_partition_list_destroy);
+ RD_IF_FREE(existing_assignment, rd_kafka_topic_partition_list_destroy);
+ RD_IF_FREE(next_assignment, rd_kafka_topic_partition_list_destroy);
+ rd_kafka_buf_destroy(resp);
+ return -1;
+}
/**
* @brief Default request handlers
@@ -2715,7 +3009,7 @@ const struct rd_kafka_mock_api_handler
[RD_KAFKAP_FindCoordinator] = {0, 3, 3,
rd_kafka_mock_handle_FindCoordinator},
[RD_KAFKAP_InitProducerId] = {0, 4, 2,
- rd_kafka_mock_handle_InitProducerId},
+ rd_kafka_mock_handle_InitProducerId},
[RD_KAFKAP_JoinGroup] = {0, 6, 6, rd_kafka_mock_handle_JoinGroup},
[RD_KAFKAP_Heartbeat] = {0, 5, 4, rd_kafka_mock_handle_Heartbeat},
[RD_KAFKAP_LeaveGroup] = {0, 4, 4, rd_kafka_mock_handle_LeaveGroup},
@@ -2729,6 +3023,8 @@ const struct rd_kafka_mock_api_handler
[RD_KAFKAP_EndTxn] = {0, 1, -1, rd_kafka_mock_handle_EndTxn},
[RD_KAFKAP_OffsetForLeaderEpoch] =
{2, 2, -1, rd_kafka_mock_handle_OffsetForLeaderEpoch},
+ [RD_KAFKAP_ConsumerGroupHeartbeat] =
+ {1, 1, 1, rd_kafka_mock_handle_ConsumerGroupHeartbeat},
[RD_KAFKAP_GetTelemetrySubscriptions] =
{0, 0, 0, rd_kafka_mock_handle_GetTelemetrySubscriptions},
[RD_KAFKAP_PushTelemetry] = {0, 0, 0,
diff --git a/src/rdkafka_mock_int.h b/src/rdkafka_mock_int.h
index b1560f4214..2ef7a2a339 100644
--- a/src/rdkafka_mock_int.h
+++ b/src/rdkafka_mock_int.h
@@ -30,6 +30,8 @@
#ifndef _RDKAFKA_MOCK_INT_H_
#define _RDKAFKA_MOCK_INT_H_
+#include "rdkafka_request.h"
+
/**
* @name Mock cluster - internal data types
*
@@ -65,33 +67,33 @@ typedef TAILQ_HEAD(rd_kafka_mock_error_stack_head_s,
/**
* @struct Consumer group protocol name and metadata.
*/
-typedef struct rd_kafka_mock_cgrp_proto_s {
+typedef struct rd_kafka_mock_cgrp_classic_proto_s {
rd_kafkap_str_t *name;
rd_kafkap_bytes_t *metadata;
-} rd_kafka_mock_cgrp_proto_t;
+} rd_kafka_mock_cgrp_classic_proto_t;
/**
* @struct Consumer group member
*/
-typedef struct rd_kafka_mock_cgrp_member_s {
- TAILQ_ENTRY(rd_kafka_mock_cgrp_member_s) link;
+typedef struct rd_kafka_mock_cgrp_classic_member_s {
+ TAILQ_ENTRY(rd_kafka_mock_cgrp_classic_member_s) link;
char *id; /**< MemberId */
char *group_instance_id; /**< Group instance id */
rd_ts_t ts_last_activity; /**< Last activity, e.g., Heartbeat */
- rd_kafka_mock_cgrp_proto_t *protos; /**< Protocol names */
- int proto_cnt; /**< Number of protocols */
- rd_kafkap_bytes_t *assignment; /**< Current assignment */
+ rd_kafka_mock_cgrp_classic_proto_t *protos; /**< Protocol names */
+ int proto_cnt; /**< Number of protocols */
+ rd_kafkap_bytes_t *assignment; /**< Current assignment */
rd_kafka_buf_t *resp; /**< Current response buffer */
struct rd_kafka_mock_connection_s *conn; /**< Connection, may be NULL
* if there is no ongoing
* request. */
-} rd_kafka_mock_cgrp_member_t;
+} rd_kafka_mock_cgrp_classic_member_t;
/**
- * @struct Consumer group.
+ * @struct Classic consumer group.
*/
-typedef struct rd_kafka_mock_cgrp_s {
- TAILQ_ENTRY(rd_kafka_mock_cgrp_s) link;
+typedef struct rd_kafka_mock_cgrp_classic_s {
+ TAILQ_ENTRY(rd_kafka_mock_cgrp_classic_s) link;
struct rd_kafka_mock_cluster_s *cluster; /**< Cluster */
struct rd_kafka_mock_connection_s *conn; /**< Connection */
char *id; /**< Group Id */
@@ -99,20 +101,83 @@ typedef struct rd_kafka_mock_cgrp_s {
char *protocol_name; /**< Elected protocol name */
int32_t generation_id; /**< Generation Id */
int session_timeout_ms; /**< Session timeout */
- enum { RD_KAFKA_MOCK_CGRP_STATE_EMPTY, /* No members */
- RD_KAFKA_MOCK_CGRP_STATE_JOINING, /* Members are joining */
- RD_KAFKA_MOCK_CGRP_STATE_SYNCING, /* Syncing assignments */
- RD_KAFKA_MOCK_CGRP_STATE_REBALANCING, /* Rebalance triggered */
- RD_KAFKA_MOCK_CGRP_STATE_UP, /* Group is operational */
+ enum {
+ RD_KAFKA_MOCK_CGRP_STATE_EMPTY, /* No members */
+ RD_KAFKA_MOCK_CGRP_STATE_JOINING, /* Members are joining */
+ RD_KAFKA_MOCK_CGRP_STATE_SYNCING, /* Syncing assignments */
+ RD_KAFKA_MOCK_CGRP_STATE_REBALANCING, /* Rebalance triggered */
+ RD_KAFKA_MOCK_CGRP_STATE_UP, /* Group is operational */
} state; /**< Consumer group state */
rd_kafka_timer_t session_tmr; /**< Session timeout timer */
rd_kafka_timer_t rebalance_tmr; /**< Rebalance state timer */
- TAILQ_HEAD(, rd_kafka_mock_cgrp_member_s) members; /**< Group members */
+ TAILQ_HEAD(, rd_kafka_mock_cgrp_classic_member_s)
+ members; /**< Group members */
int member_cnt; /**< Number of group members */
int last_member_cnt; /**< Mumber of group members at last election */
int assignment_cnt; /**< Number of member assignments in last Sync */
- rd_kafka_mock_cgrp_member_t *leader; /**< Elected leader */
-} rd_kafka_mock_cgrp_t;
+ rd_kafka_mock_cgrp_classic_member_t *leader; /**< Elected leader */
+} rd_kafka_mock_cgrp_classic_t;
+
+
+/**
+ * @struct "Consumer" Consumer group (KIP-848).
+ */
+typedef struct rd_kafka_mock_cgrp_consumer_s {
+ TAILQ_ENTRY(rd_kafka_mock_cgrp_consumer_s) link;
+ struct rd_kafka_mock_cluster_s *cluster; /**< Cluster */
+ char *id; /**< Group Id */
+ int32_t group_epoch; /**< Group epoch */
+ int session_timeout_ms; /**< Session timeout */
+ rd_kafka_timer_t session_tmr; /**< Session timeout timer */
+ int heartbeat_interval_ms; /**< Heartbeat interval */
+ TAILQ_HEAD(, rd_kafka_mock_cgrp_consumer_member_s)
+ members; /**< Group members */
+ int member_cnt; /**< Number of group members */
+ rd_bool_t manual_assignment; /**< Use manual assignment */
+} rd_kafka_mock_cgrp_consumer_t;
+
+
+/**
+ * @struct "Consumer" Consumer group member (KIP-848).
+ */
+typedef struct rd_kafka_mock_cgrp_consumer_member_s {
+ TAILQ_ENTRY(rd_kafka_mock_cgrp_consumer_member_s) link;
+ char *id; /**< MemberId */
+ char *instance_id; /**< Group instance id */
+ rd_ts_t ts_last_activity; /**< Last activity, e.g.,
+ * ConsumerGroupHeartbeat */
+ int32_t current_member_epoch; /**< Current member epoch,
+ * updated only on heartbeat. */
+ int32_t
+ target_member_epoch; /**< Target member epoch,
+ * updated only when calling
+ * rd_kafka_mock_cgrp_consumer_target_assignment.
+ */
+ rd_kafka_topic_partition_list_t
+ *current_assignment; /**< Current assignment,
+ * only updated when reported by the client.
+ */
+ rd_kafka_topic_partition_list_t *
+ target_assignment; /**< Target assignment,
+ * only updated when calling
+ * rd_kafka_mock_cgrp_consumer_target_assignment.
+ */
+ rd_kafka_topic_partition_list_t
+ *returned_assignment; /**< Returned assignment */
+
+ rd_list_t *subscribed_topics; /**< Final list of Subscribed topics after
+ considering regex as well*/
+ rd_list_t *subscribed_topic_names; /**< Subscribed topic names received
+ in the heartbeat */
+ char *subscribed_topic_regex; /**< Subscribed regex */
+
+ rd_bool_t left_static_membership; /**< Member left the group
+ * with static membership. */
+ struct rd_kafka_mock_connection_s *conn; /**< Connection, may be NULL
+ * if there is no ongoing
+ * request. */
+ rd_kafka_mock_cgrp_consumer_t *mcgrp; /**< Consumer group */
+} rd_kafka_mock_cgrp_consumer_member_t;
/**
@@ -360,7 +425,9 @@ struct rd_kafka_mock_cluster_s {
TAILQ_HEAD(, rd_kafka_mock_topic_s) topics;
int topic_cnt;
- TAILQ_HEAD(, rd_kafka_mock_cgrp_s) cgrps;
+ TAILQ_HEAD(, rd_kafka_mock_cgrp_classic_s) cgrps_classic;
+
+ TAILQ_HEAD(, rd_kafka_mock_cgrp_consumer_s) cgrps_consumer;
/** Explicit coordinators (set with mock_set_coordinator()) */
TAILQ_HEAD(, rd_kafka_mock_coord_s) coords;
@@ -394,13 +461,19 @@ struct rd_kafka_mock_cluster_s {
struct {
int partition_cnt; /**< Auto topic create part cnt */
int replication_factor; /**< Auto topic create repl factor */
+ /** Group initial rebalance delay */
+ int32_t group_initial_rebalance_delay_ms;
+ /** Session timeout (KIP 848) */
+ int group_consumer_session_timeout_ms;
+ /** Heartbeat interval (KIP 848) */
+ int group_consumer_heartbeat_interval_ms;
} defaults;
/**< Dynamic array of IO handlers for corresponding fd in .fds */
struct {
rd_kafka_mock_io_handler_t *cb; /**< Callback */
void *opaque; /**< Callbacks' opaque */
- } * handlers;
+ } *handlers;
/**< Per-protocol request error stack. */
rd_kafka_mock_error_stack_head_t errstacks;
@@ -536,47 +609,95 @@ rd_kafka_mock_pid_find(rd_kafka_mock_cluster_t *mcluster,
* @name Mock consumer group (rdkafka_mock_cgrp.c)
* @{
*/
-void rd_kafka_mock_cgrp_member_active(rd_kafka_mock_cgrp_t *mcgrp,
- rd_kafka_mock_cgrp_member_t *member);
-void rd_kafka_mock_cgrp_member_assignment_set(
- rd_kafka_mock_cgrp_t *mcgrp,
- rd_kafka_mock_cgrp_member_t *member,
+void rd_kafka_mock_cgrp_classic_member_active(
+ rd_kafka_mock_cgrp_classic_t *mcgrp,
+ rd_kafka_mock_cgrp_classic_member_t *member);
+void rd_kafka_mock_cgrp_classic_member_assignment_set(
+ rd_kafka_mock_cgrp_classic_t *mcgrp,
+ rd_kafka_mock_cgrp_classic_member_t *member,
const rd_kafkap_bytes_t *Metadata);
-rd_kafka_resp_err_t
-rd_kafka_mock_cgrp_member_sync_set(rd_kafka_mock_cgrp_t *mcgrp,
- rd_kafka_mock_cgrp_member_t *member,
- rd_kafka_mock_connection_t *mconn,
- rd_kafka_buf_t *resp);
-rd_kafka_resp_err_t
-rd_kafka_mock_cgrp_member_leave(rd_kafka_mock_cgrp_t *mcgrp,
- rd_kafka_mock_cgrp_member_t *member);
-void rd_kafka_mock_cgrp_protos_destroy(rd_kafka_mock_cgrp_proto_t *protos,
- int proto_cnt);
-rd_kafka_resp_err_t
-rd_kafka_mock_cgrp_member_add(rd_kafka_mock_cgrp_t *mcgrp,
- rd_kafka_mock_connection_t *mconn,
- rd_kafka_buf_t *resp,
- const rd_kafkap_str_t *MemberId,
- const rd_kafkap_str_t *GroupInstanceId,
- const rd_kafkap_str_t *ProtocolType,
- rd_kafka_mock_cgrp_proto_t *protos,
- int proto_cnt,
- int session_timeout_ms);
-rd_kafka_resp_err_t
-rd_kafka_mock_cgrp_check_state(rd_kafka_mock_cgrp_t *mcgrp,
- rd_kafka_mock_cgrp_member_t *member,
- const rd_kafka_buf_t *request,
- int32_t generation_id);
-rd_kafka_mock_cgrp_member_t *
-rd_kafka_mock_cgrp_member_find(const rd_kafka_mock_cgrp_t *mcgrp,
- const rd_kafkap_str_t *MemberId);
-void rd_kafka_mock_cgrp_destroy(rd_kafka_mock_cgrp_t *mcgrp);
-rd_kafka_mock_cgrp_t *rd_kafka_mock_cgrp_find(rd_kafka_mock_cluster_t *mcluster,
- const rd_kafkap_str_t *GroupId);
-rd_kafka_mock_cgrp_t *
-rd_kafka_mock_cgrp_get(rd_kafka_mock_cluster_t *mcluster,
- const rd_kafkap_str_t *GroupId,
- const rd_kafkap_str_t *ProtocolType);
+rd_kafka_resp_err_t rd_kafka_mock_cgrp_classic_member_sync_set(
+ rd_kafka_mock_cgrp_classic_t *mcgrp,
+ rd_kafka_mock_cgrp_classic_member_t *member,
+ rd_kafka_mock_connection_t *mconn,
+ rd_kafka_buf_t *resp);
+rd_kafka_resp_err_t rd_kafka_mock_cgrp_classic_member_leave(
+ rd_kafka_mock_cgrp_classic_t *mcgrp,
+ rd_kafka_mock_cgrp_classic_member_t *member);
+void rd_kafka_mock_cgrp_classic_protos_destroy(
+ rd_kafka_mock_cgrp_classic_proto_t *protos,
+ int proto_cnt);
+rd_kafka_resp_err_t rd_kafka_mock_cgrp_classic_member_add(
+ rd_kafka_mock_cgrp_classic_t *mcgrp,
+ rd_kafka_mock_connection_t *mconn,
+ rd_kafka_buf_t *resp,
+ const rd_kafkap_str_t *MemberId,
+ const rd_kafkap_str_t *GroupInstanceId,
+ const rd_kafkap_str_t *ProtocolType,
+ rd_kafka_mock_cgrp_classic_proto_t *protos,
+ int proto_cnt,
+ int session_timeout_ms);
+rd_kafka_resp_err_t rd_kafka_mock_cgrp_classic_check_state(
+ rd_kafka_mock_cgrp_classic_t *mcgrp,
+ rd_kafka_mock_cgrp_classic_member_t *member,
+ const rd_kafka_buf_t *request,
+ int32_t generation_id);
+rd_kafka_mock_cgrp_classic_member_t *rd_kafka_mock_cgrp_classic_member_find(
+ const rd_kafka_mock_cgrp_classic_t *mcgrp,
+ const rd_kafkap_str_t *MemberId);
+void rd_kafka_mock_cgrp_classic_destroy(rd_kafka_mock_cgrp_classic_t *mcgrp);
+rd_kafka_mock_cgrp_classic_t *
+rd_kafka_mock_cgrp_classic_find(rd_kafka_mock_cluster_t *mcluster,
+ const rd_kafkap_str_t *GroupId);
+rd_kafka_mock_cgrp_classic_t *
+rd_kafka_mock_cgrp_classic_get(rd_kafka_mock_cluster_t *mcluster,
+ const rd_kafkap_str_t *GroupId,
+ const rd_kafkap_str_t *ProtocolType);
+
+/* "consumer" consumer group (KIP-848) */
+
+rd_kafka_topic_partition_list_t *
+rd_kafka_mock_cgrp_consumer_member_next_assignment(
+ rd_kafka_mock_cgrp_consumer_member_t *member,
+ rd_kafka_topic_partition_list_t *current_assignment,
+ int *member_epoch);
+
+void rd_kafka_mock_cgrp_consumer_member_active(
+ rd_kafka_mock_cgrp_consumer_t *mcgrp,
+ rd_kafka_mock_cgrp_consumer_member_t *member);
+
+void rd_kafka_mock_cgrp_consumer_destroy(rd_kafka_mock_cgrp_consumer_t *mcgrp);
+
+rd_kafka_mock_cgrp_consumer_t *
+rd_kafka_mock_cgrp_consumer_find(const rd_kafka_mock_cluster_t *mcluster,
+ const rd_kafkap_str_t *GroupId);
+
+rd_kafka_mock_cgrp_consumer_t *
+rd_kafka_mock_cgrp_consumer_get(rd_kafka_mock_cluster_t *mcluster,
+ const rd_kafkap_str_t *GroupId);
+
+void rd_kafka_mock_cgrp_consumer_member_leave(
+ rd_kafka_mock_cgrp_consumer_t *mcgrp,
+ rd_kafka_mock_cgrp_consumer_member_t *member,
+ rd_bool_t static_leave);
+
+void rd_kafka_mock_cgrp_consumer_member_fenced(
+ rd_kafka_mock_cgrp_consumer_t *mcgrp,
+ rd_kafka_mock_cgrp_consumer_member_t *member);
+
+rd_kafka_mock_cgrp_consumer_member_t *rd_kafka_mock_cgrp_consumer_member_find(
+ const rd_kafka_mock_cgrp_consumer_t *mcgrp,
+ const rd_kafkap_str_t *MemberId);
+
+rd_kafka_mock_cgrp_consumer_member_t *rd_kafka_mock_cgrp_consumer_member_add(
+ rd_kafka_mock_cgrp_consumer_t *mcgrp,
+ struct rd_kafka_mock_connection_s *conn,
+ const rd_kafkap_str_t *MemberId,
+ const rd_kafkap_str_t *InstanceId,
+ rd_kafkap_str_t *SubscribedTopicNames,
+ int32_t SubscribedTopicNamesCnt,
+ const rd_kafkap_str_t *SubscribedTopicRegex);
+
void rd_kafka_mock_cgrps_connection_closed(rd_kafka_mock_cluster_t *mcluster,
rd_kafka_mock_connection_t *mconn);
/**
diff --git a/src/rdkafka_msg.c b/src/rdkafka_msg.c
index 3fc3967c92..137d29ad81 100644
--- a/src/rdkafka_msg.c
+++ b/src/rdkafka_msg.c
@@ -383,6 +383,7 @@ rd_kafka_produceva(rd_kafka_t *rk, const rd_kafka_vu_t *vus, size_t cnt) {
rd_kafka_error_t *error = NULL;
rd_kafka_headers_t *hdrs = NULL;
rd_kafka_headers_t *app_hdrs = NULL; /* App-provided headers list */
+ int existing = 0;
size_t i;
if (unlikely(rd_kafka_check_produce(rk, &error)))
@@ -392,8 +393,11 @@ rd_kafka_produceva(rd_kafka_t *rk, const rd_kafka_vu_t *vus, size_t cnt) {
const rd_kafka_vu_t *vu = &vus[i];
switch (vu->vtype) {
case RD_KAFKA_VTYPE_TOPIC:
- rkt =
- rd_kafka_topic_new0(rk, vu->u.cstr, NULL, NULL, 1);
+ rkt = rd_kafka_topic_new0(rk, vu->u.cstr, NULL,
+ &existing, 1);
+ if (!existing)
+ rd_kafka_topic_fast_leader_query(
+ rk, rd_true /* force */);
break;
case RD_KAFKA_VTYPE_RKT:
@@ -487,6 +491,8 @@ rd_kafka_produceva(rd_kafka_t *rk, const rd_kafka_vu_t *vus, size_t cnt) {
rd_kafka_err2str(err));
goto err;
}
+ /* 'hdrs' is now owned by 'rkm' */
+ hdrs = NULL;
/* Partition the message */
err = rd_kafka_msg_partitioner(rkt, rkm, 1);
@@ -549,6 +555,7 @@ rd_kafka_resp_err_t rd_kafka_producev(rd_kafka_t *rk, ...) {
rd_kafka_resp_err_t err;
rd_kafka_headers_t *hdrs = NULL;
rd_kafka_headers_t *app_hdrs = NULL; /* App-provided headers list */
+ int existing = 0;
if (unlikely((err = rd_kafka_check_produce(rk, NULL))))
return err;
@@ -559,7 +566,10 @@ rd_kafka_resp_err_t rd_kafka_producev(rd_kafka_t *rk, ...) {
switch (vtype) {
case RD_KAFKA_VTYPE_TOPIC:
rkt = rd_kafka_topic_new0(rk, va_arg(ap, const char *),
- NULL, NULL, 1);
+ NULL, &existing, 1);
+ if (!existing)
+ rd_kafka_topic_fast_leader_query(
+ rk, rd_true /* force */);
break;
case RD_KAFKA_VTYPE_RKT:
@@ -2528,22 +2538,28 @@ int unittest_msg(void) {
{10, 10},
{33692865, 33692865},
{0, 0}});
- fails += unittest_msgq_insert_sort(
- "many messages", insert_baseline, NULL,
- (const struct ut_msg_range[]) {{100000, 200000},
- {400000, 450000},
- {900000, 920000},
- {33692864, 33751992},
- {33906868, 33993690},
- {40000000, 44000000},
- {0, 0}},
- (const struct ut_msg_range[]) {{1, 199},
- {350000, 360000},
- {500000, 500010},
- {1000000, 1000200},
- {33751993, 33906867},
- {50000001, 50000001},
- {0, 0}});
+ if (rd_unittest_with_valgrind) {
+ RD_UT_WARN(
+ "Skipping large message range test "
+ "when using Valgrind");
+ } else {
+ fails += unittest_msgq_insert_sort(
+ "many messages", insert_baseline, NULL,
+ (const struct ut_msg_range[]) {{100000, 200000},
+ {400000, 450000},
+ {900000, 920000},
+ {33692864, 33751992},
+ {33906868, 33993690},
+ {40000000, 44000000},
+ {0, 0}},
+ (const struct ut_msg_range[]) {{1, 199},
+ {350000, 360000},
+ {500000, 500010},
+ {1000000, 1000200},
+ {33751993, 33906867},
+ {50000001, 50000001},
+ {0, 0}});
+ }
fails += unittest_msgq_insert_sort(
"issue #2508", insert_baseline, NULL,
(const struct ut_msg_range[]) {
diff --git a/src/rdkafka_msg.h b/src/rdkafka_msg.h
index 663aa005d6..063fe96b95 100644
--- a/src/rdkafka_msg.h
+++ b/src/rdkafka_msg.h
@@ -535,9 +535,9 @@ rd_kafka_msgq_overlap(const rd_kafka_msgq_t *a, const rd_kafka_msgq_t *b) {
la = rd_kafka_msgq_last(a);
lb = rd_kafka_msgq_last(b);
- return (rd_bool_t)(
- fa->rkm_u.producer.msgid <= lb->rkm_u.producer.msgid &&
- fb->rkm_u.producer.msgid <= la->rkm_u.producer.msgid);
+ return (
+ rd_bool_t)(fa->rkm_u.producer.msgid <= lb->rkm_u.producer.msgid &&
+ fb->rkm_u.producer.msgid <= la->rkm_u.producer.msgid);
}
/**
diff --git a/src/rdkafka_msgset_writer.c b/src/rdkafka_msgset_writer.c
index 6f71d827f2..9ff595bfbe 100644
--- a/src/rdkafka_msgset_writer.c
+++ b/src/rdkafka_msgset_writer.c
@@ -114,9 +114,9 @@ rd_kafka_msgset_writer_select_MsgVersion(rd_kafka_msgset_writer_t *msetw) {
int feature;
int16_t ApiVersion;
} compr_req[RD_KAFKA_COMPRESSION_NUM] = {
- [RD_KAFKA_COMPRESSION_LZ4] = {RD_KAFKA_FEATURE_LZ4, 0},
+ [RD_KAFKA_COMPRESSION_LZ4] = {RD_KAFKA_FEATURE_LZ4, 0},
#if WITH_ZSTD
- [RD_KAFKA_COMPRESSION_ZSTD] = {RD_KAFKA_FEATURE_ZSTD, 7},
+ [RD_KAFKA_COMPRESSION_ZSTD] = {RD_KAFKA_FEATURE_ZSTD, 7},
#endif
};
@@ -145,12 +145,16 @@ rd_kafka_msgset_writer_select_MsgVersion(rd_kafka_msgset_writer_t *msetw) {
* by both client and broker, else disable compression.
*/
if (msetw->msetw_compression &&
- (rd_kafka_broker_ApiVersion_supported(
- rkb, RD_KAFKAP_Produce, 0,
- compr_req[msetw->msetw_compression].ApiVersion, NULL) == -1 ||
- (compr_req[msetw->msetw_compression].feature &&
- !(msetw->msetw_rkb->rkb_features &
- compr_req[msetw->msetw_compression].feature)))) {
+ (
+ /* ApiVersion is smaller than the minimum one */
+ !rd_kafka_broker_ApiVersion_at_least(
+ rkb, RD_KAFKAP_Produce,
+ compr_req[msetw->msetw_compression].ApiVersion) ||
+ /* There's a corresponding feature and it isn't supported
+ * by this broker. */
+ (compr_req[msetw->msetw_compression].feature &&
+ !(msetw->msetw_rkb->rkb_features &
+ compr_req[msetw->msetw_compression].feature)))) {
if (unlikely(
rd_interval(&rkb->rkb_suppress.unsupported_compression,
/* at most once per day */
@@ -705,8 +709,8 @@ rd_kafka_msgset_writer_write_msg_v2(rd_kafka_msgset_writer_t *msetw,
sizeof(varint_OffsetDelta), Offset);
sz_KeyLen = rd_uvarint_enc_i32(varint_KeyLen, sizeof(varint_KeyLen),
rkm->rkm_key
- ? (int32_t)rkm->rkm_key_len
- : (int32_t)RD_KAFKAP_BYTES_LEN_NULL);
+ ? (int32_t)rkm->rkm_key_len
+ : (int32_t)RD_KAFKAP_BYTES_LEN_NULL);
sz_ValueLen = rd_uvarint_enc_i32(
varint_ValueLen, sizeof(varint_ValueLen),
rkm->rkm_payload ? (int32_t)rkm->rkm_len
diff --git a/src/rdkafka_offset.c b/src/rdkafka_offset.c
index 3da38117ac..cf21d60c55 100644
--- a/src/rdkafka_offset.c
+++ b/src/rdkafka_offset.c
@@ -900,7 +900,21 @@ static void rd_kafka_offset_validate_tmr_cb(rd_kafka_timers_t *rkts,
rd_kafka_toppar_t *rktp = arg;
rd_kafka_toppar_lock(rktp);
- rd_kafka_offset_validate(rktp, "retrying offset validation");
+ /* Retry validation only when it's still needed.
+ * Even if validation can be started in fetch states ACTIVE and
+ * VALIDATE_EPOCH_WAIT, its retry should be done only
+ * in fetch state VALIDATE_EPOCH_WAIT. */
+ if (rktp->rktp_fetch_state == RD_KAFKA_TOPPAR_FETCH_VALIDATE_EPOCH_WAIT)
+ rd_kafka_offset_validate(rktp, "retrying offset validation");
+ else {
+ rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, FETCH, "VALIDATE",
+ "%.*s [%" PRId32
+ "]: skipping offset "
+ "validation retry in fetch state %s",
+ RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic),
+ rktp->rktp_partition,
+ rd_kafka_fetch_states[rktp->rktp_fetch_state]);
+ }
rd_kafka_toppar_unlock(rktp);
}
@@ -923,6 +937,9 @@ static void rd_kafka_toppar_handle_OffsetForLeaderEpoch(rd_kafka_t *rk,
rd_kafka_topic_partition_t *rktpar;
int64_t end_offset;
int32_t end_offset_leader_epoch;
+ rd_kafka_toppar_lock(rktp);
+ rktp->rktp_flags &= ~RD_KAFKA_TOPPAR_F_VALIDATING;
+ rd_kafka_toppar_unlock(rktp);
if (err == RD_KAFKA_RESP_ERR__DESTROY) {
rd_kafka_toppar_destroy(rktp); /* Drop refcnt */
@@ -1142,12 +1159,10 @@ void rd_kafka_offset_validate(rd_kafka_toppar_t *rktp, const char *fmt, ...) {
rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, FETCH, "VALIDATE",
"%.*s [%" PRId32
"]: unable to perform offset "
- "validation: partition leader not available",
+ "validation: partition leader not available. "
+ "Retrying when available",
RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic),
rktp->rktp_partition);
-
- rd_kafka_toppar_set_fetch_state(rktp,
- RD_KAFKA_TOPPAR_FETCH_ACTIVE);
return;
}
@@ -1169,8 +1184,21 @@ void rd_kafka_offset_validate(rd_kafka_toppar_t *rktp, const char *fmt, ...) {
return;
}
+ if (rktp->rktp_flags & RD_KAFKA_TOPPAR_F_VALIDATING) {
+ rd_kafka_dbg(
+ rktp->rktp_rkt->rkt_rk, FETCH, "VALIDATE",
+ "%.*s [%" PRId32
+ "]: skipping offset "
+ "validation for %s: validation is already ongoing",
+ RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic),
+ rktp->rktp_partition,
+ rd_kafka_fetch_pos2str(rktp->rktp_offset_validation_pos));
+ return;
+ }
+
rd_kafka_toppar_set_fetch_state(
rktp, RD_KAFKA_TOPPAR_FETCH_VALIDATE_EPOCH_WAIT);
+ rktp->rktp_flags |= RD_KAFKA_TOPPAR_F_VALIDATING;
/* Construct and send OffsetForLeaderEpochRequest */
parts = rd_kafka_topic_partition_list_new(1);
diff --git a/src/rdkafka_op.c b/src/rdkafka_op.c
index 5c2e3023f1..5dbbf9c9d4 100644
--- a/src/rdkafka_op.c
+++ b/src/rdkafka_op.c
@@ -122,7 +122,7 @@ const char *rd_kafka_op2str(rd_kafka_op_type_t type) {
"REPLY:RD_KAFKA_OP_SET_TELEMETRY_BROKER",
[RD_KAFKA_OP_TERMINATE_TELEMETRY] =
"REPLY:RD_KAFKA_OP_TERMINATE_TELEMETRY",
-
+ [RD_KAFKA_OP_ELECTLEADERS] = "REPLY:ELECTLEADERS",
};
if (type & RD_KAFKA_OP_REPLY)
@@ -228,7 +228,7 @@ rd_kafka_op_t *rd_kafka_op_new0(const char *source, rd_kafka_op_type_t type) {
[RD_KAFKA_OP_PARTITION_JOIN] = _RD_KAFKA_OP_EMPTY,
[RD_KAFKA_OP_PARTITION_LEAVE] = _RD_KAFKA_OP_EMPTY,
[RD_KAFKA_OP_REBALANCE] = sizeof(rko->rko_u.rebalance),
- [RD_KAFKA_OP_TERMINATE] = _RD_KAFKA_OP_EMPTY,
+ [RD_KAFKA_OP_TERMINATE] = sizeof(rko->rko_u.terminated),
[RD_KAFKA_OP_COORD_QUERY] = _RD_KAFKA_OP_EMPTY,
[RD_KAFKA_OP_SUBSCRIBE] = sizeof(rko->rko_u.subscribe),
[RD_KAFKA_OP_ASSIGN] = sizeof(rko->rko_u.assign),
@@ -286,6 +286,7 @@ rd_kafka_op_t *rd_kafka_op_new0(const char *source, rd_kafka_op_type_t type) {
[RD_KAFKA_OP_SET_TELEMETRY_BROKER] =
sizeof(rko->rko_u.telemetry_broker),
[RD_KAFKA_OP_TERMINATE_TELEMETRY] = _RD_KAFKA_OP_EMPTY,
+ [RD_KAFKA_OP_ELECTLEADERS] = sizeof(rko->rko_u.admin_request),
};
size_t tsize = op2size[type & ~RD_KAFKA_OP_FLAGMASK];
@@ -439,6 +440,7 @@ void rd_kafka_op_destroy(rd_kafka_op_t *rko) {
case RD_KAFKA_OP_ALTERUSERSCRAMCREDENTIALS:
case RD_KAFKA_OP_DESCRIBEUSERSCRAMCREDENTIALS:
case RD_KAFKA_OP_LISTOFFSETS:
+ case RD_KAFKA_OP_ELECTLEADERS:
rd_kafka_replyq_destroy(&rko->rko_u.admin_request.replyq);
rd_list_destroy(&rko->rko_u.admin_request.args);
if (rko->rko_u.admin_request.options.match_consumer_group_states
@@ -446,6 +448,11 @@ void rd_kafka_op_destroy(rd_kafka_op_t *rko) {
rd_list_destroy(rko->rko_u.admin_request.options
.match_consumer_group_states.u.PTR);
}
+ if (rko->rko_u.admin_request.options.match_consumer_group_types
+ .u.PTR) {
+ rd_list_destroy(rko->rko_u.admin_request.options
+ .match_consumer_group_types.u.PTR);
+ }
rd_assert(!rko->rko_u.admin_request.fanout_parent);
RD_IF_FREE(rko->rko_u.admin_request.coordkey, rd_free);
break;
diff --git a/src/rdkafka_op.h b/src/rdkafka_op.h
index 1bf47b6445..e79309aa02 100644
--- a/src/rdkafka_op.h
+++ b/src/rdkafka_op.h
@@ -186,6 +186,9 @@ typedef enum {
telemetry. */
RD_KAFKA_OP_TERMINATE_TELEMETRY, /**< Start termination sequence for
telemetry. */
+ RD_KAFKA_OP_ELECTLEADERS, /**< Admin:
+ * ElectLeaders
+ * u.admin_request */
RD_KAFKA_OP__END
} rd_kafka_op_type_t;
@@ -397,6 +400,8 @@ struct rd_kafka_op_s {
struct {
rd_kafka_metadata_t *md;
rd_kafka_metadata_internal_t *mdi;
+ /* subscription version for this call */
+ int32_t subscription_version;
int force; /* force request regardless of outstanding
* metadata requests. */
} metadata;
@@ -410,7 +415,6 @@ struct rd_kafka_op_s {
} dr;
struct {
- int32_t nodeid;
char nodename[RD_KAFKA_NODENAME_SIZE];
} node;
@@ -467,13 +471,14 @@ struct rd_kafka_op_s {
struct rd_kafka_admin_worker_cbs *cbs;
/** Worker state */
- enum { RD_KAFKA_ADMIN_STATE_INIT,
- RD_KAFKA_ADMIN_STATE_WAIT_BROKER,
- RD_KAFKA_ADMIN_STATE_WAIT_CONTROLLER,
- RD_KAFKA_ADMIN_STATE_WAIT_FANOUTS,
- RD_KAFKA_ADMIN_STATE_CONSTRUCT_REQUEST,
- RD_KAFKA_ADMIN_STATE_WAIT_RESPONSE,
- RD_KAFKA_ADMIN_STATE_WAIT_BROKER_LIST,
+ enum {
+ RD_KAFKA_ADMIN_STATE_INIT,
+ RD_KAFKA_ADMIN_STATE_WAIT_BROKER,
+ RD_KAFKA_ADMIN_STATE_WAIT_CONTROLLER,
+ RD_KAFKA_ADMIN_STATE_WAIT_FANOUTS,
+ RD_KAFKA_ADMIN_STATE_CONSTRUCT_REQUEST,
+ RD_KAFKA_ADMIN_STATE_WAIT_RESPONSE,
+ RD_KAFKA_ADMIN_STATE_WAIT_BROKER_LIST,
} state;
int32_t broker_id; /**< Requested broker id to
@@ -541,6 +546,11 @@ struct rd_kafka_op_s {
/** Result cb for this op */
void (*result_cb)(rd_kafka_op_t *);
+ struct rd_kafka_admin_worker_cbs
+ *cbs; /**< Worker Callbacks
+ * Moved from admin request
+ */
+
rd_list_t results; /**< Type depends on request type:
*
* (rd_kafka_topic_result_t *):
@@ -567,19 +577,22 @@ struct rd_kafka_op_s {
/**< Mock cluster command */
struct {
- enum { RD_KAFKA_MOCK_CMD_TOPIC_SET_ERROR,
- RD_KAFKA_MOCK_CMD_TOPIC_CREATE,
- RD_KAFKA_MOCK_CMD_PART_SET_LEADER,
- RD_KAFKA_MOCK_CMD_PART_SET_FOLLOWER,
- RD_KAFKA_MOCK_CMD_PART_SET_FOLLOWER_WMARKS,
- RD_KAFKA_MOCK_CMD_PART_PUSH_LEADER_RESPONSE,
- RD_KAFKA_MOCK_CMD_BROKER_SET_UPDOWN,
- RD_KAFKA_MOCK_CMD_BROKER_SET_RTT,
- RD_KAFKA_MOCK_CMD_BROKER_SET_RACK,
- RD_KAFKA_MOCK_CMD_COORD_SET,
- RD_KAFKA_MOCK_CMD_APIVERSION_SET,
- RD_KAFKA_MOCK_CMD_REQUESTED_METRICS_SET,
- RD_KAFKA_MOCK_CMD_TELEMETRY_PUSH_INTERVAL_SET,
+ enum {
+ RD_KAFKA_MOCK_CMD_TOPIC_SET_ERROR,
+ RD_KAFKA_MOCK_CMD_TOPIC_CREATE,
+ RD_KAFKA_MOCK_CMD_PART_SET_LEADER,
+ RD_KAFKA_MOCK_CMD_PART_SET_FOLLOWER,
+ RD_KAFKA_MOCK_CMD_PART_SET_FOLLOWER_WMARKS,
+ RD_KAFKA_MOCK_CMD_PART_PUSH_LEADER_RESPONSE,
+ RD_KAFKA_MOCK_CMD_BROKER_SET_UPDOWN,
+ RD_KAFKA_MOCK_CMD_BROKER_SET_RTT,
+ RD_KAFKA_MOCK_CMD_BROKER_SET_RACK,
+ RD_KAFKA_MOCK_CMD_BROKER_DECOMMISSION,
+ RD_KAFKA_MOCK_CMD_BROKER_ADD,
+ RD_KAFKA_MOCK_CMD_COORD_SET,
+ RD_KAFKA_MOCK_CMD_APIVERSION_SET,
+ RD_KAFKA_MOCK_CMD_REQUESTED_METRICS_SET,
+ RD_KAFKA_MOCK_CMD_TELEMETRY_PUSH_INTERVAL_SET,
} cmd;
rd_kafka_resp_err_t err; /**< Error for:
@@ -607,6 +620,8 @@ struct rd_kafka_op_s {
* PART_SET_LEADER
* BROKER_SET_UPDOWN
* BROKER_SET_RACK
+ * BROKER_DECOMMISSION
+ * BROKER_ADD
* COORD_SET */
int64_t lo; /**< Low offset, for:
* TOPIC_CREATE (part cnt)
@@ -697,6 +712,18 @@ struct rd_kafka_op_s {
rd_kafka_broker_t *rkb;
} telemetry_broker;
+ struct {
+ /**
+ * Terminated and freed broker pointer,
+ * can only be used for pointer comparison.
+ */
+ void *rkb;
+
+ /** Termination callback to trigger
+ * on the op handler's thread. */
+ void (*cb)(rd_kafka_t *rk, void *rkb);
+ } terminated;
+
} rko_u;
};
diff --git a/src/rdkafka_partition.c b/src/rdkafka_partition.c
index 451d06eb08..ce4f01b467 100644
--- a/src/rdkafka_partition.c
+++ b/src/rdkafka_partition.c
@@ -146,8 +146,8 @@ static void rd_kafka_toppar_consumer_lag_req(rd_kafka_toppar_t *rktp) {
* broker supports FETCH >= v5, since this will be set when
* doing fetch requests.
*/
- if (rd_kafka_broker_ApiVersion_supported(
- rktp->rktp_broker, RD_KAFKAP_Fetch, 0, 5, NULL) == 5) {
+ if (rd_kafka_broker_ApiVersion_at_least(rktp->rktp_broker,
+ RD_KAFKAP_Fetch, 5)) {
rd_kafka_toppar_unlock(rktp);
return;
}
@@ -376,14 +376,24 @@ void rd_kafka_toppar_set_fetch_state(rd_kafka_toppar_t *rktp, int fetch_state) {
rktp->rktp_fetch_state = fetch_state;
- if (fetch_state == RD_KAFKA_TOPPAR_FETCH_ACTIVE)
+ if (fetch_state == RD_KAFKA_TOPPAR_FETCH_ACTIVE) {
+ rktp->rktp_ts_fetch_backoff = 0;
+
+ /* Wake-up broker thread which might be idling on IO */
+ if (rktp->rktp_broker)
+ rd_kafka_broker_wakeup(rktp->rktp_broker,
+ "fetch start");
+
rd_kafka_dbg(
rktp->rktp_rkt->rkt_rk, CONSUMER | RD_KAFKA_DBG_TOPIC,
"FETCH",
"Partition %.*s [%" PRId32 "] start fetching at %s",
RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic),
rktp->rktp_partition,
- rd_kafka_fetch_pos2str(rktp->rktp_next_fetch_start));
+ rd_kafka_fetch_pos2str(
+ rd_kafka_toppar_fetch_decide_next_fetch_start_pos(
+ rktp)));
+ }
}
@@ -1009,7 +1019,71 @@ void rd_kafka_toppar_insert_msgq(rd_kafka_toppar_t *rktp,
rd_kafka_toppar_unlock(rktp);
}
+/**
+ * @brief Purge internal fetch queue if toppar is stopped
+ * (RD_KAFKA_TOPPAR_FETCH_STOPPED) and removed from the cluster
+ * (RD_KAFKA_TOPPAR_F_REMOVE). Will be called from different places as it's
+ * removed starting from a metadata response and stopped from a rebalance or a
+ * consumer close.
+ *
+ * @remark Avoids circular dependencies in from `rktp_fetchq` ops to the same
+ * toppar that stop destroying a consumer.
+ *
+ * @locks rd_kafka_toppar_lock() MUST be held
+ */
+void rd_kafka_toppar_purge_internal_fetch_queue_maybe(rd_kafka_toppar_t *rktp) {
+ rd_kafka_q_t *rkq;
+ rkq = rktp->rktp_fetchq;
+ mtx_lock(&rkq->rkq_lock);
+ if (rktp->rktp_flags & RD_KAFKA_TOPPAR_F_REMOVE &&
+ !rktp->rktp_fetchq->rkq_fwdq) {
+ rd_kafka_op_t *rko;
+ int cnt = 0, barrier_cnt = 0, message_cnt = 0, other_cnt = 0;
+
+ /* Partition is being removed from the cluster and it's stopped,
+ * so rktp->rktp_fetchq->rkq_fwdq is NULL.
+ * Purge remaining operations in rktp->rktp_fetchq->rkq_q,
+ * while holding lock, to avoid circular references */
+ rko = TAILQ_FIRST(&rkq->rkq_q);
+ while (rko) {
+ if (rko->rko_type != RD_KAFKA_OP_BARRIER &&
+ rko->rko_type != RD_KAFKA_OP_FETCH) {
+ rd_kafka_log(
+ rktp->rktp_rkt->rkt_rk, LOG_WARNING,
+ "PARTDEL",
+ "Purging toppar fetch queue buffer op"
+ "with unexpected type: %s",
+ rd_kafka_op2str(rko->rko_type));
+ }
+ if (rko->rko_type == RD_KAFKA_OP_BARRIER)
+ barrier_cnt++;
+ else if (rko->rko_type == RD_KAFKA_OP_FETCH)
+ message_cnt++;
+ else
+ other_cnt++;
+
+ rko = TAILQ_NEXT(rko, rko_link);
+ cnt++;
+ }
+
+ if (cnt) {
+ rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, CGRP, "PARTDEL",
+ "Purge toppar fetch queue buffer "
+ "containing %d op(s) "
+ "(%d barrier(s), %d message(s), %d other)"
+ " to avoid "
+ "circular references",
+ cnt, barrier_cnt, message_cnt, other_cnt);
+ rd_kafka_q_purge0(rktp->rktp_fetchq, rd_false);
+ } else {
+ rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, CGRP, "PARTDEL",
+ "Not purging toppar fetch queue buffer."
+ " No ops present in the buffer.");
+ }
+ }
+ mtx_unlock(&rkq->rkq_lock);
+}
/**
* Helper method for purging queues when removing a toppar.
@@ -1161,7 +1235,8 @@ void rd_kafka_toppar_broker_delegate(rd_kafka_toppar_t *rktp,
/* Undelegated toppars are delgated to the internal
* broker for bookkeeping. */
- if (!rkb && !rd_kafka_terminating(rk)) {
+ if (!rd_kafka_terminating(rk) &&
+ (!rkb || rd_kafka_broker_termination_in_progress(rkb))) {
rkb = rd_kafka_broker_internal(rk);
internal_fallback = 1;
}
@@ -1274,7 +1349,7 @@ void rd_kafka_toppar_next_offset_handle(rd_kafka_toppar_t *rktp,
if (rktp->rktp_query_pos.offset <= RD_KAFKA_OFFSET_TAIL_BASE) {
int64_t orig_offset = next_pos.offset;
int64_t tail_cnt = llabs(rktp->rktp_query_pos.offset -
- RD_KAFKA_OFFSET_TAIL_BASE);
+ RD_KAFKA_OFFSET_TAIL_BASE);
if (tail_cnt > next_pos.offset)
next_pos.offset = 0;
@@ -1295,10 +1370,6 @@ void rd_kafka_toppar_next_offset_handle(rd_kafka_toppar_t *rktp,
rd_kafka_toppar_set_next_fetch_position(rktp, next_pos);
rd_kafka_toppar_set_fetch_state(rktp, RD_KAFKA_TOPPAR_FETCH_ACTIVE);
-
- /* Wake-up broker thread which might be idling on IO */
- if (rktp->rktp_broker)
- rd_kafka_broker_wakeup(rktp->rktp_broker, "ready to fetch");
}
@@ -1359,7 +1430,7 @@ static void rd_kafka_toppar_handle_Offset(rd_kafka_t *rk,
rd_kafka_toppar_lock(rktp);
/* Drop reply from previous partition leader */
- if (err != RD_KAFKA_RESP_ERR__DESTROY && rktp->rktp_broker != rkb)
+ if (err != RD_KAFKA_RESP_ERR__DESTROY && rktp->rktp_leader != rkb)
err = RD_KAFKA_RESP_ERR__OUTDATED;
rd_kafka_toppar_unlock(rktp);
@@ -1549,7 +1620,7 @@ void rd_kafka_toppar_offset_request(rd_kafka_toppar_t *rktp,
rd_kafka_assert(NULL,
thrd_is_current(rktp->rktp_rkt->rkt_rk->rk_thread));
- rkb = rktp->rktp_broker;
+ rkb = rktp->rktp_leader;
if (!backoff_ms && (!rkb || rkb->rkb_source == RD_KAFKA_INTERNAL))
backoff_ms = 500;
@@ -1682,11 +1753,6 @@ static void rd_kafka_toppar_fetch_start(rd_kafka_toppar_t *rktp,
rd_kafka_toppar_set_fetch_state(rktp,
RD_KAFKA_TOPPAR_FETCH_ACTIVE);
-
- /* Wake-up broker thread which might be idling on IO */
- if (rktp->rktp_broker)
- rd_kafka_broker_wakeup(rktp->rktp_broker,
- "fetch start");
}
rktp->rktp_offsets_fin.eof_offset = RD_KAFKA_OFFSET_INVALID;
@@ -2096,6 +2162,7 @@ static rd_kafka_op_res_t rd_kafka_toppar_op_serve(rd_kafka_t *rk,
rd_kafka_toppar_lock(rktp);
if (rko->rko_err) {
+ int actions;
rd_kafka_dbg(
rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET",
"Failed to fetch offset for "
@@ -2110,10 +2177,15 @@ static rd_kafka_op_res_t rd_kafka_toppar_op_serve(rd_kafka_t *rk,
rd_kafka_toppar_unlock(rktp);
- /* Propagate error to application */
+ actions = rd_kafka_handle_OffsetFetch_err_action(
+ NULL, rko->rko_err, NULL);
+ /* Propagate error to application. Exclude
+ * permanent errors that caused a coordinator
+ * refresh like `NOT_COORDINATOR` */
if (rko->rko_err != RD_KAFKA_RESP_ERR__WAIT_COORD &&
rko->rko_err !=
- RD_KAFKA_RESP_ERR_UNSTABLE_OFFSET_COMMIT)
+ RD_KAFKA_RESP_ERR_UNSTABLE_OFFSET_COMMIT &&
+ !(actions & RD_KAFKA_ERR_ACTION_REFRESH))
rd_kafka_consumer_err(
rktp->rktp_fetchq, RD_KAFKA_NODEID_UA,
rko->rko_err, 0, NULL, rktp,
@@ -2518,7 +2590,8 @@ void rd_kafka_toppar_leader_unavailable(rd_kafka_toppar_t *rktp,
rkt->rkt_flags |= RD_KAFKA_TOPIC_F_LEADER_UNAVAIL;
rd_kafka_topic_wrunlock(rkt);
- rd_kafka_topic_fast_leader_query(rkt->rkt_rk);
+ rd_kafka_topic_fast_leader_query(rkt->rkt_rk,
+ rd_false /* don't force */);
}
@@ -3610,11 +3683,12 @@ static rd_bool_t rd_kafka_topic_partition_list_get_leaders(
struct rd_kafka_partition_leader *leader;
const rd_kafka_metadata_topic_t *mtopic;
const rd_kafka_metadata_partition_t *mpart;
+ const rd_kafka_metadata_partition_internal_t *mdpi;
rd_bool_t topic_wait_cache;
rd_kafka_metadata_cache_topic_partition_get(
- rk, &mtopic, &mpart, rktpar->topic, rktpar->partition,
- 0 /*negative entries too*/);
+ rk, &mtopic, &mpart, &mdpi, rktpar->topic,
+ rktpar->partition, 0 /*negative entries too*/);
topic_wait_cache =
!mtopic ||
@@ -3682,9 +3756,11 @@ static rd_bool_t rd_kafka_topic_partition_list_get_leaders(
rd_kafka_topic_partition_update(rktpar2, rktpar);
} else {
/* Make a copy of rktpar and add to partitions list */
- rd_kafka_topic_partition_list_add_copy(
+ rktpar2 = rd_kafka_topic_partition_list_add_copy(
leader->partitions, rktpar);
}
+ rd_kafka_topic_partition_set_current_leader_epoch(
+ rktpar2, mdpi->leader_epoch);
rktpar->err = RD_KAFKA_RESP_ERR_NO_ERROR;
@@ -3808,7 +3884,7 @@ rd_kafka_topic_partition_list_query_leaders_async_worker(rd_kafka_op_t *rko) {
rd_kafka_metadata_refresh_topics(
rk, NULL, &query_topics, rd_true /*force*/,
rd_false /*!allow_auto_create*/, rd_false /*!cgrp_update*/,
- "query partition leaders");
+ -1, "query partition leaders");
}
rd_list_destroy(leaders);
@@ -3997,7 +4073,7 @@ rd_kafka_resp_err_t rd_kafka_topic_partition_list_query_leaders(
rd_kafka_metadata_refresh_topics(
rk, NULL, &query_topics, rd_true /*force*/,
rd_false /*!allow_auto_create*/,
- rd_false /*!cgrp_update*/,
+ rd_false /*!cgrp_update*/, -1,
"query partition leaders");
ts_query = now;
query_cnt++;
@@ -4357,6 +4433,89 @@ int rd_kafka_topic_partition_list_regex_cnt(
}
+/**
+ * @brief Match function that returns true if topic is not a regex.
+ */
+static int rd_kafka_topic_partition_not_regex(const void *elem,
+ const void *opaque) {
+ const rd_kafka_topic_partition_t *rktpar = elem;
+ return *rktpar->topic != '^';
+}
+
+/**
+ * @brief Return a new list with all regex topics removed.
+ *
+ * @remark The caller is responsible for freeing the returned list.
+ */
+rd_kafka_topic_partition_list_t *rd_kafka_topic_partition_list_remove_regexes(
+ const rd_kafka_topic_partition_list_t *rktparlist) {
+ return rd_kafka_topic_partition_list_match(
+ rktparlist, rd_kafka_topic_partition_not_regex, NULL);
+}
+
+
+/**
+ * @brief Combine regexes present in the list into a single regex.
+ */
+rd_kafkap_str_t *rd_kafka_topic_partition_list_combine_regexes(
+ const rd_kafka_topic_partition_list_t *rktparlist) {
+ int i;
+ int combined_regex_len = 1; /* 1 for null-terminator */
+ int regex_cnt = 0;
+ int j = 1;
+ rd_bool_t is_first_regex = rd_true;
+ char *combined_regex_str;
+ rd_kafkap_str_t *combined_regex_kstr;
+
+ // Count the number of characters needed for the combined regex string
+ for (i = 0; i < rktparlist->cnt; i++) {
+ const rd_kafka_topic_partition_t *rktpar =
+ &(rktparlist->elems[i]);
+ if (*rktpar->topic == '^') {
+ combined_regex_len += strlen(rktpar->topic);
+ regex_cnt++;
+ }
+ }
+
+ if (regex_cnt == 0)
+ return rd_kafkap_str_new("", 0);
+
+ combined_regex_len +=
+ 3 * (regex_cnt - 1); /* 1 for each ')|(' separator */
+ combined_regex_len += 2; /* 2 for enclosing brackets */
+
+ // memory allocation for the combined regex string
+ combined_regex_str = rd_malloc(combined_regex_len);
+
+ // Construct the combined regex string
+ combined_regex_str[0] = '(';
+ for (i = 0; i < rktparlist->cnt; i++) {
+ const rd_kafka_topic_partition_t *rktpar =
+ &(rktparlist->elems[i]);
+ char *topic = rktpar->topic;
+ if (*topic == '^') {
+ if (!is_first_regex) {
+ combined_regex_str[j++] = ')';
+ combined_regex_str[j++] = '|';
+ combined_regex_str[j++] = '(';
+ }
+ while (*topic) {
+ combined_regex_str[j++] = *topic;
+ topic++;
+ }
+ is_first_regex = rd_false;
+ }
+ }
+ combined_regex_str[j++] = ')';
+ combined_regex_str[j] = '\0';
+
+ combined_regex_kstr =
+ rd_kafkap_str_new(combined_regex_str, combined_regex_len - 1);
+ rd_free(combined_regex_str);
+ return combined_regex_kstr;
+}
+
+
/**
* @brief Reset base sequence for this toppar.
*
diff --git a/src/rdkafka_partition.h b/src/rdkafka_partition.h
index b74daf8e2f..97a704f03b 100644
--- a/src/rdkafka_partition.h
+++ b/src/rdkafka_partition.h
@@ -292,13 +292,14 @@ struct rd_kafka_toppar_s { /* rd_kafka_toppar_t */
int32_t rktp_fetch_version; /* Op version of curr fetch.
(broker thread) */
- enum { RD_KAFKA_TOPPAR_FETCH_NONE = 0,
- RD_KAFKA_TOPPAR_FETCH_STOPPING,
- RD_KAFKA_TOPPAR_FETCH_STOPPED,
- RD_KAFKA_TOPPAR_FETCH_OFFSET_QUERY,
- RD_KAFKA_TOPPAR_FETCH_OFFSET_WAIT,
- RD_KAFKA_TOPPAR_FETCH_VALIDATE_EPOCH_WAIT,
- RD_KAFKA_TOPPAR_FETCH_ACTIVE,
+ enum {
+ RD_KAFKA_TOPPAR_FETCH_NONE = 0,
+ RD_KAFKA_TOPPAR_FETCH_STOPPING,
+ RD_KAFKA_TOPPAR_FETCH_STOPPED,
+ RD_KAFKA_TOPPAR_FETCH_OFFSET_QUERY,
+ RD_KAFKA_TOPPAR_FETCH_OFFSET_WAIT,
+ RD_KAFKA_TOPPAR_FETCH_VALIDATE_EPOCH_WAIT,
+ RD_KAFKA_TOPPAR_FETCH_ACTIVE,
} rktp_fetch_state; /* Broker thread's state */
#define RD_KAFKA_TOPPAR_FETCH_IS_STARTED(fetch_state) \
@@ -433,6 +434,8 @@ struct rd_kafka_toppar_s { /* rd_kafka_toppar_t */
#define RD_KAFKA_TOPPAR_F_ASSIGNED \
0x2000 /**< Toppar is part of the consumer \
* assignment. */
+#define RD_KAFKA_TOPPAR_F_VALIDATING \
+ 0x4000 /**< Toppar is currently requesting validation. */
/*
* Timers
@@ -648,6 +651,8 @@ void rd_kafka_toppar_offset_request(rd_kafka_toppar_t *rktp,
rd_kafka_fetch_pos_t query_pos,
int backoff_ms);
+void rd_kafka_toppar_purge_internal_fetch_queue_maybe(rd_kafka_toppar_t *rktp);
+
int rd_kafka_toppar_purge_queues(rd_kafka_toppar_t *rktp,
int purge_flags,
rd_bool_t include_xmit_msgq);
@@ -1006,6 +1011,12 @@ rd_kafka_resp_err_t rd_kafka_topic_partition_list_get_err(
int rd_kafka_topic_partition_list_regex_cnt(
const rd_kafka_topic_partition_list_t *rktparlist);
+rd_kafka_topic_partition_list_t *rd_kafka_topic_partition_list_remove_regexes(
+ const rd_kafka_topic_partition_list_t *rktparlist);
+
+rd_kafkap_str_t *rd_kafka_topic_partition_list_combine_regexes(
+ const rd_kafka_topic_partition_list_t *rktparlist);
+
void *rd_kafka_topic_partition_list_copy_opaque(const void *src, void *opaque);
/**
diff --git a/src/rdkafka_proto.h b/src/rdkafka_proto.h
index 895e338c83..02565ecb3b 100644
--- a/src/rdkafka_proto.h
+++ b/src/rdkafka_proto.h
@@ -172,6 +172,7 @@ static RD_UNUSED const char *rd_kafka_ApiKey2str(int16_t ApiKey) {
[RD_KAFKAP_ListTransactions] = "ListTransactions",
[RD_KAFKAP_AllocateProducerIds] = "AllocateProducerIds",
[RD_KAFKAP_ConsumerGroupHeartbeat] = "ConsumerGroupHeartbeat",
+ [RD_KAFKAP_ConsumerGroupDescribe] = "ConsumerGroupDescribe",
[RD_KAFKAP_GetTelemetrySubscriptions] = "GetTelemetrySubscriptions",
[RD_KAFKAP_PushTelemetry] = "PushTelemetry",
@@ -283,6 +284,8 @@ typedef struct rd_kafkap_str_s {
#define RD_KAFKAP_STR_INITIALIZER \
{ .len = RD_KAFKAP_STR_LEN_NULL, .str = NULL }
+#define RD_KAFKAP_STR_INITIALIZER_EMPTY \
+ { .len = 0, .str = "" }
/**
* Frees a Kafka string previously allocated with `rd_kafkap_str_new()`
*/
@@ -620,7 +623,7 @@ static RD_INLINE RD_UNUSED int rd_kafka_Uuid_ptr_cmp(void *a, void *b) {
rd_kafka_Uuid_t rd_kafka_Uuid_random();
-const char *rd_kafka_Uuid_str(const rd_kafka_Uuid_t *uuid);
+char *rd_kafka_Uuid_str(const rd_kafka_Uuid_t *uuid);
unsigned int rd_kafka_Uuid_hash(const rd_kafka_Uuid_t *uuid);
diff --git a/src/rdkafka_protocol.h b/src/rdkafka_protocol.h
index 4755494d0b..19190e1447 100644
--- a/src/rdkafka_protocol.h
+++ b/src/rdkafka_protocol.h
@@ -123,5 +123,6 @@
#define RD_KAFKAP__NUM 74
+#define RD_KAFKAP_RPC_VERSION_MAX INT16_MAX
#endif /* _RDKAFKA_PROTOCOL_H_ */
diff --git a/src/rdkafka_queue.c b/src/rdkafka_queue.c
index 3e30379558..92eddccae9 100644
--- a/src/rdkafka_queue.c
+++ b/src/rdkafka_queue.c
@@ -93,10 +93,12 @@ void rd_kafka_q_init0(rd_kafka_q_t *rkq,
rkq->rkq_flags = RD_KAFKA_Q_F_READY;
if (for_consume)
rkq->rkq_flags |= RD_KAFKA_Q_F_CONSUMER;
- rkq->rkq_rk = rk;
- rkq->rkq_qio = NULL;
- rkq->rkq_serve = NULL;
- rkq->rkq_opaque = NULL;
+ rkq->rkq_rk = rk;
+ rkq->rkq_qio = NULL;
+ rkq->rkq_serve = NULL;
+ rkq->rkq_opaque = NULL;
+ rkq->rkq_ts_last_poll_start = 0;
+ rkq->rkq_ts_last_poll_end = 0;
mtx_init(&rkq->rkq_lock, mtx_plain);
cnd_init(&rkq->rkq_cond);
#if ENABLE_DEVEL
@@ -380,16 +382,26 @@ rd_kafka_op_filter(rd_kafka_q_t *rkq, rd_kafka_op_t *rko, int version) {
* Serve q like rd_kafka_q_serve() until an op is found that can be returned
* as an event to the application.
*
+ * @param rkq Queue to pop from.
+ * @param timeout_us Maximum time to wait for an op, in microseconds.
+ * @param version Fetch version to filter out outdated ops.
+ * @param cb_type Callback type to use for the op.
+ * @param callback Callback to use for the op, if any.
+ * @param opaque Opaque pointer to pass to the callback.
+ * @param is_consume_call If `rd_true` and it could be a consumer call it
+ * checks if this queue can contain fetched messages.
+ *
* @returns the first event:able op, or NULL on timeout.
*
- * Locality: any thread
+ * @locality any thread
*/
-rd_kafka_op_t *rd_kafka_q_pop_serve(rd_kafka_q_t *rkq,
- rd_ts_t timeout_us,
- int32_t version,
- rd_kafka_q_cb_type_t cb_type,
- rd_kafka_q_serve_cb_t *callback,
- void *opaque) {
+static rd_kafka_op_t *rd_kafka_q_pop_serve0(rd_kafka_q_t *rkq,
+ rd_ts_t timeout_us,
+ int32_t version,
+ rd_kafka_q_cb_type_t cb_type,
+ rd_kafka_q_serve_cb_t *callback,
+ void *opaque,
+ rd_bool_t is_consume_call) {
rd_kafka_op_t *rko;
rd_kafka_q_t *fwdq;
@@ -400,14 +412,14 @@ rd_kafka_op_t *rd_kafka_q_pop_serve(rd_kafka_q_t *rkq,
rd_kafka_yield_thread = 0;
if (!(fwdq = rd_kafka_q_fwd_get(rkq, 0))) {
const rd_bool_t can_q_contain_fetched_msgs =
+ is_consume_call &&
rd_kafka_q_can_contain_fetched_msgs(rkq, RD_DONT_LOCK);
- struct timespec timeout_tspec;
-
- rd_timeout_init_timespec_us(&timeout_tspec, timeout_us);
+ rd_ts_t abs_timeout = rd_timeout_init_us(timeout_us);
- if (timeout_us && can_q_contain_fetched_msgs)
- rd_kafka_app_poll_blocking(rkq->rkq_rk);
+ if (can_q_contain_fetched_msgs)
+ rd_kafka_app_poll_start(rkq->rkq_rk, rkq, 0,
+ timeout_us);
while (1) {
rd_kafka_op_res_t res;
@@ -447,14 +459,14 @@ rd_kafka_op_t *rd_kafka_q_pop_serve(rd_kafka_q_t *rkq,
} else if (unlikely(res ==
RD_KAFKA_OP_RES_YIELD)) {
if (can_q_contain_fetched_msgs)
- rd_kafka_app_polled(
- rkq->rkq_rk);
+ rd_kafka_app_polled(rkq->rkq_rk,
+ rkq);
/* Callback yielded, unroll */
return NULL;
} else {
if (can_q_contain_fetched_msgs)
- rd_kafka_app_polled(
- rkq->rkq_rk);
+ rd_kafka_app_polled(rkq->rkq_rk,
+ rkq);
break; /* Proper op, handle below. */
}
}
@@ -463,7 +475,7 @@ rd_kafka_op_t *rd_kafka_q_pop_serve(rd_kafka_q_t *rkq,
if (is_locked)
mtx_unlock(&rkq->rkq_lock);
if (can_q_contain_fetched_msgs)
- rd_kafka_app_polled(rkq->rkq_rk);
+ rd_kafka_app_polled(rkq->rkq_rk, rkq);
return NULL;
}
@@ -471,10 +483,10 @@ rd_kafka_op_t *rd_kafka_q_pop_serve(rd_kafka_q_t *rkq,
mtx_lock(&rkq->rkq_lock);
if (cnd_timedwait_abs(&rkq->rkq_cond, &rkq->rkq_lock,
- &timeout_tspec) != thrd_success) {
+ abs_timeout) != thrd_success) {
mtx_unlock(&rkq->rkq_lock);
if (can_q_contain_fetched_msgs)
- rd_kafka_app_polled(rkq->rkq_rk);
+ rd_kafka_app_polled(rkq->rkq_rk, rkq);
return NULL;
}
}
@@ -483,8 +495,8 @@ rd_kafka_op_t *rd_kafka_q_pop_serve(rd_kafka_q_t *rkq,
/* Since the q_pop may block we need to release the parent
* queue's lock. */
mtx_unlock(&rkq->rkq_lock);
- rko = rd_kafka_q_pop_serve(fwdq, timeout_us, version, cb_type,
- callback, opaque);
+ rko = rd_kafka_q_pop_serve0(fwdq, timeout_us, version, cb_type,
+ callback, opaque, is_consume_call);
rd_kafka_q_destroy(fwdq);
}
@@ -492,35 +504,72 @@ rd_kafka_op_t *rd_kafka_q_pop_serve(rd_kafka_q_t *rkq,
return rko;
}
+rd_kafka_op_t *rd_kafka_q_pop_serve(rd_kafka_q_t *rkq,
+ rd_ts_t timeout_us,
+ int32_t version,
+ rd_kafka_q_cb_type_t cb_type,
+ rd_kafka_q_serve_cb_t *callback,
+ void *opaque) {
+ return rd_kafka_q_pop_serve0(rkq, timeout_us, version, cb_type,
+ callback, opaque, rd_false);
+}
+
+/**
+ * @brief Same as `rd_kafka_q_pop_serve`, use this call when the queue
+ * could be a fetch queue, use the other one when it
+ * can never be.
+ */
+rd_kafka_op_t *
+rd_kafka_q_pop_serve_maybe_consume(rd_kafka_q_t *rkq,
+ rd_ts_t timeout_us,
+ int32_t version,
+ rd_kafka_q_cb_type_t cb_type,
+ rd_kafka_q_serve_cb_t *callback,
+ void *opaque) {
+ return rd_kafka_q_pop_serve0(rkq, timeout_us, version, cb_type,
+ callback, opaque,
+ /* Only check if to call app_polled when
+ * this is a consumer. */
+ rkq->rkq_rk->rk_type == RD_KAFKA_CONSUMER);
+}
+
rd_kafka_op_t *
rd_kafka_q_pop(rd_kafka_q_t *rkq, rd_ts_t timeout_us, int32_t version) {
return rd_kafka_q_pop_serve(rkq, timeout_us, version,
RD_KAFKA_Q_CB_RETURN, NULL, NULL);
}
-
/**
* Pop all available ops from a queue and call the provided
* callback for each op.
- * `max_cnt` limits the number of ops served, 0 = no limit.
*
- * Returns the number of ops served.
+ * @param rkq Queue to serve.
+ * @param max_cnt Limits the number of ops served, 0 = no limit.
+ * @param cb_type Callback type to use.
+ * @param callback Callback to call for each op.
+ * @param opaque Opaque pointer to pass to the callback.
+ * @param is_consume_call If `rd_true` and it could be a consumer call it
+ * checks if this queue can contain fetched messages.
*
- * Locality: any thread.
+ * @return The number of ops served.
+ *
+ * @locality any thread.
*/
-int rd_kafka_q_serve(rd_kafka_q_t *rkq,
- int timeout_ms,
- int max_cnt,
- rd_kafka_q_cb_type_t cb_type,
- rd_kafka_q_serve_cb_t *callback,
- void *opaque) {
+int rd_kafka_q_serve0(rd_kafka_q_t *rkq,
+ int timeout_ms,
+ int max_cnt,
+ rd_kafka_q_cb_type_t cb_type,
+ rd_kafka_q_serve_cb_t *callback,
+ void *opaque,
+ rd_bool_t is_consume_call) {
rd_kafka_t *rk = rkq->rkq_rk;
rd_kafka_op_t *rko;
rd_kafka_q_t localq;
rd_kafka_q_t *fwdq;
int cnt = 0;
- struct timespec timeout_tspec;
+ rd_ts_t abs_timeout;
const rd_bool_t can_q_contain_fetched_msgs =
+ is_consume_call &&
rd_kafka_q_can_contain_fetched_msgs(rkq, RD_DONT_LOCK);
rd_dassert(cb_type);
@@ -533,23 +582,23 @@ int rd_kafka_q_serve(rd_kafka_q_t *rkq,
/* Since the q_pop may block we need to release the parent
* queue's lock. */
mtx_unlock(&rkq->rkq_lock);
- ret = rd_kafka_q_serve(fwdq, timeout_ms, max_cnt, cb_type,
- callback, opaque);
+ ret = rd_kafka_q_serve0(fwdq, timeout_ms, max_cnt, cb_type,
+ callback, opaque, is_consume_call);
rd_kafka_q_destroy(fwdq);
return ret;
}
- rd_timeout_init_timespec(&timeout_tspec, timeout_ms);
+ abs_timeout = rd_timeout_init(timeout_ms);
- if (timeout_ms && can_q_contain_fetched_msgs)
- rd_kafka_app_poll_blocking(rk);
+ if (can_q_contain_fetched_msgs)
+ rd_kafka_app_poll_start(rk, rkq, 0, timeout_ms);
/* Wait for op */
while (!(rko = TAILQ_FIRST(&rkq->rkq_q)) &&
!rd_kafka_q_check_yield(rkq) &&
- cnd_timedwait_abs(&rkq->rkq_cond, &rkq->rkq_lock,
- &timeout_tspec) == thrd_success)
+ cnd_timedwait_abs(&rkq->rkq_cond, &rkq->rkq_lock, abs_timeout) ==
+ thrd_success)
;
rd_kafka_q_mark_served(rkq);
@@ -557,7 +606,7 @@ int rd_kafka_q_serve(rd_kafka_q_t *rkq,
if (!rko) {
mtx_unlock(&rkq->rkq_lock);
if (can_q_contain_fetched_msgs)
- rd_kafka_app_polled(rk);
+ rd_kafka_app_polled(rk, rkq);
return 0;
}
@@ -593,13 +642,41 @@ int rd_kafka_q_serve(rd_kafka_q_t *rkq,
}
if (can_q_contain_fetched_msgs)
- rd_kafka_app_polled(rk);
+ rd_kafka_app_polled(rk, rkq);
rd_kafka_q_destroy_owner(&localq);
return cnt;
}
+int rd_kafka_q_serve(rd_kafka_q_t *rkq,
+ int timeout_ms,
+ int max_cnt,
+ rd_kafka_q_cb_type_t cb_type,
+ rd_kafka_q_serve_cb_t *callback,
+ void *opaque) {
+ return rd_kafka_q_serve0(rkq, timeout_ms, max_cnt, cb_type, callback,
+ opaque, rd_false);
+}
+
+/**
+ * @brief Same as `rd_kafka_q_serve`, use this call when the queue
+ * could be a fetch queue, use the other one when it
+ * can never be.
+ */
+int rd_kafka_q_serve_maybe_consume(rd_kafka_q_t *rkq,
+ int timeout_ms,
+ int max_cnt,
+ rd_kafka_q_cb_type_t cb_type,
+ rd_kafka_q_serve_cb_t *callback,
+ void *opaque) {
+ return rd_kafka_q_serve0(rkq, timeout_ms, max_cnt, cb_type, callback,
+ opaque,
+ /* Only check if to call app_polled when
+ * this is a consumer. */
+ rkq->rkq_rk->rk_type == RD_KAFKA_CONSUMER);
+}
+
/**
* @brief Filter out and destroy outdated messages.
*
@@ -665,7 +742,7 @@ int rd_kafka_q_serve_rkmessages(rd_kafka_q_t *rkq,
rd_kafka_op_t *rko, *next;
rd_kafka_t *rk = rkq->rkq_rk;
rd_kafka_q_t *fwdq;
- struct timespec timeout_tspec;
+ rd_ts_t abs_timeout;
int i;
mtx_lock(&rkq->rkq_lock);
@@ -681,10 +758,9 @@ int rd_kafka_q_serve_rkmessages(rd_kafka_q_t *rkq,
mtx_unlock(&rkq->rkq_lock);
- if (timeout_ms)
- rd_kafka_app_poll_blocking(rk);
+ abs_timeout = rd_timeout_init(timeout_ms);
- rd_timeout_init_timespec(&timeout_tspec, timeout_ms);
+ rd_kafka_app_poll_start(rk, rkq, 0, timeout_ms);
rd_kafka_yield_thread = 0;
while (cnt < rkmessages_size) {
@@ -695,7 +771,7 @@ int rd_kafka_q_serve_rkmessages(rd_kafka_q_t *rkq,
while (!(rko = TAILQ_FIRST(&rkq->rkq_q)) &&
!rd_kafka_q_check_yield(rkq) &&
cnd_timedwait_abs(&rkq->rkq_cond, &rkq->rkq_lock,
- &timeout_tspec) == thrd_success)
+ abs_timeout) == thrd_success)
;
rd_kafka_q_mark_served(rkq);
@@ -788,7 +864,7 @@ int rd_kafka_q_serve_rkmessages(rd_kafka_q_t *rkq,
rd_kafka_op_destroy(rko);
}
- rd_kafka_app_polled(rk);
+ rd_kafka_app_polled(rk, rkq);
return cnt;
}
diff --git a/src/rdkafka_queue.h b/src/rdkafka_queue.h
index eb329d1c1d..ff1a465770 100644
--- a/src/rdkafka_queue.h
+++ b/src/rdkafka_queue.h
@@ -91,6 +91,12 @@ struct rd_kafka_q_s {
* Shall return 1 if op was handled, else 0. */
rd_kafka_q_serve_cb_t *rkq_serve;
void *rkq_opaque;
+ rd_ts_t rkq_ts_last_poll_start; /**< Timestamp of last queue
+ * poll() call start
+ * Only relevant for a consumer. */
+ rd_ts_t rkq_ts_last_poll_end; /**< Timestamp of last queue
+ * poll() call end
+ * Only relevant for a consumer. */
#if ENABLE_DEVEL
char rkq_name[64]; /* Debugging: queue name (FUNC:LINE) */
@@ -841,6 +847,13 @@ rd_kafka_op_t *rd_kafka_q_pop_serve(rd_kafka_q_t *rkq,
rd_kafka_q_serve_cb_t *callback,
void *opaque);
rd_kafka_op_t *
+rd_kafka_q_pop_serve_maybe_consume(rd_kafka_q_t *rkq,
+ rd_ts_t timeout_us,
+ int32_t version,
+ rd_kafka_q_cb_type_t cb_type,
+ rd_kafka_q_serve_cb_t *callback,
+ void *opaque);
+rd_kafka_op_t *
rd_kafka_q_pop(rd_kafka_q_t *rkq, rd_ts_t timeout_us, int32_t version);
int rd_kafka_q_serve(rd_kafka_q_t *rkq,
int timeout_ms,
@@ -848,6 +861,12 @@ int rd_kafka_q_serve(rd_kafka_q_t *rkq,
rd_kafka_q_cb_type_t cb_type,
rd_kafka_q_serve_cb_t *callback,
void *opaque);
+int rd_kafka_q_serve_maybe_consume(rd_kafka_q_t *rkq,
+ int timeout_ms,
+ int max_cnt,
+ rd_kafka_q_cb_type_t cb_type,
+ rd_kafka_q_serve_cb_t *callback,
+ void *opaque);
int rd_kafka_q_move_cnt(rd_kafka_q_t *dstq,
diff --git a/src/rdkafka_range_assignor.c b/src/rdkafka_range_assignor.c
index a869c139bd..4664de069c 100644
--- a/src/rdkafka_range_assignor.c
+++ b/src/rdkafka_range_assignor.c
@@ -1424,7 +1424,7 @@ static int ut_testRackAwareAssignmentWithCoPartitioning0(
char *subscription12[] = {"t1", "t2"};
char *subscription34[] = {"t3", "t4"};
char **subscriptions[] = {subscription12, subscription12,
- subscription34, subscription34};
+ subscription34, subscription34};
int racks[] = {0, 1, 1, 0};
if (parametrization !=
diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c
index 8623be97d3..663a07eae3 100644
--- a/src/rdkafka_request.c
+++ b/src/rdkafka_request.c
@@ -123,6 +123,7 @@ int rd_kafka_err_action(rd_kafka_broker_t *rkb,
case RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE:
case RD_KAFKA_RESP_ERR_NOT_COORDINATOR:
case RD_KAFKA_RESP_ERR__WAIT_COORD:
+ case RD_KAFKA_RESP_ERR__DESTROY_BROKER:
/* Request metadata information update */
actions |= RD_KAFKA_ERR_ACTION_REFRESH |
RD_KAFKA_ERR_ACTION_MSG_NOT_PERSISTED;
@@ -200,16 +201,15 @@ int rd_kafka_err_action(rd_kafka_broker_t *rkb,
return actions;
}
-
/**
* @brief Read a list of topic+partitions+extra from \p rkbuf.
*
- * @param rkbuf buffer to read from
+ * @param rkbuf Buffer to read from
* @param fields An array of fields to read from the buffer and set on
* the rktpar object, in the specified order, must end
* with RD_KAFKA_TOPIC_PARTITION_FIELD_END.
*
- * @returns a newly allocated list on success, or NULL on parse error.
+ * @returns A newly allocated list on success, or NULL on parse error.
*/
rd_kafka_topic_partition_list_t *rd_kafka_buf_read_topic_partitions(
rd_kafka_buf_t *rkbuf,
@@ -217,14 +217,44 @@ rd_kafka_topic_partition_list_t *rd_kafka_buf_read_topic_partitions(
rd_bool_t use_topic_name,
size_t estimated_part_cnt,
const rd_kafka_topic_partition_field_t *fields) {
+ rd_bool_t parse_err;
+ /* Even if NULL it should be treated as a parse error,
+ * as this field isn't nullable. */
+ return rd_kafka_buf_read_topic_partitions_nullable(
+ rkbuf, use_topic_id, use_topic_name, estimated_part_cnt, fields,
+ &parse_err);
+}
+
+/**
+ * @brief Read a nullable list of topic+partitions+extra from \p rkbuf.
+ *
+ * @param rkbuf Buffer to read from
+ * @param fields An array of fields to read from the buffer and set on
+ * the rktpar object, in the specified order, must end
+ * with RD_KAFKA_TOPIC_PARTITION_FIELD_END.
+ * @param parse_err Is set to true if a parsing error occurred.
+ *
+ * @returns A newly allocated list, or NULL.
+ */
+rd_kafka_topic_partition_list_t *rd_kafka_buf_read_topic_partitions_nullable(
+ rd_kafka_buf_t *rkbuf,
+ rd_bool_t use_topic_id,
+ rd_bool_t use_topic_name,
+ size_t estimated_part_cnt,
+ const rd_kafka_topic_partition_field_t *fields,
+ rd_bool_t *parse_err) {
const int log_decode_errors = LOG_ERR;
int32_t TopicArrayCnt;
rd_kafka_topic_partition_list_t *parts = NULL;
- /* We assume here that the topic partition list is not NULL.
- * FIXME: check NULL topic array case, if required in future. */
+ rd_dassert(parse_err);
+ *parse_err = rd_false;
rd_kafka_buf_read_arraycnt(rkbuf, &TopicArrayCnt, RD_KAFKAP_TOPICS_MAX);
+ if (TopicArrayCnt < -1)
+ goto err_parse;
+ else if (TopicArrayCnt == -1)
+ return NULL;
parts = rd_kafka_topic_partition_list_new(
RD_MAX(TopicArrayCnt * 4, (int)estimated_part_cnt));
@@ -334,6 +364,7 @@ rd_kafka_topic_partition_list_t *rd_kafka_buf_read_topic_partitions(
if (parts)
rd_kafka_topic_partition_list_destroy(parts);
+ *parse_err = rd_true;
return NULL;
}
@@ -1132,6 +1163,11 @@ void rd_kafka_OffsetForLeaderEpochRequest(
}
+int rd_kafka_handle_OffsetFetch_err_action(rd_kafka_broker_t *rkb,
+ rd_kafka_resp_err_t err,
+ rd_kafka_buf_t *request) {
+ return rd_kafka_err_action(rkb, err, request, RD_KAFKA_ERR_ACTION_END);
+}
/**
* Generic handler for OffsetFetch responses.
@@ -1289,9 +1325,15 @@ rd_kafka_handle_OffsetFetch(rd_kafka_t *rk,
rktpar->metadata = NULL;
rktpar->metadata_size = 0;
} else {
- rktpar->metadata = RD_KAFKAP_STR_DUP(&metadata);
- rktpar->metadata_size =
- RD_KAFKAP_STR_LEN(&metadata);
+ /* It cannot use strndup because
+ * it stops at first 0 occurrence. */
+ size_t len = RD_KAFKAP_STR_LEN(&metadata);
+ rktpar->metadata_size = len;
+ unsigned char *metadata_bytes =
+ rd_malloc(len + 1);
+ rktpar->metadata = metadata_bytes;
+ memcpy(rktpar->metadata, metadata.str, len);
+ metadata_bytes[len] = '\0';
}
/* Loose ref from get_toppar() */
@@ -1325,8 +1367,7 @@ rd_kafka_handle_OffsetFetch(rd_kafka_t *rk,
seen_cnt, (*offsets)->cnt, retry_unstable,
rd_kafka_err2str(err));
- actions =
- rd_kafka_err_action(rkb, err, request, RD_KAFKA_ERR_ACTION_END);
+ actions = rd_kafka_handle_OffsetFetch_err_action(rkb, err, request);
if (actions & RD_KAFKA_ERR_ACTION_REFRESH) {
/* Re-query for coordinator */
@@ -1689,9 +1730,9 @@ rd_kafka_handle_OffsetCommit(rd_kafka_t *rk,
continue;
}
+ rktpar->err = partition->err;
if (partition->err) {
- rktpar->err = partition->err;
- err = partition->err;
+ err = partition->err;
errcnt++;
/* Accumulate actions for per-partition
* errors. */
@@ -2236,7 +2277,8 @@ void rd_kafka_ConsumerGroupHeartbeatRequest(
const rd_kafkap_str_t *group_instance_id,
const rd_kafkap_str_t *rack_id,
int32_t rebalance_timeout_ms,
- const rd_kafka_topic_partition_list_t *subscribe_topics,
+ const rd_kafka_topic_partition_list_t *subscribed_topics,
+ rd_kafkap_str_t *subscribed_topic_regex,
const rd_kafkap_str_t *remote_assignor,
const rd_kafka_topic_partition_list_t *current_assignments,
rd_kafka_replyq_t replyq,
@@ -2247,26 +2289,37 @@ void rd_kafka_ConsumerGroupHeartbeatRequest(
int16_t ApiVersion = 0;
int features;
size_t rkbuf_size = 0;
+ rd_kafkap_str_t *subscribed_topic_regex_to_send =
+ subscribed_topic_regex;
ApiVersion = rd_kafka_broker_ApiVersion_supported(
- rkb, RD_KAFKAP_ConsumerGroupHeartbeat, 0, 0, &features);
+ rkb, RD_KAFKAP_ConsumerGroupHeartbeat, 1, 1, &features);
+
+ if (ApiVersion == -1) {
+ rd_kafka_cgrp_coord_dead(rkb->rkb_rk->rk_cgrp,
+ RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE,
+ "ConsumerGroupHeartbeatRequest not "
+ "supported by broker");
+ return;
+ }
if (rd_rkb_is_dbg(rkb, CGRP)) {
- char current_assignments_str[512] = "NULL";
- char subscribe_topics_str[512] = "NULL";
- const char *member_id_str = "NULL";
- const char *group_instance_id_str = "NULL";
- const char *remote_assignor_str = "NULL";
+ char current_assignments_str[512] = "NULL";
+ char subscribed_topics_str[512] = "NULL";
+ const char *member_id_str = "NULL";
+ const char *group_instance_id_str = "NULL";
+ const char *remote_assignor_str = "NULL";
+ const char *subscribed_topic_regex_to_send_str = "NULL";
if (current_assignments) {
rd_kafka_topic_partition_list_str(
current_assignments, current_assignments_str,
sizeof(current_assignments_str), 0);
}
- if (subscribe_topics) {
+ if (subscribed_topics) {
rd_kafka_topic_partition_list_str(
- subscribe_topics, subscribe_topics_str,
- sizeof(subscribe_topics_str), 0);
+ subscribed_topics, subscribed_topics_str,
+ sizeof(subscribed_topics_str), 0);
}
if (member_id)
member_id_str = member_id->str;
@@ -2274,6 +2327,9 @@ void rd_kafka_ConsumerGroupHeartbeatRequest(
group_instance_id_str = group_instance_id->str;
if (remote_assignor)
remote_assignor_str = remote_assignor->str;
+ if (subscribed_topic_regex_to_send)
+ subscribed_topic_regex_to_send_str =
+ subscribed_topic_regex_to_send->str;
rd_rkb_dbg(rkb, CGRP, "HEARTBEAT",
"ConsumerGroupHeartbeat of member id \"%s\", group "
@@ -2281,19 +2337,27 @@ void rd_kafka_ConsumerGroupHeartbeatRequest(
"generation id %" PRId32
", group instance id \"%s\""
", current assignment \"%s\""
- ", subscribe topics \"%s\""
+ ", subscribed topics \"%s\""
+ ", subscribed topic regex \"%s\""
", remote assignor \"%s\"",
member_id_str, group_id->str, member_epoch,
group_instance_id_str, current_assignments_str,
- subscribe_topics_str, remote_assignor_str);
+ subscribed_topics_str,
+ subscribed_topic_regex_to_send_str,
+ remote_assignor_str);
}
size_t next_subscription_size = 0;
- if (subscribe_topics) {
+ if (!subscribed_topic_regex_to_send)
+ subscribed_topic_regex_to_send = rd_kafkap_str_new(NULL, -1);
+
+ if (subscribed_topics) {
next_subscription_size =
- ((subscribe_topics->cnt * (4 + 50)) + 4);
+ ((subscribed_topics->cnt * (4 + 50)) + 4);
}
+ next_subscription_size +=
+ RD_KAFKAP_STR_SIZE(subscribed_topic_regex_to_send);
if (group_id)
rkbuf_size += RD_KAFKAP_STR_SIZE(group_id);
@@ -2323,23 +2387,23 @@ void rd_kafka_ConsumerGroupHeartbeatRequest(
rd_kafka_buf_write_kstr(rkbuf, rack_id);
rd_kafka_buf_write_i32(rkbuf, rebalance_timeout_ms);
- if (subscribe_topics) {
- size_t of_TopicsArrayCnt;
- int topics_cnt = subscribe_topics->cnt;
+ if (subscribed_topics) {
+ int topics_cnt = subscribed_topics->cnt;
/* write Topics */
- of_TopicsArrayCnt = rd_kafka_buf_write_arraycnt_pos(rkbuf);
- rd_kafka_buf_finalize_arraycnt(rkbuf, of_TopicsArrayCnt,
- topics_cnt);
+ rd_kafka_buf_write_arraycnt(rkbuf, topics_cnt);
while (--topics_cnt >= 0)
rd_kafka_buf_write_str(
- rkbuf, subscribe_topics->elems[topics_cnt].topic,
+ rkbuf, subscribed_topics->elems[topics_cnt].topic,
-1);
} else {
rd_kafka_buf_write_arraycnt(rkbuf, -1);
}
+ if (ApiVersion >= 1)
+ rd_kafka_buf_write_kstr(rkbuf, subscribed_topic_regex_to_send);
+
rd_kafka_buf_write_kstr(rkbuf, remote_assignor);
if (current_assignments) {
@@ -2373,13 +2437,17 @@ void rd_kafka_ConsumerGroupHeartbeatRequest(
}
rd_kafka_broker_buf_enq_replyq(rkb, rkbuf, replyq, resp_cb, opaque);
+
+ if (!subscribed_topic_regex)
+ rd_kafkap_str_destroy(subscribed_topic_regex_to_send);
}
/**
* @brief Construct and send ListGroupsRequest to \p rkb
- * with the states (const char *) in \p states.
+ * with the states (const char *) in \p states,
+ * and the types (const char *) in \p types.
* Uses \p max_ApiVersion as maximum API version,
* pass -1 to use the maximum available version.
*
@@ -2393,6 +2461,8 @@ rd_kafka_error_t *rd_kafka_ListGroupsRequest(rd_kafka_broker_t *rkb,
int16_t max_ApiVersion,
const char **states,
size_t states_cnt,
+ const char **types,
+ size_t types_cnt,
rd_kafka_replyq_t replyq,
rd_kafka_resp_cb_t *resp_cb,
void *opaque) {
@@ -2401,7 +2471,7 @@ rd_kafka_error_t *rd_kafka_ListGroupsRequest(rd_kafka_broker_t *rkb,
size_t i;
if (max_ApiVersion < 0)
- max_ApiVersion = 4;
+ max_ApiVersion = 5;
if (max_ApiVersion > ApiVersion) {
/* Remark: don't check if max_ApiVersion is zero.
@@ -2423,12 +2493,17 @@ rd_kafka_error_t *rd_kafka_ListGroupsRequest(rd_kafka_broker_t *rkb,
4 + 1 + 32 * states_cnt, ApiVersion >= 3 /* is_flexver */);
if (ApiVersion >= 4) {
- size_t of_GroupsArrayCnt =
- rd_kafka_buf_write_arraycnt_pos(rkbuf);
+ rd_kafka_buf_write_arraycnt(rkbuf, states_cnt);
for (i = 0; i < states_cnt; i++) {
rd_kafka_buf_write_str(rkbuf, states[i], -1);
}
- rd_kafka_buf_finalize_arraycnt(rkbuf, of_GroupsArrayCnt, i);
+ }
+
+ if (ApiVersion >= 5) {
+ rd_kafka_buf_write_arraycnt(rkbuf, types_cnt);
+ for (i = 0; i < types_cnt; i++) {
+ rd_kafka_buf_write_str(rkbuf, types[i], -1);
+ }
}
rd_kafka_buf_ApiVersion_set(rkbuf, ApiVersion, 0);
@@ -2517,9 +2592,11 @@ static void rd_kafka_handle_Metadata(rd_kafka_t *rk,
rd_kafka_op_t *rko = opaque; /* Possibly NULL */
rd_kafka_metadata_internal_t *mdi = NULL;
const rd_list_t *topics = request->rkbuf_u.Metadata.topics;
+ const int32_t cgrp_subscription_version =
+ request->rkbuf_u.Metadata.cgrp_subscription_version;
int actions;
- rd_kafka_assert(NULL, err == RD_KAFKA_RESP_ERR__DESTROY ||
+ rd_kafka_assert(NULL, rd_kafka_broker_is_any_err_destroy(err) ||
thrd_is_current(rk->rk_thread));
/* Avoid metadata updates when we're terminating. */
@@ -2547,12 +2624,15 @@ static void rd_kafka_handle_Metadata(rd_kafka_t *rk,
if (err)
goto err;
+ rd_kafka_rebootstrap_tmr_stop(rk);
if (rko && rko->rko_replyq.q) {
/* Reply to metadata requester, passing on the metadata.
* Reuse requesting rko for the reply. */
rko->rko_err = err;
rko->rko_u.metadata.md = &mdi->metadata;
rko->rko_u.metadata.mdi = mdi;
+ rko->rko_u.metadata.subscription_version =
+ cgrp_subscription_version;
rd_kafka_replyq_enq(&rko->rko_replyq, rko, 0);
rko = NULL;
} else {
@@ -2563,35 +2643,50 @@ static void rd_kafka_handle_Metadata(rd_kafka_t *rk,
goto done;
err:
- actions = rd_kafka_err_action(rkb, err, request,
+ actions = rd_kafka_err_action(
+ rkb, err, request,
- RD_KAFKA_ERR_ACTION_RETRY,
- RD_KAFKA_RESP_ERR__PARTIAL,
+ RD_KAFKA_ERR_ACTION_SPECIAL, RD_KAFKA_RESP_ERR_REBOOTSTRAP_REQUIRED,
- RD_KAFKA_ERR_ACTION_END);
+ RD_KAFKA_ERR_ACTION_RETRY, RD_KAFKA_RESP_ERR__PARTIAL,
+
+ RD_KAFKA_ERR_ACTION_END);
+
+ if (actions & RD_KAFKA_ERR_ACTION_SPECIAL) {
+ rd_kafka_rebootstrap(rk);
+ }
if (actions & RD_KAFKA_ERR_ACTION_RETRY) {
- if (rd_kafka_buf_retry(rkb, request))
+ /* In case it's a brokers full refresh call,
+ * avoid retrying it on this same broker.
+ * This is to prevent client is hung
+ * until it can connect to this broker again.
+ * No need to acquire the lock here but
+ * when decrementing the integer pointed
+ * by `decr`. */
+ if (!request->rkbuf_u.Metadata.decr &&
+ rd_kafka_buf_retry(rkb, request))
return;
/* FALLTHRU */
- } else {
+ }
+
+ if (actions & RD_KAFKA_ERR_ACTION_PERMANENT) {
rd_rkb_log(rkb, LOG_WARNING, "METADATA",
"Metadata request failed: %s: %s (%dms): %s",
request->rkbuf_u.Metadata.reason,
rd_kafka_err2str(err),
(int)(request->rkbuf_ts_sent / 1000),
rd_kafka_actions2str(actions));
- /* Respond back to caller on non-retriable errors */
- if (rko && rko->rko_replyq.q) {
- rko->rko_err = err;
- rko->rko_u.metadata.md = NULL;
- rko->rko_u.metadata.mdi = NULL;
- rd_kafka_replyq_enq(&rko->rko_replyq, rko, 0);
- rko = NULL;
- }
}
-
+ /* Respond back to caller on non-retriable errors */
+ if (rko && rko->rko_replyq.q) {
+ rko->rko_err = err;
+ rko->rko_u.metadata.md = NULL;
+ rko->rko_u.metadata.mdi = NULL;
+ rd_kafka_replyq_enq(&rko->rko_replyq, rko, 0);
+ rko = NULL;
+ }
/* FALLTHRU */
@@ -2654,6 +2749,18 @@ static void rd_kafka_handle_Metadata(rd_kafka_t *rk,
*
* @sa rd_kafka_MetadataRequest().
* @sa rd_kafka_MetadataRequest_resp_cb().
+ *
+ * @locality any
+ * @locks none
+ * @locks_acquired
+ * - mtx_lock(&rkb->rkb_rk->rk_metadata_cache.rkmc_full_lock) in
+ * this function
+ * - rd_kafka_broker_lock() in
+ * rd_kafka_broker_ApiVersion_supported()
+ * - rd_kafka_timers_lock() in
+ * rd_kafka_rebootstrap_tmr_start_maybe()
+ * - mtx_lock(&rkq->rkq_lock) in
+ * rd_kafka_broker_buf_enq_replyq()
*/
static rd_kafka_resp_err_t
rd_kafka_MetadataRequest0(rd_kafka_broker_t *rkb,
@@ -2664,6 +2771,7 @@ rd_kafka_MetadataRequest0(rd_kafka_broker_t *rkb,
rd_bool_t include_cluster_authorized_operations,
rd_bool_t include_topic_authorized_operations,
rd_bool_t cgrp_update,
+ int32_t cgrp_subscription_version,
rd_bool_t force_racks,
rd_kafka_op_t *rko,
rd_kafka_resp_cb_t *resp_cb,
@@ -2680,7 +2788,7 @@ rd_kafka_MetadataRequest0(rd_kafka_broker_t *rkb,
int *full_incr = NULL;
void *handler_arg = NULL;
rd_kafka_resp_cb_t *handler_cb = rd_kafka_handle_Metadata;
- int16_t metadata_max_version = 12;
+ int16_t metadata_max_version = 13;
rd_kafka_replyq_t use_replyq = replyq;
/* In case we want cluster authorized operations in the Metadata
@@ -2707,9 +2815,10 @@ rd_kafka_MetadataRequest0(rd_kafka_broker_t *rkb,
if (!reason)
reason = "";
- rkbuf->rkbuf_u.Metadata.reason = rd_strdup(reason);
- rkbuf->rkbuf_u.Metadata.cgrp_update = cgrp_update;
- rkbuf->rkbuf_u.Metadata.force_racks = force_racks;
+ rkbuf->rkbuf_u.Metadata.reason = rd_strdup(reason);
+ rkbuf->rkbuf_u.Metadata.cgrp_update = cgrp_update;
+ rkbuf->rkbuf_u.Metadata.force_racks = force_racks;
+ rkbuf->rkbuf_u.Metadata.cgrp_subscription_version = -1;
/* TopicArrayCnt */
of_TopicArrayCnt = rd_kafka_buf_write_arraycnt_pos(rkbuf);
@@ -2785,9 +2894,28 @@ rd_kafka_MetadataRequest0(rd_kafka_broker_t *rkb,
rkbuf->rkbuf_u.Metadata.decr = full_incr;
rkbuf->rkbuf_u.Metadata.decr_lock =
&rkb->rkb_rk->rk_metadata_cache.rkmc_full_lock;
+ } else if (!resp_cb) {
+ /*
+ * Conditions for starting the rebootstrap timer:
+ *
+ * !full_incr:
+ * In case it's a full request, forced or not, it won't be
+ * retried on the same broker to avoid blocking metadata
+ * requests, because of the lock, when that broker isn't
+ * available. We don't start the timer as we cannot ensure the
+ * request is retried for the duration of
+ * `metadata.recovery.rebootstrap.trigger.ms`.
+ *
+ * !resp_cb:
+ * Same reasoning applies when we use a custom callback, for the
+ * AdminClient requests for example.
+ *
+ * Start the rebootstrap timer only if it's the first
+ * metadata refresh request after last successful response,
+ * so the timer is not reset if already scheduled. */
+ rd_kafka_rebootstrap_tmr_start_maybe(rkb->rkb_rk);
}
-
if (topic_cnt > 0) {
char *topic;
int i;
@@ -2884,6 +3012,13 @@ rd_kafka_MetadataRequest0(rd_kafka_broker_t *rkb,
if (!use_replyq.q)
use_replyq = RD_KAFKA_REPLYQ(rkb->rkb_rk->rk_ops, 0);
+ if (cgrp_update && rkb->rkb_rk->rk_cgrp && total_topic_cnt > 0) {
+ rkbuf->rkbuf_u.Metadata.cgrp_subscription_version =
+ cgrp_subscription_version >= 0
+ ? cgrp_subscription_version
+ : rd_atomic32_get(
+ &rkb->rkb_rk->rk_cgrp->rkcg_subscription_version);
+ }
rd_kafka_broker_buf_enq_replyq(
rkb, rkbuf, use_replyq,
/* The default response handler is rd_kafka_handle_Metadata, but we
@@ -2910,6 +3045,7 @@ rd_kafka_MetadataRequest0(rd_kafka_broker_t *rkb,
* This is best-effort, depending on broker
* config and version.
* @param cgrp_update - Update cgrp in parse_Metadata (see comment there).
+ * @param subscription_version - Consumer group subscription version.
* @param force_racks - Force partition to rack mapping computation in
* parse_Metadata (see comment there).
* @param rko - (optional) rko with replyq for handling response.
@@ -2932,13 +3068,14 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb,
const char *reason,
rd_bool_t allow_auto_create_topics,
rd_bool_t cgrp_update,
+ int32_t cgrp_subscription_version,
rd_bool_t force_racks,
rd_kafka_op_t *rko) {
return rd_kafka_MetadataRequest0(
rkb, topics, topic_ids, reason, allow_auto_create_topics,
rd_false /*don't include cluster authorized operations*/,
rd_false /*don't include topic authorized operations*/, cgrp_update,
- force_racks, rko,
+ cgrp_subscription_version, force_racks, rko,
/* We use the default rd_kafka_handle_Metadata rather than a custom
resp_cb */
NULL,
@@ -2997,6 +3134,7 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest_resp_cb(
rd_bool_t include_cluster_authorized_operations,
rd_bool_t include_topic_authorized_operations,
rd_bool_t cgrp_update,
+ int32_t cgrp_subscription_version,
rd_bool_t force_racks,
rd_kafka_resp_cb_t *resp_cb,
rd_kafka_replyq_t replyq,
@@ -3005,7 +3143,8 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest_resp_cb(
return rd_kafka_MetadataRequest0(
rkb, topics, topics_ids, reason, allow_auto_create_topics,
include_cluster_authorized_operations,
- include_topic_authorized_operations, cgrp_update, force_racks,
+ include_topic_authorized_operations, cgrp_update,
+ cgrp_subscription_version, force_racks,
NULL /* No op - using custom resp_cb. */, resp_cb, replyq, force,
opaque);
}
@@ -3230,6 +3369,9 @@ void rd_kafka_handle_SaslAuthenticate(rd_kafka_t *rk,
rd_kafkap_bytes_t auth_data;
char errstr[512];
+ if (rd_kafka_broker_is_any_err_destroy(err))
+ return;
+
if (err) {
rd_snprintf(errstr, sizeof(errstr),
"SaslAuthenticateRequest failed: %s",
@@ -5245,7 +5387,9 @@ rd_kafka_AlterConfigsRequest(rd_kafka_broker_t *rkb,
int ei;
/* ResourceType */
- rd_kafka_buf_write_i8(rkbuf, config->restype);
+ rd_kafka_buf_write_i8(
+ rkbuf, rd_kafka_ResourceType_to_ConfigResourceType(
+ config->restype));
/* ResourceName */
rd_kafka_buf_write_str(rkbuf, config->name, -1);
@@ -5327,7 +5471,9 @@ rd_kafka_resp_err_t rd_kafka_IncrementalAlterConfigsRequest(
int ei;
/* ResourceType */
- rd_kafka_buf_write_i8(rkbuf, config->restype);
+ rd_kafka_buf_write_i8(
+ rkbuf, rd_kafka_ResourceType_to_ConfigResourceType(
+ config->restype));
/* ResourceName */
rd_kafka_buf_write_str(rkbuf, config->name, -1);
@@ -5421,7 +5567,9 @@ rd_kafka_resp_err_t rd_kafka_DescribeConfigsRequest(
int ei;
/* resource_type */
- rd_kafka_buf_write_i8(rkbuf, config->restype);
+ rd_kafka_buf_write_i8(
+ rkbuf, rd_kafka_ResourceType_to_ConfigResourceType(
+ config->restype));
/* resource_name */
rd_kafka_buf_write_str(rkbuf, config->name, -1);
@@ -5870,6 +6018,150 @@ rd_kafka_DeleteAclsRequest(rd_kafka_broker_t *rkb,
return RD_KAFKA_RESP_ERR_NO_ERROR;
}
+/**
+ * @brief Construct and send ElectLeadersRequest to \p rkb
+ * with the partitions (ElectLeaders_t*) in \p elect_leaders, using
+ * \p options.
+ *
+ * The response (unparsed) will be enqueued on \p replyq
+ * for handling by \p resp_cb (with \p opaque passed).
+ *
+ * @returns RD_KAFKA_RESP_ERR_NO_ERROR if the request was enqueued for
+ * transmission, otherwise an error code and errstr will be
+ * updated with a human readable error string.
+ */
+rd_kafka_resp_err_t rd_kafka_ElectLeadersRequest(
+ rd_kafka_broker_t *rkb,
+ const rd_list_t *elect_leaders /*(rd_kafka_EleactLeaders_t*)*/,
+ rd_kafka_AdminOptions_t *options,
+ char *errstr,
+ size_t errstr_size,
+ rd_kafka_replyq_t replyq,
+ rd_kafka_resp_cb_t *resp_cb,
+ void *opaque) {
+ rd_kafka_buf_t *rkbuf;
+ int16_t ApiVersion;
+ const rd_kafka_ElectLeaders_t *elect_leaders_request;
+ int rd_buf_size_estimate;
+ int op_timeout;
+
+ if (rd_list_cnt(elect_leaders) == 0) {
+ rd_snprintf(errstr, errstr_size,
+ "No partitions specified for leader election");
+ rd_kafka_replyq_destroy(&replyq);
+ return RD_KAFKA_RESP_ERR__INVALID_ARG;
+ }
+
+ elect_leaders_request = rd_list_elem(elect_leaders, 0);
+
+ ApiVersion = rd_kafka_broker_ApiVersion_supported(
+ rkb, RD_KAFKAP_ElectLeaders, 0, 2, NULL);
+ if (ApiVersion == -1) {
+ rd_snprintf(errstr, errstr_size,
+ "ElectLeaders Admin API (KIP-460) not supported "
+ "by broker, requires broker version >= 2.4.0");
+ rd_kafka_replyq_destroy(&replyq);
+ return RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE;
+ }
+
+ rd_buf_size_estimate =
+ 1 /* ElectionType */ + 4 /* #TopicPartitions */ + 4 /* TimeoutMs */;
+ if (elect_leaders_request->partitions)
+ rd_buf_size_estimate +=
+ (50 + 4) * elect_leaders_request->partitions->cnt;
+ rkbuf = rd_kafka_buf_new_flexver_request(rkb, RD_KAFKAP_ElectLeaders, 1,
+ rd_buf_size_estimate,
+ ApiVersion >= 2);
+
+ if (ApiVersion >= 1) {
+ /* Election type */
+ rd_kafka_buf_write_i8(rkbuf,
+ elect_leaders_request->election_type);
+ }
+
+ /* Write partition list */
+ if (elect_leaders_request->partitions) {
+ const rd_kafka_topic_partition_field_t fields[] = {
+ RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION,
+ RD_KAFKA_TOPIC_PARTITION_FIELD_END};
+ rd_kafka_buf_write_topic_partitions(
+ rkbuf, elect_leaders_request->partitions,
+ rd_false /*don't skip invalid offsets*/,
+ rd_false /* any offset */,
+ rd_false /* don't use topic_id */,
+ rd_true /* use topic_names */, fields);
+ } else {
+ rd_kafka_buf_write_arraycnt(rkbuf, -1);
+ }
+
+ /* timeout */
+ op_timeout = rd_kafka_confval_get_int(&options->operation_timeout);
+ rd_kafka_buf_write_i32(rkbuf, op_timeout);
+
+ if (op_timeout > rkb->rkb_rk->rk_conf.socket_timeout_ms)
+ rd_kafka_buf_set_abs_timeout(rkbuf, op_timeout + 1000, 0);
+
+ rd_kafka_buf_ApiVersion_set(rkbuf, ApiVersion, 0);
+
+ rd_kafka_broker_buf_enq_replyq(rkb, rkbuf, replyq, resp_cb, opaque);
+
+ return RD_KAFKA_RESP_ERR_NO_ERROR;
+}
+
+/**
+ * @brief Construct and send ConsumerGroupDescribe requests
+ * to \p rkb with the groups (const char *) in \p groups.
+ * Uses \p include_authorized_operations to get
+ * group ACL authorized operations.
+ *
+ * The response (unparsed) will be enqueued on \p replyq
+ * for handling by \p resp_cb (with \p opaque passed).
+ *
+ * @return NULL on success, a new error instance that must be
+ * released with rd_kafka_error_destroy() in case of error.
+ */
+rd_kafka_error_t *
+rd_kafka_ConsumerGroupDescribeRequest(rd_kafka_broker_t *rkb,
+ char **groups,
+ size_t group_cnt,
+ rd_bool_t include_authorized_operations,
+ rd_kafka_replyq_t replyq,
+ rd_kafka_resp_cb_t *resp_cb,
+ void *opaque) {
+ rd_kafka_buf_t *rkbuf;
+ size_t i;
+
+ int16_t ApiVersion = rd_kafka_broker_ApiVersion_supported(
+ rkb, RD_KAFKAP_ConsumerGroupDescribe, 0, 0, NULL);
+
+ if (ApiVersion == -1) {
+ return rd_kafka_error_new(
+ RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE,
+ "ConsumerGroupDescribe (KIP-848) "
+ "not supported by broker, "
+ "requires broker version >= 4.0.0");
+ }
+
+ rkbuf = rd_kafka_buf_new_flexver_request(
+ rkb, RD_KAFKAP_ConsumerGroupDescribe, 1,
+ 4 /* rd_kafka_buf_write_arraycnt_pos */ +
+ 1 /* IncludeAuthorizedOperations */ + 1 /* tags */ +
+ 32 * group_cnt /* Groups */,
+ rd_true /* flexver */);
+
+ rd_kafka_buf_write_arraycnt(rkbuf, group_cnt);
+
+ for (i = 0; i < group_cnt; i++) {
+ rd_kafka_buf_write_str(rkbuf, groups[i], -1);
+ }
+
+ rd_kafka_buf_write_bool(rkbuf, include_authorized_operations);
+ rd_kafka_buf_ApiVersion_set(rkbuf, ApiVersion, 0);
+ rd_kafka_broker_buf_enq_replyq(rkb, rkbuf, replyq, resp_cb, opaque);
+
+ return NULL;
+}
+
/**
* @brief Parses and handles an InitProducerId reply.
*
@@ -6261,16 +6553,17 @@ rd_kafka_PushTelemetryRequest(rd_kafka_broker_t *rkb,
}
size_t len = sizeof(rd_kafka_Uuid_t) + sizeof(int32_t) +
- sizeof(rd_bool_t) + sizeof(compression_type) +
- metrics_size;
+ sizeof(rd_bool_t) + sizeof(int8_t) + metrics_size;
rkbuf = rd_kafka_buf_new_flexver_request(rkb, RD_KAFKAP_PushTelemetry,
1, len, rd_true);
rd_kafka_buf_write_uuid(rkbuf, client_instance_id);
rd_kafka_buf_write_i32(rkbuf, subscription_id);
rd_kafka_buf_write_bool(rkbuf, terminating);
- rd_kafka_buf_write_i8(rkbuf, compression_type);
+ rd_kafka_buf_write_i8(rkbuf, (int8_t)compression_type);
+ rd_dassert(metrics != NULL);
+ rd_dassert(metrics_size >= 0);
rd_kafkap_bytes_t *metric_bytes =
rd_kafkap_bytes_new(metrics, metrics_size);
rd_kafka_buf_write_kbytes(rkbuf, metric_bytes);
@@ -6332,10 +6625,12 @@ void rd_kafka_handle_GetTelemetrySubscriptions(rd_kafka_t *rk,
rk->rk_telemetry.accepted_compression_types =
rd_calloc(arraycnt, sizeof(rd_kafka_compression_t));
- for (i = 0; i < (size_t)arraycnt; i++)
- rd_kafka_buf_read_i8(
- rkbuf,
- &rk->rk_telemetry.accepted_compression_types[i]);
+ for (i = 0; i < (size_t)arraycnt; i++) {
+ int8_t AcceptedCompressionType;
+ rd_kafka_buf_read_i8(rkbuf, &AcceptedCompressionType);
+ rk->rk_telemetry.accepted_compression_types[i] =
+ AcceptedCompressionType;
+ }
} else {
rk->rk_telemetry.accepted_compression_types_cnt = 1;
rk->rk_telemetry.accepted_compression_types =
@@ -6375,7 +6670,7 @@ void rd_kafka_handle_GetTelemetrySubscriptions(rd_kafka_t *rk,
rd_kafkap_str_t Metric;
rd_kafka_buf_read_str(rkbuf, &Metric);
rk->rk_telemetry.requested_metrics[i] =
- rd_strdup(Metric.str);
+ RD_KAFKAP_STR_DUP(&Metric);
}
}
@@ -6700,6 +6995,85 @@ static int unittest_idempotent_producer(void) {
return 0;
}
+/**
+ * @brief Test for the GetTelemetrySubscriptions response handling.
+ *
+ * @returns 1 on failure, 0 on success.
+ */
+static int unittest_handle_GetTelemetrySubscriptions(void) {
+ rd_kafka_t *rk;
+ rd_kafka_broker_t *rkb;
+ rd_kafka_buf_t *rkbuf;
+
+ RD_UT_SAY("Verifying GetTelemetrySubscriptions response handling");
+
+ rk = rd_kafka_new(RD_KAFKA_CONSUMER, NULL, NULL, 0);
+ rkb = rd_kafka_broker_add_logical(rk, "unittest");
+
+ rkbuf = rd_kafka_buf_new(0, 0);
+ rkbuf->rkbuf_rkb = rkb;
+ rd_kafka_buf_write_i32(rkbuf, 0); /* ThrottleTime */
+ rd_kafka_buf_write_i16(rkbuf, 0); /* ErrorCode */
+
+ rd_kafka_buf_write_uuid(rkbuf, &rk->rk_telemetry.client_instance_id);
+
+ rd_kafka_buf_write_i32(rkbuf, 0); /* SubscriptionId */
+
+ rd_kafka_buf_write_arraycnt(rkbuf, 2); /* #AcceptedCompressionTypes */
+ /* AcceptedCompressionTypes[0] */
+ rd_kafka_buf_write_i8(rkbuf, RD_KAFKA_COMPRESSION_GZIP);
+ /* AcceptedCompressionTypes[1] */
+ rd_kafka_buf_write_i8(rkbuf, RD_KAFKA_COMPRESSION_LZ4);
+
+ rd_kafka_buf_write_i32(rkbuf, 0); /* PushIntervalMs */
+ rd_kafka_buf_write_i32(rkbuf, 0); /* TelemetryMaxBytes */
+ rd_kafka_buf_write_bool(rkbuf, 0); /* DeltaTemporality */
+
+ rd_kafka_buf_write_arraycnt(rkbuf, 2); /* #RequestedMetrics */
+ /* RequestedMetrics[0] */
+ rd_kafka_buf_write_str(rkbuf, "metric1", -1);
+ /* RequestedMetrics[1] */
+ rd_kafka_buf_write_str(rkbuf, "metric2", -1);
+
+ /* Set up a buffer reader for sending the buffer. */
+ rd_slice_init_full(&rkbuf->rkbuf_reader, &rkbuf->rkbuf_buf);
+
+ /* Handle the response */
+ rd_kafka_handle_GetTelemetrySubscriptions(
+ rk, rkb, RD_KAFKA_RESP_ERR_NO_ERROR, rkbuf, NULL, NULL);
+
+
+ RD_UT_ASSERT(rk->rk_telemetry.accepted_compression_types_cnt == 2,
+ "Expected 2 accepted compression types, got %" PRIusz,
+ rk->rk_telemetry.accepted_compression_types_cnt);
+ RD_UT_ASSERT(rk->rk_telemetry.accepted_compression_types[0] ==
+ RD_KAFKA_COMPRESSION_GZIP,
+ "Expected 'gzip' compression type, got '%s'",
+ rd_kafka_compression2str(
+ rk->rk_telemetry.accepted_compression_types[0]));
+ RD_UT_ASSERT(rk->rk_telemetry.accepted_compression_types[1] ==
+ RD_KAFKA_COMPRESSION_LZ4,
+ "Expected 'lz4' compression type, got '%s'",
+ rd_kafka_compression2str(
+ rk->rk_telemetry.accepted_compression_types[1]));
+
+ RD_UT_ASSERT(rk->rk_telemetry.requested_metrics_cnt == 2,
+ "Expected 2 requested metrics, got %" PRIusz,
+ rk->rk_telemetry.requested_metrics_cnt);
+ RD_UT_ASSERT(
+ rd_strcmp(rk->rk_telemetry.requested_metrics[0], "metric1") == 0,
+ "Expected 'metric1', got '%s'",
+ rk->rk_telemetry.requested_metrics[0]);
+ RD_UT_ASSERT(
+ rd_strcmp(rk->rk_telemetry.requested_metrics[1], "metric2") == 0,
+ "Expected 'metric2', got '%s'",
+ rk->rk_telemetry.requested_metrics[1]);
+
+ rd_kafka_buf_destroy(rkbuf);
+ rd_kafka_destroy(rk);
+ return 0;
+}
+
/**
* @brief Request/response unit tests
*/
@@ -6707,6 +7081,7 @@ int unittest_request(void) {
int fails = 0;
fails += unittest_idempotent_producer();
+ fails += unittest_handle_GetTelemetrySubscriptions();
return fails;
}
diff --git a/src/rdkafka_request.h b/src/rdkafka_request.h
index b291a324a3..c508ffdaaf 100644
--- a/src/rdkafka_request.h
+++ b/src/rdkafka_request.h
@@ -167,6 +167,14 @@ rd_kafka_topic_partition_list_t *rd_kafka_buf_read_topic_partitions(
size_t estimated_part_cnt,
const rd_kafka_topic_partition_field_t *fields);
+rd_kafka_topic_partition_list_t *rd_kafka_buf_read_topic_partitions_nullable(
+ rd_kafka_buf_t *rkbuf,
+ rd_bool_t use_topic_id,
+ rd_bool_t use_topic_name,
+ size_t estimated_part_cnt,
+ const rd_kafka_topic_partition_field_t *fields,
+ rd_bool_t *parse_err);
+
int rd_kafka_buf_write_topic_partitions(
rd_kafka_buf_t *rkbuf,
const rd_kafka_topic_partition_list_t *parts,
@@ -237,6 +245,9 @@ void rd_kafka_OffsetForLeaderEpochRequest(
rd_kafka_resp_cb_t *resp_cb,
void *opaque);
+int rd_kafka_handle_OffsetFetch_err_action(rd_kafka_broker_t *rkb,
+ rd_kafka_resp_err_t err,
+ rd_kafka_buf_t *request);
rd_kafka_resp_err_t
rd_kafka_handle_OffsetFetch(rd_kafka_t *rk,
@@ -347,6 +358,8 @@ rd_kafka_error_t *rd_kafka_ListGroupsRequest(rd_kafka_broker_t *rkb,
int16_t max_ApiVersion,
const char **states,
size_t states_cnt,
+ const char **types,
+ size_t types_cnt,
rd_kafka_replyq_t replyq,
rd_kafka_resp_cb_t *resp_cb,
void *opaque);
@@ -379,7 +392,8 @@ void rd_kafka_ConsumerGroupHeartbeatRequest(
const rd_kafkap_str_t *group_instance_id,
const rd_kafkap_str_t *rack_id,
int32_t rebalance_timeout_ms,
- const rd_kafka_topic_partition_list_t *subscribe_topics,
+ const rd_kafka_topic_partition_list_t *subscribed_topics,
+ rd_kafkap_str_t *subscribed_topic_regex,
const rd_kafkap_str_t *remote_assignor,
const rd_kafka_topic_partition_list_t *current_assignments,
rd_kafka_replyq_t replyq,
@@ -392,6 +406,7 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb,
const char *reason,
rd_bool_t allow_auto_create_topics,
rd_bool_t cgrp_update,
+ int32_t cgrp_subscription_version,
rd_bool_t force_racks,
rd_kafka_op_t *rko);
@@ -404,6 +419,7 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest_resp_cb(
rd_bool_t include_cluster_authorized_operations,
rd_bool_t include_topic_authorized_operations,
rd_bool_t cgrp_update,
+ int32_t cgrp_subscription_version,
rd_bool_t force_racks,
rd_kafka_resp_cb_t *resp_cb,
rd_kafka_replyq_t replyq,
@@ -619,6 +635,25 @@ rd_kafka_DeleteAclsRequest(rd_kafka_broker_t *rkb,
rd_kafka_resp_cb_t *resp_cb,
void *opaque);
+rd_kafka_resp_err_t rd_kafka_ElectLeadersRequest(
+ rd_kafka_broker_t *rkb,
+ const rd_list_t *elect_leaders /*(rd_kafka_EleactLeaders_t*)*/,
+ rd_kafka_AdminOptions_t *options,
+ char *errstr,
+ size_t errstr_size,
+ rd_kafka_replyq_t replyq,
+ rd_kafka_resp_cb_t *resp_cb,
+ void *opaque);
+
+rd_kafka_error_t *
+rd_kafka_ConsumerGroupDescribeRequest(rd_kafka_broker_t *rkb,
+ char **groups,
+ size_t group_cnt,
+ rd_bool_t include_authorized_operations,
+ rd_kafka_replyq_t replyq,
+ rd_kafka_resp_cb_t *resp_cb,
+ void *opaque);
+
void rd_kafkap_leader_discovery_tmpabuf_add_alloc_brokers(
rd_tmpabuf_t *tbuf,
rd_kafkap_NodeEndpoints_t *NodeEndpoints);
diff --git a/src/rdkafka_sasl.c b/src/rdkafka_sasl.c
index 32ebe3b198..4cd5027eec 100644
--- a/src/rdkafka_sasl.c
+++ b/src/rdkafka_sasl.c
@@ -172,12 +172,14 @@ int rd_kafka_sasl_io_event(rd_kafka_transport_t *rktrans,
r = rd_kafka_transport_framed_recv(rktrans, &rkbuf, errstr,
errstr_size);
if (r == -1) {
- if (!strcmp(errstr, "Disconnected"))
- rd_snprintf(errstr, errstr_size,
- "Disconnected: check client %s credentials "
+ if (rd_kafka_transport_error_disconnected(errstr)) {
+ int curr_len = strlen(errstr);
+ rd_snprintf(errstr + curr_len, errstr_size - curr_len,
+ ": check client %s credentials "
"and broker logs",
rktrans->rktrans_rkb->rkb_rk->rk_conf.sasl
.mechanisms);
+ }
return -1;
} else if (r == 0) /* not fully received yet */
return 0;
diff --git a/src/rdkafka_sasl_oauthbearer.c b/src/rdkafka_sasl_oauthbearer.c
index 2065751ccb..daf77da8fb 100644
--- a/src/rdkafka_sasl_oauthbearer.c
+++ b/src/rdkafka_sasl_oauthbearer.c
@@ -124,9 +124,10 @@ struct rd_kafka_sasl_oauthbearer_token {
* @brief Per-connection state
*/
struct rd_kafka_sasl_oauthbearer_state {
- enum { RD_KAFKA_SASL_OAUTHB_STATE_SEND_CLIENT_FIRST_MESSAGE,
- RD_KAFKA_SASL_OAUTHB_STATE_RECV_SERVER_FIRST_MSG,
- RD_KAFKA_SASL_OAUTHB_STATE_RECV_SERVER_MSG_AFTER_FAIL,
+ enum {
+ RD_KAFKA_SASL_OAUTHB_STATE_SEND_CLIENT_FIRST_MESSAGE,
+ RD_KAFKA_SASL_OAUTHB_STATE_RECV_SERVER_FIRST_MSG,
+ RD_KAFKA_SASL_OAUTHB_STATE_RECV_SERVER_MSG_AFTER_FAIL,
} state;
char *server_error_msg;
@@ -1091,8 +1092,8 @@ static void rd_kafka_sasl_oauthbearer_build_client_first_message(
buf = out->ptr;
size_written = 0;
r = rd_snprintf(buf, out->size + 1 - size_written,
- "%s%sauth=Bearer %s%s", gs2_header, kvsep,
- state->token_value, kvsep);
+ "%s%sauth=Bearer %s%s", gs2_header, kvsep,
+ state->token_value, kvsep);
rd_assert(r < out->size + 1 - size_written);
size_written += r;
buf = out->ptr + size_written;
@@ -1255,8 +1256,13 @@ static int rd_kafka_sasl_oauthbearer_client_new(rd_kafka_transport_t *rktrans,
return -1;
}
- state->token_value = rd_strdup(handle->token_value);
- state->md_principal_name = rd_strdup(handle->md_principal_name);
+ state->token_value = rd_strdup(handle->token_value);
+ if (handle->md_principal_name)
+ state->md_principal_name = rd_strdup(handle->md_principal_name);
+ else
+ state->md_principal_name = NULL;
+
+ rd_list_init_copy(&state->extensions, &handle->extensions);
rd_list_copy_to(&state->extensions, &handle->extensions,
rd_strtup_list_copy, NULL);
@@ -1332,13 +1338,11 @@ static int rd_kafka_sasl_oauthbearer_init(rd_kafka_t *rk,
#if WITH_OAUTHBEARER_OIDC
if (rk->rk_conf.sasl.oauthbearer.method ==
RD_KAFKA_SASL_OAUTHBEARER_METHOD_OIDC &&
- rk->rk_conf.sasl.oauthbearer.token_refresh_cb ==
- rd_kafka_oidc_token_refresh_cb) {
+ rk->rk_conf.sasl.oauthbearer.builtin_token_refresh_cb) {
handle->internal_refresh = rd_true;
rd_kafka_sasl_background_callbacks_enable(rk);
}
#endif
-
/* Otherwise enqueue a refresh callback for the application. */
rd_kafka_oauthbearer_enqueue_token_refresh(handle);
diff --git a/src/rdkafka_sasl_oauthbearer_oidc.c b/src/rdkafka_sasl_oauthbearer_oidc.c
index d56efbf355..b1d729638c 100644
--- a/src/rdkafka_sasl_oauthbearer_oidc.c
+++ b/src/rdkafka_sasl_oauthbearer_oidc.c
@@ -51,8 +51,9 @@
*
* @locality Any thread.
*/
-static char *rd_kafka_oidc_build_auth_header(const char *client_id,
- const char *client_secret) {
+static char *
+rd_kafka_oidc_client_credentials_build_auth_header(const char *client_id,
+ const char *client_secret) {
rd_chariov_t client_authorization_in;
rd_chariov_t client_authorization_out;
@@ -90,13 +91,15 @@ static char *rd_kafka_oidc_build_auth_header(const char *client_id,
*
* @locality Any thread.
*/
-static void rd_kafka_oidc_build_headers(const char *client_id,
- const char *client_secret,
- struct curl_slist **headersp) {
+static void
+rd_kafka_oidc_client_credentials_build_headers(const char *client_id,
+ const char *client_secret,
+ struct curl_slist **headersp) {
char *authorization_base64_header;
authorization_base64_header =
- rd_kafka_oidc_build_auth_header(client_id, client_secret);
+ rd_kafka_oidc_client_credentials_build_auth_header(client_id,
+ client_secret);
*headersp = curl_slist_append(*headersp, "Accept: application/json");
*headersp = curl_slist_append(*headersp, authorization_base64_header);
@@ -198,9 +201,10 @@ static const char *rd_kafka_jwt_b64_decode_payload(const char *src,
* The post_fields_size will be returned in \p post_fields_size.
*
*/
-static void rd_kafka_oidc_build_post_fields(const char *scope,
- char **post_fields,
- size_t *post_fields_size) {
+static void
+rd_kafka_oidc_client_credentials_build_post_fields(const char *scope,
+ char **post_fields,
+ size_t *post_fields_size) {
size_t scope_size = 0;
if (scope)
@@ -217,6 +221,684 @@ static void rd_kafka_oidc_build_post_fields(const char *scope,
}
}
+/**
+ * @brief Get JWT algorithm label string for the specified signing algorithm.
+ *
+ * @param token_signing_algo The algorithm enum value
+ *
+ * @returns String representation of the algorithm.
+ *
+ * @locality Any thread.
+ */
+static char *rd_kafka_oidc_assertion_get_algo_label(
+ const rd_kafka_oauthbearer_assertion_algorithm_t token_signing_algo) {
+ switch (token_signing_algo) {
+ case RD_KAFKA_SASL_OAUTHBEARER_ASSERTION_ALGORITHM_RS256:
+ return "RS256";
+ case RD_KAFKA_SASL_OAUTHBEARER_ASSERTION_ALGORITHM_ES256:
+ return "ES256";
+ default:
+ rd_assert(!*"Unknown JOSE algorithm");
+ return NULL;
+ }
+}
+
+/**
+ * @brief Parse a JWT template file and extract header and payload JSON
+ * objects.
+ *
+ * Reads and parses the JWT template file, which should contain a JSON object
+ * with "header" and "payload" properties.
+ *
+ * @param rk
+ * @param jwt_template_file_path Path to the template file
+ * @param header Pointer to store the parsed header JSON object
+ * @param payload Pointer to store the parsed payload JSON object
+ *
+ * @returns 0 on success, -1 on failure
+ *
+ * @locality Any thread.
+ */
+static int
+rd_kafka_oidc_assertion_parse_template_file(rd_kafka_t *rk,
+ const char *jwt_template_file_path,
+ cJSON **header,
+ cJSON **payload) {
+ char *template_content = NULL;
+ cJSON *template_json = NULL;
+ int ret = -1;
+ size_t file_size;
+
+ *header = NULL;
+ *payload = NULL;
+
+ template_content =
+ rd_file_read(jwt_template_file_path, &file_size, 1024 * 1024);
+ if (!template_content) {
+ rd_kafka_log(rk, LOG_ERR, "JWT",
+ "Failed to open JWT template file: %s",
+ jwt_template_file_path);
+ return -1;
+ }
+
+ if (file_size == 0) {
+ rd_kafka_log(rk, LOG_ERR, "JWT",
+ "JWT template file is empty or invalid");
+ rd_free(template_content);
+ return -1;
+ }
+
+ template_json = cJSON_Parse((char *)template_content);
+ if (!template_json) {
+ rd_kafka_log(rk, LOG_ERR, "JWT",
+ "Failed to parse JWT template JSON");
+ goto cleanup;
+ }
+
+ cJSON *header_item = cJSON_GetObjectItem(template_json, "header");
+ cJSON *payload_item = cJSON_GetObjectItem(template_json, "payload");
+
+ if (!header_item || !payload_item) {
+ rd_kafka_log(rk, LOG_ERR, "JWT",
+ "JWT template must contain both 'header' "
+ "and 'payload' objects");
+ goto cleanup;
+ }
+
+ *header = cJSON_Duplicate(header_item, 1);
+ *payload = cJSON_Duplicate(payload_item, 1);
+
+ if (!*header || !*payload) {
+ rd_kafka_log(rk, LOG_ERR, "JWT",
+ "Failed to duplicate header or payload objects");
+ if (*header) {
+ cJSON_Delete(*header);
+ *header = NULL;
+ }
+ goto cleanup;
+ }
+
+ ret = 0;
+
+cleanup:
+ if (template_content)
+ rd_free(template_content);
+ if (template_json)
+ cJSON_Delete(template_json);
+
+ return ret;
+}
+
+/**
+ * @brief Create JWT assertion.
+ *
+ * Creates a JWT token signed with the specified private key using the
+ * algorithm specified. The token can be created from a template file or
+ * will create a minimal default token if no template is provided.
+ *
+ * @param rk The rd_kafka_t instance for logging
+ * @param private_key_pem PEM formatted private key string (mutually exclusive
+ * with key_file_location)
+ * @param key_file_location Path to private key file (mutually exclusive with
+ * private_key_pem)
+ * @param passphrase Optional passphrase for encrypted private key
+ * @param token_signing_algo Algorithm to use for signing (RS256 or ES256)
+ * @param jwt_template_file Optional path to JWT template file
+ * @param subject Optional subject claim value.
+ * @param issuer Optional issuer claim value.
+ * @param audience Optional audience claim value.
+ * @param nbf `nbf` claim value to express seconds of validity in the past.
+ * @param exp `exp` claim value to express seconds of validity in the future.
+ * @param jti_include Whether to include a JTI claim (UUID)
+ *
+ * @returns Newly allocated JWT string, caller must free with rd_free(). NULL on
+ * error.
+ *
+ * @locality Any thread.
+ */
+static char *rd_kafka_oidc_assertion_create(
+ rd_kafka_t *rk,
+ const char *private_key_pem,
+ const char *key_file_location,
+ const char *passphrase,
+ const rd_kafka_oauthbearer_assertion_algorithm_t token_signing_algo,
+ const char *jwt_template_file,
+ const char *subject,
+ const char *issuer,
+ const char *audience,
+ const int nbf,
+ const int exp,
+ const rd_bool_t jti_include) {
+
+ char *encoded_header = NULL;
+ char *encoded_payload = NULL;
+ char *encoded_signature = NULL;
+ char *unsigned_token = NULL;
+ char *result = NULL;
+ char *header_str = NULL;
+ char *payload_str = NULL;
+ EVP_PKEY *pkey = NULL;
+ BIO *bio = NULL;
+ cJSON *header_json_obj = NULL;
+ cJSON *payload_json_obj = NULL;
+ EVP_MD_CTX *mdctx = NULL;
+ unsigned char *sig = NULL;
+ rd_chariov_t header_iov;
+ rd_chariov_t payload_iov;
+ rd_chariov_t sig_iov;
+ rd_kafka_Uuid_t jti_uuid;
+ char *jti_uuid_str = NULL;
+
+ rd_ts_t issued_at = rd_uclock() / 1000000;
+ rd_ts_t not_before = issued_at - nbf;
+ rd_ts_t expiration_time = issued_at + exp;
+
+ if (jwt_template_file) {
+ if (rd_kafka_oidc_assertion_parse_template_file(
+ rk, jwt_template_file, &header_json_obj,
+ &payload_json_obj) != 0) {
+ rd_kafka_log(rk, LOG_ERR, "JWT",
+ "Failed to process JWT template file %s",
+ jwt_template_file);
+ return NULL;
+ }
+ } else {
+ header_json_obj = cJSON_CreateObject();
+ payload_json_obj = cJSON_CreateObject();
+ }
+
+ /* Add required header fields */
+ cJSON_DeleteItemFromObjectCaseSensitive(header_json_obj, "alg");
+ cJSON_DeleteItemFromObjectCaseSensitive(header_json_obj, "typ");
+ cJSON_DeleteItemFromObjectCaseSensitive(payload_json_obj, "iat");
+ cJSON_DeleteItemFromObjectCaseSensitive(payload_json_obj, "exp");
+ cJSON_DeleteItemFromObjectCaseSensitive(payload_json_obj, "nbf");
+ cJSON_AddStringToObject(
+ header_json_obj, "alg",
+ rd_kafka_oidc_assertion_get_algo_label(token_signing_algo));
+ cJSON_AddStringToObject(header_json_obj, "typ", "JWT");
+
+ /* Add required payload fields */
+ cJSON_AddNumberToObject(payload_json_obj, "iat", (double)issued_at);
+ cJSON_AddNumberToObject(payload_json_obj, "exp",
+ (double)expiration_time);
+ cJSON_AddNumberToObject(payload_json_obj, "nbf", (double)not_before);
+
+ if (subject) {
+ cJSON_DeleteItemFromObjectCaseSensitive(payload_json_obj,
+ "sub");
+ cJSON_AddStringToObject(payload_json_obj, "sub", subject);
+ }
+
+ if (issuer) {
+ cJSON_DeleteItemFromObjectCaseSensitive(payload_json_obj,
+ "iss");
+ cJSON_AddStringToObject(payload_json_obj, "iss", issuer);
+ }
+
+ if (audience) {
+ cJSON_DeleteItemFromObjectCaseSensitive(payload_json_obj,
+ "aud");
+ cJSON_AddStringToObject(payload_json_obj, "aud", audience);
+ }
+
+ if (jti_include) {
+ jti_uuid = rd_kafka_Uuid_random();
+ jti_uuid_str = rd_kafka_Uuid_str(&jti_uuid);
+ cJSON_DeleteItemFromObjectCaseSensitive(payload_json_obj,
+ "jti");
+ cJSON_AddStringToObject(payload_json_obj, "jti", jti_uuid_str);
+ rd_free(jti_uuid_str);
+ }
+
+ header_str = cJSON_PrintUnformatted(header_json_obj);
+ payload_str = cJSON_PrintUnformatted(payload_json_obj);
+
+ if (!header_str || !payload_str) {
+ rd_kafka_log(rk, LOG_ERR, "JWT",
+ "Failed to convert template objects to JSON");
+ goto cleanup;
+ }
+
+ header_iov.ptr = header_str;
+ header_iov.size = strlen(header_str);
+ encoded_header = rd_base64_encode_str_urlsafe(&header_iov);
+
+ payload_iov.ptr = payload_str;
+ payload_iov.size = strlen(payload_str);
+ encoded_payload = rd_base64_encode_str_urlsafe(&payload_iov);
+ if (!encoded_header || !encoded_payload)
+ goto cleanup;
+
+ size_t unsigned_token_len =
+ strlen(encoded_header) + strlen(encoded_payload) + 2;
+ unsigned_token = rd_malloc(unsigned_token_len);
+
+ if (!unsigned_token)
+ goto cleanup;
+ rd_snprintf(unsigned_token, unsigned_token_len, "%s.%s", encoded_header,
+ encoded_payload);
+
+ if (private_key_pem) {
+ bio = BIO_new_mem_buf((void *)private_key_pem, -1);
+ } else if (key_file_location) {
+ bio = BIO_new_file(key_file_location, "r");
+ }
+
+ if (!bio) {
+ rd_kafka_log(rk, LOG_ERR, "JWT",
+ "Failed to create BIO for private key");
+ goto cleanup;
+ }
+
+ if (passphrase) {
+ pkey = PEM_read_bio_PrivateKey(bio, NULL, NULL,
+ (void *)passphrase);
+ } else {
+ pkey = PEM_read_bio_PrivateKey(bio, NULL, NULL, NULL);
+ }
+ BIO_free(bio);
+ bio = NULL;
+
+ if (!pkey) {
+ rd_kafka_log(rk, LOG_ERR, "JWT", "Failed to load private key");
+ goto cleanup;
+ }
+
+ mdctx = EVP_MD_CTX_new();
+ if (!mdctx) {
+ rd_kafka_log(rk, LOG_ERR, "JWT",
+ "Failed to create message digest context");
+ goto cleanup;
+ }
+
+ const EVP_MD *md = EVP_sha256(); /* Both RS256 and ES256 use SHA-256 */
+
+ if (EVP_DigestSignInit(mdctx, NULL, md, NULL, pkey) != 1) {
+ rd_kafka_log(rk, LOG_ERR, "JWT",
+ "Failed to initialize signing context");
+ goto cleanup;
+ }
+
+ if (EVP_DigestSignUpdate(mdctx, unsigned_token,
+ strlen(unsigned_token)) != 1) {
+ rd_kafka_log(rk, LOG_ERR, "JWT",
+ "Failed to update digest with token data");
+ goto cleanup;
+ }
+
+ size_t siglen = 0;
+ if (EVP_DigestSignFinal(mdctx, NULL, &siglen) != 1) {
+ rd_kafka_log(rk, LOG_ERR, "JWT",
+ "Failed to get signature length");
+ goto cleanup;
+ }
+
+ sig = rd_malloc(siglen);
+ if (!sig) {
+ rd_kafka_log(rk, LOG_ERR, "JWT",
+ "Failed to allocate memory for signature");
+ goto cleanup;
+ }
+
+ if (EVP_DigestSignFinal(mdctx, sig, &siglen) != 1) {
+ rd_kafka_log(rk, LOG_ERR, "JWT", "Failed to create signature");
+ goto cleanup;
+ }
+
+ sig_iov.ptr = (char *)sig;
+ sig_iov.size = siglen;
+ encoded_signature = rd_base64_encode_str_urlsafe(&sig_iov);
+
+ if (!encoded_signature)
+ goto cleanup;
+
+ size_t jwt_len = strlen(encoded_header) + strlen(encoded_payload) +
+ strlen(encoded_signature) + 3;
+ result = rd_malloc(jwt_len);
+ if (!result)
+ goto cleanup;
+ rd_snprintf(result, jwt_len, "%s.%s.%s", encoded_header,
+ encoded_payload, encoded_signature);
+
+cleanup:
+ if (encoded_header)
+ rd_free(encoded_header);
+ if (encoded_payload)
+ rd_free(encoded_payload);
+ if (encoded_signature)
+ rd_free(encoded_signature);
+ if (unsigned_token)
+ rd_free(unsigned_token);
+ if (sig)
+ rd_free(sig);
+
+ if (header_json_obj) {
+ if (header_str)
+ free(header_str); /* cJSON_PrintUnformatted uses malloc
+ */
+ cJSON_Delete(header_json_obj);
+ } else if (header_str) {
+ rd_free(header_str); /* rd_malloc was used */
+ }
+
+ if (payload_json_obj) {
+ if (payload_str)
+ free(payload_str); /* cJSON_PrintUnformatted uses malloc
+ */
+ cJSON_Delete(payload_json_obj);
+ } else if (payload_str) {
+ rd_free(payload_str); /* rd_malloc was used */
+ }
+
+ if (pkey)
+ EVP_PKEY_free(pkey);
+ if (mdctx)
+ EVP_MD_CTX_free(mdctx);
+
+ return result;
+}
+
+
+/**
+ * @brief Build request body for JWT bearer token request.
+ *
+ * Creates a URL-encoded request body for token exchange with the JWT assertion
+ * and optional scope.
+ *
+ * @param assertion The JWT assertion to include in the request.
+ * @param scope Optional scope to include in the request (will be URL encoded).
+ *
+ * @returns Newly allocated string with the URL-encoded request body.
+ * Caller must free with rd_free(). NULL on memory allocation failure.
+ *
+ * @locality Any thread.
+ */
+static char *rd_kafka_oidc_jwt_bearer_build_request_body(const char *assertion,
+ const char *scope) {
+ const char *assertion_prefix =
+ "grant_type=urn:ietf:params:oauth:"
+ "grant-type:jwt-bearer"
+ "&assertion=";
+ int assertion_prefix_len = strlen(assertion_prefix) + strlen(assertion);
+ int body_size = assertion_prefix_len + 1;
+ char *scope_escaped = NULL;
+ if (scope) {
+ scope_escaped = curl_easy_escape(NULL, scope, 0);
+ body_size += strlen("&scope=") + strlen(scope_escaped);
+ }
+
+ char *body = rd_malloc(body_size);
+
+ rd_snprintf(body, body_size, "%s%s", assertion_prefix, assertion);
+ if (scope) {
+ rd_snprintf(&body[assertion_prefix_len],
+ body_size - assertion_prefix_len, "&scope=%s",
+ scope_escaped);
+ rd_free(scope_escaped);
+ }
+ return body;
+}
+
+/**
+ * @brief JWT assertion from file
+ *
+ * @param file_path Path to the file containing the JWT assertion.
+ *
+ * @returns Newly allocated string with the JWT assertion.
+ * Caller must free with rd_free(). NULL on error.
+ */
+static char *rd_kafka_oidc_assertion_read_from_file(const char *file_path) {
+ if (!file_path)
+ return NULL;
+ const size_t max_size = 1024 * 1024; /* 1MB limit */
+ return rd_file_read(file_path, NULL, max_size);
+}
+
+/**
+ * @brief Try to validate a token field from the JSON response.
+ * Extracts and validates the token, then decodes its payload to get
+ * subject and expiration.
+ *
+ * @param rk The rd_kafka_t instance
+ * @param json The JSON response from the token endpoint
+ * @param field_name The name of the field to extract (e.g., "access_token" or
+ * "id_token")
+ * @param token_out Pointer to store the extracted token
+ * @param sub_out Pointer to store the subject from the token
+ * @param exp_out Pointer to store the expiration from the token
+ * @param errstr_out Buffer to store error message
+ * @param errstr_size Size of error message buffer
+ *
+ * @returns The extracted token or NULL on failure.
+ */
+static char *rd_kafka_oidc_token_try_validate(cJSON *json,
+ const char *field,
+ char **sub,
+ double *exp,
+ char *errstr,
+ size_t errstr_size) {
+ cJSON *access_token_json, *jwt_exp, *jwt_sub, *payloads = NULL;
+ char *jwt_token = NULL, *decoded_payloads = NULL;
+ const char *decode_errstr = NULL;
+ *sub = NULL;
+
+ access_token_json = cJSON_GetObjectItem(json, field);
+
+ if (!access_token_json) {
+ rd_snprintf(errstr, errstr_size,
+ "Expected JSON response with \"%s\" field", field);
+ goto fail;
+ }
+
+ jwt_token = cJSON_GetStringValue(access_token_json);
+ if (!jwt_token) {
+ rd_snprintf(errstr, errstr_size,
+ "Expected token as a string value");
+ goto fail;
+ }
+
+ decode_errstr =
+ rd_kafka_jwt_b64_decode_payload(jwt_token, &decoded_payloads);
+ if (decode_errstr != NULL) {
+ rd_snprintf(errstr, errstr_size,
+ "Failed to decode JWT payload: %s", decode_errstr);
+ goto fail;
+ }
+
+ payloads = cJSON_Parse(decoded_payloads);
+ if (payloads == NULL) {
+ rd_snprintf(errstr, errstr_size,
+ "Failed to parse JSON JWT payload");
+ goto fail;
+ }
+
+ jwt_exp = cJSON_GetObjectItem(payloads, "exp");
+ if (jwt_exp == NULL) {
+ rd_snprintf(errstr, errstr_size,
+ "Expected JSON JWT response with "
+ "\"exp\" field");
+ goto fail;
+ }
+
+ *exp = cJSON_GetNumberValue(jwt_exp);
+ if (*exp <= 0) {
+ rd_snprintf(errstr, errstr_size,
+ "Expected JSON JWT response with "
+ "valid \"exp\" field");
+ goto fail;
+ }
+
+ jwt_sub = cJSON_GetObjectItem(payloads, "sub");
+ if (jwt_sub == NULL) {
+ rd_snprintf(errstr, errstr_size,
+ "Expected JSON JWT response with "
+ "\"sub\" field");
+ goto fail;
+ }
+
+ *sub = cJSON_GetStringValue(jwt_sub);
+ if (*sub == NULL) {
+ rd_snprintf(errstr, errstr_size,
+ "Expected JSON JWT response with "
+ "valid \"sub\" field");
+ goto fail;
+ }
+ *sub = rd_strdup(*sub);
+done:
+ if (payloads)
+ cJSON_Delete(payloads);
+ if (decoded_payloads)
+ rd_free(decoded_payloads);
+ return jwt_token;
+fail:
+ jwt_token = NULL;
+ goto done;
+}
+
+/**
+ * @brief Implementation of JWT token refresh callback function.
+ * Creates a JWT assertion, exchanges it for an access token,
+ * and sets the token for SASL OAUTHBEARER authentication.
+ *
+ * @param rk The rd_kafka_t instance.
+ * @param oauthbearer_config The OAUTHBEARER configuration.
+ * @param opaque Opaque pointer passed to the callback.
+ *
+ * @locality rdkafka main thread
+ */
+void rd_kafka_oidc_token_jwt_bearer_refresh_cb(rd_kafka_t *rk,
+ const char *oauthbearer_config,
+ void *opaque) {
+ const int timeout_s = 20;
+ const int retry = 4;
+ const int retry_ms = 5 * 1000;
+
+ char *jwt_assertion = NULL;
+ char *request_body = NULL;
+ struct curl_slist *headers = NULL;
+ rd_http_error_t *herr = NULL;
+ cJSON *json = NULL;
+ char *jwt_token = NULL;
+ char set_token_errstr[512];
+ double exp = 0;
+ char **extensions = NULL;
+ char **extension_key_value = NULL;
+ size_t extension_key_value_cnt = 0;
+ size_t extension_cnt;
+ char *sub = NULL;
+ char validate_errstr[512];
+
+ if (rd_kafka_terminating(rk))
+ return;
+
+ if (rk->rk_conf.sasl.oauthbearer.assertion.file) {
+ jwt_assertion = rd_kafka_oidc_assertion_read_from_file(
+ rk->rk_conf.sasl.oauthbearer.assertion.file);
+ } else {
+ jwt_assertion = rd_kafka_oidc_assertion_create(
+ rk, rk->rk_conf.sasl.oauthbearer.assertion.private_key.pem,
+ rk->rk_conf.sasl.oauthbearer.assertion.private_key.file,
+ rk->rk_conf.sasl.oauthbearer.assertion.private_key
+ .passphrase,
+ rk->rk_conf.sasl.oauthbearer.assertion.algorithm,
+ rk->rk_conf.sasl.oauthbearer.assertion.jwt_template_file,
+ rk->rk_conf.sasl.oauthbearer.assertion.claim.subject,
+ rk->rk_conf.sasl.oauthbearer.assertion.claim.issuer,
+ rk->rk_conf.sasl.oauthbearer.assertion.claim.audience,
+ rk->rk_conf.sasl.oauthbearer.assertion.claim.not_before_s,
+ rk->rk_conf.sasl.oauthbearer.assertion.claim.expiration_s,
+ rk->rk_conf.sasl.oauthbearer.assertion.claim.jti_include);
+ }
+
+ if (!jwt_assertion) {
+ rd_kafka_oauthbearer_set_token_failure(
+ rk, "Failed to create JWT assertion");
+ goto done;
+ }
+
+ request_body = rd_kafka_oidc_jwt_bearer_build_request_body(
+ jwt_assertion, rk->rk_conf.sasl.oauthbearer.scope);
+
+ if (!request_body) {
+ rd_kafka_oauthbearer_set_token_failure(
+ rk, "Failed to build JWT request body");
+ goto done;
+ }
+
+ headers = curl_slist_append(
+ headers, "Content-Type: application/x-www-form-urlencoded");
+ headers = curl_slist_append(headers, "Accept: application/json");
+
+ herr = rd_http_post_expect_json(
+ rk, rk->rk_conf.sasl.oauthbearer.token_endpoint_url, headers,
+ request_body, strlen(request_body), timeout_s, retry, retry_ms,
+ &json);
+
+ if (unlikely(herr != NULL)) {
+ rd_kafka_log(
+ rk, LOG_ERR, "JWT",
+ "Failed to retrieve JWT token from \"%s\": %s (%d)",
+ rk->rk_conf.sasl.oauthbearer.token_endpoint_url,
+ herr->errstr, herr->code);
+ rd_kafka_oauthbearer_set_token_failure(rk, herr->errstr);
+ rd_http_error_destroy(herr);
+ goto done;
+ }
+
+ /*
+ * RFC 7523 Section 1 says that an access token should be returned
+ * https://datatracker.ietf.org/doc/html/rfc7523#section-1
+ * Some providers (e.g. GCP) return an `id_token` instead, depending
+ * on the configured `target_audience` in the request JWT bearer token.
+ * This may be because the validation endpoint is not accessible
+ * for validating the `access_token` while the `id_token` is validated
+ * through the JWKS URL.
+ * This function will try to validate the `access_token` and then the
+ * `id_token`.
+ */
+ jwt_token = rd_kafka_oidc_token_try_validate(json, "access_token", &sub,
+ &exp, validate_errstr,
+ sizeof(validate_errstr));
+ if (!jwt_token)
+ jwt_token = rd_kafka_oidc_token_try_validate(
+ json, "id_token", &sub, &exp, validate_errstr,
+ sizeof(validate_errstr));
+
+ if (!jwt_token) {
+ rd_kafka_oauthbearer_set_token_failure(rk, validate_errstr);
+ goto done;
+ }
+
+
+ if (rk->rk_conf.sasl.oauthbearer.extensions_str) {
+ extensions =
+ rd_string_split(rk->rk_conf.sasl.oauthbearer.extensions_str,
+ ',', rd_true, &extension_cnt);
+
+ extension_key_value = rd_kafka_conf_kv_split(
+ (const char **)extensions, extension_cnt,
+ &extension_key_value_cnt);
+ }
+
+ if (rd_kafka_oauthbearer_set_token(
+ rk, jwt_token, (int64_t)exp * 1000, sub,
+ (const char **)extension_key_value, extension_key_value_cnt,
+ set_token_errstr,
+ sizeof(set_token_errstr)) != RD_KAFKA_RESP_ERR_NO_ERROR) {
+ rd_kafka_oauthbearer_set_token_failure(rk, validate_errstr);
+ }
+
+done:
+ RD_IF_FREE(sub, rd_free);
+ RD_IF_FREE(jwt_assertion, rd_free);
+ RD_IF_FREE(request_body, rd_free);
+ RD_IF_FREE(headers, curl_slist_free_all);
+ RD_IF_FREE(json, cJSON_Delete);
+ RD_IF_FREE(extensions, rd_free);
+ RD_IF_FREE(extension_key_value, rd_free);
+ /* jwt_token is freed as part of the json object */
+}
/**
* @brief Implementation of Oauth/OIDC token refresh callback function,
@@ -224,37 +906,33 @@ static void rd_kafka_oidc_build_post_fields(const char *scope,
* then extract the jwt from the JSON response, and forward it to
* the broker.
*/
-void rd_kafka_oidc_token_refresh_cb(rd_kafka_t *rk,
- const char *oauthbearer_config,
- void *opaque) {
+void rd_kafka_oidc_token_client_credentials_refresh_cb(
+ rd_kafka_t *rk,
+ const char *oauthbearer_config,
+ void *opaque) {
const int timeout_s = 20;
const int retry = 4;
const int retry_ms = 5 * 1000;
double exp;
- cJSON *json = NULL;
- cJSON *payloads = NULL;
- cJSON *parsed_token, *jwt_exp, *jwt_sub;
+ cJSON *json = NULL;
rd_http_error_t *herr;
char *jwt_token;
- char *post_fields;
- char *decoded_payloads = NULL;
+ char *post_fields = NULL;
struct curl_slist *headers = NULL;
const char *token_url;
- const char *sub;
- const char *errstr;
+ char *sub = NULL;
size_t post_fields_size;
size_t extension_cnt;
size_t extension_key_value_cnt = 0;
char set_token_errstr[512];
- char decode_payload_errstr[512];
char **extensions = NULL;
char **extension_key_value = NULL;
@@ -262,13 +940,14 @@ void rd_kafka_oidc_token_refresh_cb(rd_kafka_t *rk,
if (rd_kafka_terminating(rk))
return;
- rd_kafka_oidc_build_headers(rk->rk_conf.sasl.oauthbearer.client_id,
- rk->rk_conf.sasl.oauthbearer.client_secret,
- &headers);
+ rd_kafka_oidc_client_credentials_build_headers(
+ rk->rk_conf.sasl.oauthbearer.client_id,
+ rk->rk_conf.sasl.oauthbearer.client_secret, &headers);
/* Build post fields */
- rd_kafka_oidc_build_post_fields(rk->rk_conf.sasl.oauthbearer.scope,
- &post_fields, &post_fields_size);
+ rd_kafka_oidc_client_credentials_build_post_fields(
+ rk->rk_conf.sasl.oauthbearer.scope, &post_fields,
+ &post_fields_size);
token_url = rk->rk_conf.sasl.oauthbearer.token_endpoint_url;
@@ -286,75 +965,129 @@ void rd_kafka_oidc_token_refresh_cb(rd_kafka_t *rk,
goto done;
}
- parsed_token = cJSON_GetObjectItem(json, "access_token");
-
- if (parsed_token == NULL) {
- rd_kafka_oauthbearer_set_token_failure(
- rk,
- "Expected JSON JWT response with "
- "\"access_token\" field");
+ jwt_token = rd_kafka_oidc_token_try_validate(json, "access_token", &sub,
+ &exp, set_token_errstr,
+ sizeof(set_token_errstr));
+ if (!jwt_token) {
+ rd_kafka_oauthbearer_set_token_failure(rk, set_token_errstr);
goto done;
}
- jwt_token = cJSON_GetStringValue(parsed_token);
- if (jwt_token == NULL) {
- rd_kafka_oauthbearer_set_token_failure(
- rk,
- "Expected JSON "
- "response as a value string");
- goto done;
- }
+ if (rk->rk_conf.sasl.oauthbearer.extensions_str) {
+ extensions =
+ rd_string_split(rk->rk_conf.sasl.oauthbearer.extensions_str,
+ ',', rd_true, &extension_cnt);
- errstr = rd_kafka_jwt_b64_decode_payload(jwt_token, &decoded_payloads);
- if (errstr != NULL) {
- rd_snprintf(decode_payload_errstr,
- sizeof(decode_payload_errstr),
- "Failed to decode JWT payload: %s", errstr);
- rd_kafka_oauthbearer_set_token_failure(rk,
- decode_payload_errstr);
- goto done;
+ extension_key_value = rd_kafka_conf_kv_split(
+ (const char **)extensions, extension_cnt,
+ &extension_key_value_cnt);
}
- payloads = cJSON_Parse(decoded_payloads);
- if (payloads == NULL) {
- rd_kafka_oauthbearer_set_token_failure(
- rk, "Failed to parse JSON JWT payload");
- goto done;
- }
+ if (rd_kafka_oauthbearer_set_token(
+ rk, jwt_token, (int64_t)exp * 1000, sub,
+ (const char **)extension_key_value, extension_key_value_cnt,
+ set_token_errstr,
+ sizeof(set_token_errstr)) != RD_KAFKA_RESP_ERR_NO_ERROR)
+ rd_kafka_oauthbearer_set_token_failure(rk, set_token_errstr);
- jwt_exp = cJSON_GetObjectItem(payloads, "exp");
- if (jwt_exp == NULL) {
- rd_kafka_oauthbearer_set_token_failure(
- rk,
- "Expected JSON JWT response with "
- "\"exp\" field");
- goto done;
- }
+done:
+ RD_IF_FREE(sub, rd_free);
+ RD_IF_FREE(post_fields, rd_free);
+ RD_IF_FREE(json, cJSON_Delete);
+ RD_IF_FREE(headers, curl_slist_free_all);
+ RD_IF_FREE(extensions, rd_free);
+ RD_IF_FREE(extension_key_value, rd_free);
+}
- exp = cJSON_GetNumberValue(jwt_exp);
- if (exp <= 0) {
- rd_kafka_oauthbearer_set_token_failure(
- rk,
- "Expected JSON JWT response with "
- "valid \"exp\" field");
- goto done;
+/**
+ * @brief Implementation of Oauth/OIDC token refresh callback function
+ * for Azure IMDS,
+ * will receive the JSON response after HTTP(S) GET call to token
+ * provider, then extract the jwt from the JSON response, and
+ * forward it to the broker.
+ */
+void rd_kafka_oidc_token_metadata_azure_imds_refresh_cb(
+ rd_kafka_t *rk,
+ const char *oauthbearer_config,
+ void *opaque) {
+ const int timeout_s = 20;
+ const int retries = 4;
+ const int retry_ms = 5 * 1000;
+
+ double exp;
+
+ cJSON *json = NULL;
+
+ rd_http_error_t *herr;
+
+ char *jwt_token;
+
+ struct curl_slist *headers = NULL;
+
+ const char *token_endpoint_url_initial = NULL;
+ char *token_endpoint_url = NULL;
+ char *sub = NULL;
+
+ size_t extension_cnt;
+ size_t extension_key_value_cnt = 0;
+
+ char set_token_errstr[512];
+
+ char **extensions = NULL;
+ char **extension_key_value = NULL;
+ char *query = NULL;
+ static char *headers_array[] = {"Metadata: true"};
+
+ if (rd_kafka_terminating(rk))
+ return;
+
+ if (rk->rk_conf.sasl.oauthbearer_config)
+ query = rd_kafka_conf_kv_get(
+ rk->rk_conf.sasl.oauthbearer_config, "query", ',');
+ token_endpoint_url_initial =
+ rk->rk_conf.sasl.oauthbearer.token_endpoint_url;
+ if (!token_endpoint_url_initial)
+ token_endpoint_url_initial =
+ RD_KAFKA_SASL_OAUTHBEARER_METADATA_AUTHENTICATION_URL_AZURE_IMDS;
+ if (query && *query) {
+ token_endpoint_url = rd_http_get_params_append(
+ token_endpoint_url_initial, query);
+
+ if (token_endpoint_url == NULL) {
+ rd_snprintf(
+ set_token_errstr, sizeof(set_token_errstr),
+ "Failed to append params \"%s\" to token endpoint "
+ "URL \"%s\"",
+ query,
+ rk->rk_conf.sasl.oauthbearer.token_endpoint_url);
+ rd_kafka_log(rk, LOG_ERR, "OIDC", "%s",
+ set_token_errstr);
+ rd_kafka_oauthbearer_set_token_failure(
+ rk, set_token_errstr);
+ goto done;
+ }
+ } else {
+ token_endpoint_url = rd_strdup(token_endpoint_url_initial);
}
- jwt_sub = cJSON_GetObjectItem(payloads, "sub");
- if (jwt_sub == NULL) {
- rd_kafka_oauthbearer_set_token_failure(
- rk,
- "Expected JSON JWT response with "
- "\"sub\" field");
+ herr = rd_http_get_json(rk, token_endpoint_url, headers_array, 1,
+ timeout_s, retries, retry_ms, &json);
+
+ if (unlikely(herr != NULL)) {
+ rd_kafka_log(rk, LOG_ERR, "OIDC",
+ "Failed to retrieve OIDC "
+ "token from \"%s\": %s (%d)",
+ token_endpoint_url, herr->errstr, herr->code);
+ rd_kafka_oauthbearer_set_token_failure(rk, herr->errstr);
+ rd_http_error_destroy(herr);
goto done;
}
- sub = cJSON_GetStringValue(jwt_sub);
- if (sub == NULL) {
- rd_kafka_oauthbearer_set_token_failure(
- rk,
- "Expected JSON JWT response with "
- "valid \"sub\" field");
+ jwt_token = rd_kafka_oidc_token_try_validate(json, "access_token", &sub,
+ &exp, set_token_errstr,
+ sizeof(set_token_errstr));
+ if (!jwt_token) {
+ rd_kafka_oauthbearer_set_token_failure(rk, set_token_errstr);
goto done;
}
@@ -376,16 +1109,15 @@ void rd_kafka_oidc_token_refresh_cb(rd_kafka_t *rk,
rd_kafka_oauthbearer_set_token_failure(rk, set_token_errstr);
done:
- RD_IF_FREE(decoded_payloads, rd_free);
- RD_IF_FREE(post_fields, rd_free);
+ RD_IF_FREE(sub, rd_free);
RD_IF_FREE(json, cJSON_Delete);
RD_IF_FREE(headers, curl_slist_free_all);
RD_IF_FREE(extensions, rd_free);
RD_IF_FREE(extension_key_value, rd_free);
- RD_IF_FREE(payloads, cJSON_Delete);
+ RD_IF_FREE(token_endpoint_url, rd_free);
+ RD_IF_FREE(query, rd_free);
}
-
/**
* @brief Make sure the jwt is able to be extracted from HTTP(S) response.
* The JSON response after HTTP(S) call to token provider will be in
@@ -421,10 +1153,11 @@ static int ut_sasl_oauthbearer_oidc_should_succeed(void) {
cJSON *json = NULL;
char *token;
cJSON *parsed_token;
+ rd_kafka_t *rk = rd_calloc(1, sizeof(*rk));
RD_UT_BEGIN();
- herr = rd_http_req_init(&hreq, "");
+ herr = rd_http_req_init(rk, &hreq, "");
RD_UT_ASSERT(!herr,
"Expected initialize to succeed, "
@@ -460,6 +1193,7 @@ static int ut_sasl_oauthbearer_oidc_should_succeed(void) {
rd_http_error_destroy(herr);
rd_http_req_destroy(&hreq);
cJSON_Delete(json);
+ rd_free(rk);
RD_UT_PASS();
}
@@ -476,10 +1210,11 @@ static int ut_sasl_oauthbearer_oidc_with_empty_key(void) {
rd_http_error_t *herr;
cJSON *json = NULL;
cJSON *parsed_token;
+ rd_kafka_t *rk = rd_calloc(1, sizeof(*rk));
RD_UT_BEGIN();
- herr = rd_http_req_init(&hreq, "");
+ herr = rd_http_req_init(rk, &hreq, "");
RD_UT_ASSERT(!herr,
"Expected initialization to succeed, "
"but it failed with error code: %d, error string: %s",
@@ -507,6 +1242,7 @@ static int ut_sasl_oauthbearer_oidc_with_empty_key(void) {
rd_http_error_destroy(herr);
cJSON_Delete(json);
cJSON_Delete(parsed_token);
+ rd_free(rk);
RD_UT_PASS();
}
@@ -526,7 +1262,8 @@ static int ut_sasl_oauthbearer_oidc_post_fields(void) {
RD_UT_BEGIN();
- rd_kafka_oidc_build_post_fields(scope, &post_fields, &post_fields_size);
+ rd_kafka_oidc_client_credentials_build_post_fields(scope, &post_fields,
+ &post_fields_size);
RD_UT_ASSERT(expected_post_fields_size == post_fields_size,
"Expected expected_post_fields_size is %" PRIusz
@@ -558,7 +1295,8 @@ static int ut_sasl_oauthbearer_oidc_post_fields_with_empty_scope(void) {
RD_UT_BEGIN();
- rd_kafka_oidc_build_post_fields(scope, &post_fields, &post_fields_size);
+ rd_kafka_oidc_client_credentials_build_post_fields(scope, &post_fields,
+ &post_fields_size);
RD_UT_ASSERT(expected_post_fields_size == post_fields_size,
"Expected expected_post_fields_size is %" PRIusz
@@ -587,3 +1325,342 @@ int unittest_sasl_oauthbearer_oidc(void) {
fails += ut_sasl_oauthbearer_oidc_post_fields_with_empty_scope();
return fails;
}
+
+/**
+ * @brief Test the Base64Url encoding functionality.
+ * Verifies that the encoding correctly handles special characters
+ * and padding removal.
+ */
+static int ut_sasl_oauthbearer_oidc_jwt_bearer_base64url_encode(void) {
+ /* Test cases with expected inputs and outputs */
+ static const struct {
+ const char *input;
+ const char *expected_output;
+ } test_cases[] = {
+ /* Regular case */
+ {"Hello, world!", "SGVsbG8sIHdvcmxkIQ"},
+ /* Case with padding characters that should be removed */
+ {"test", "dGVzdA"},
+ /* Empty string */
+ {"", ""},
+ /* Special characters that trigger Base64 padding */
+ {"f", "Zg"},
+ {"fo", "Zm8"},
+ {"foo", "Zm9v"},
+ {"foob", "Zm9vYg"},
+ {"fooba", "Zm9vYmE"},
+ {"foobar", "Zm9vYmFy"},
+ /* Characters that produce + and / in standard Base64 */
+ {"\x3E\x3F",
+ "Pj8"}, /* encodes to ">?" in standard Base64 with + and / */
+ };
+ unsigned int i;
+
+ RD_UT_BEGIN();
+
+ for (i = 0; i < RD_ARRAYSIZE(test_cases); i++) {
+ rd_chariov_t input_iov;
+ input_iov.ptr = (char *)test_cases[i].input;
+ input_iov.size = strlen(test_cases[i].input);
+ char *output = rd_base64_encode_str_urlsafe(&input_iov);
+
+ RD_UT_ASSERT(output != NULL,
+ "Expected non-NULL output for input: %s",
+ test_cases[i].input);
+
+ RD_UT_ASSERT(!strcmp(output, test_cases[i].expected_output),
+ "Base64Url encoding failed: expected %s, got %s",
+ test_cases[i].expected_output, output);
+
+ rd_free(output);
+ }
+
+ RD_UT_PASS();
+}
+
+/**
+ * @brief Test JWT request body building.
+ * Verifies that the request body is correctly formatted with
+ * the required parameters.
+ */
+static int ut_sasl_oauthbearer_oidc_jwt_bearer_build_request_body(void) {
+ const char *assertion = "test.jwt.assertion";
+ const char *scope = "test.scope";
+ const char *expected =
+ "grant_type=urn:ietf:params:oauth:grant-type:jwt-bearer&assertion="
+ "test.jwt.assertion&scope=test.scope";
+ char *body;
+
+ RD_UT_BEGIN();
+
+ body = rd_kafka_oidc_jwt_bearer_build_request_body(assertion, scope);
+
+ RD_UT_ASSERT(body != NULL, "Expected non-NULL request body");
+
+ RD_UT_ASSERT(!strcmp(body, expected),
+ "Request body incorrect: expected '%s', got '%s'",
+ expected, body);
+
+ rd_free(body);
+
+ RD_UT_PASS();
+}
+
+/**
+ * @brief Test JWT assertion file parsing.
+ * Verifies that the function correctly reads a JWT from a file.
+ */
+static int ut_sasl_oauthbearer_oidc_assertion_parse_from_file(void) {
+
+ char tempfile_path[512];
+ FILE *tempfile;
+ const char *test_jwt = "header.payload.signature";
+ char *result;
+
+ RD_UT_BEGIN();
+
+ tempfile = rd_file_mkstemp("rdtmp", "wb", tempfile_path,
+ sizeof(tempfile_path));
+ fprintf(tempfile, "%s", test_jwt);
+ fclose(tempfile);
+
+ /* Test parsing from file */
+ result = rd_kafka_oidc_assertion_read_from_file(tempfile_path);
+ RD_UT_ASSERT(result != NULL,
+ "Expected non-NULL result from parsing file");
+ RD_UT_ASSERT(!strcmp(result, test_jwt),
+ "Incorrect JWT parsed: expected '%s', got '%s'", test_jwt,
+ result);
+
+ rd_free(result);
+
+ /* Test with NULL path */
+ result = rd_kafka_oidc_assertion_read_from_file(NULL);
+ RD_UT_ASSERT(result == NULL, "Expected NULL result with NULL path");
+
+ /* Test with non-existent file */
+ result =
+ rd_kafka_oidc_assertion_read_from_file("/non/existent/file/path");
+ RD_UT_ASSERT(result == NULL,
+ "Expected NULL result with non-existent file");
+
+ remove(tempfile_path);
+
+ RD_UT_PASS();
+}
+
+/**
+ * @brief Mock function for testing JWT template processing.
+ * Creates a file with valid JWT template JSON.
+ */
+static char *ut_create_mock_jwt_template_file(void) {
+ FILE *tempfile;
+ char tempfile_path[512];
+
+ const char *template_json =
+ "{\n"
+ " \"header\": {\n"
+ " \"kid\": \"test-key-id\"\n"
+ " },\n"
+ " \"payload\": {\n"
+ " \"sub\": \"test-subject\",\n"
+ " \"aud\": \"test-audience\"\n"
+ " }\n"
+ "}";
+
+ tempfile = rd_file_mkstemp("rdtmp", "wb", tempfile_path,
+ sizeof(tempfile_path));
+ if (!tempfile)
+ return NULL;
+
+ fprintf(tempfile, "%s", template_json);
+ fclose(tempfile);
+
+ return rd_strdup(tempfile_path);
+}
+
+/**
+ * @brief Test JWT template file processing.
+ * Verifies that the function correctly parses header and payload from
+ * template.
+ */
+static int ut_sasl_oauthbearer_oidc_assertion_process_template_file(void) {
+ char *template_path;
+ rd_kafka_t *rk;
+ cJSON *header = NULL, *payload = NULL;
+ int result;
+
+ RD_UT_BEGIN();
+
+ rk = rd_calloc(1, sizeof(*rk));
+
+ template_path = ut_create_mock_jwt_template_file();
+ RD_UT_ASSERT(template_path != NULL, "Failed to create template file");
+
+ /* Test template processing */
+ result = rd_kafka_oidc_assertion_parse_template_file(rk, template_path,
+ &header, &payload);
+ RD_UT_ASSERT(result == 0, "Expected success from template processing");
+ RD_UT_ASSERT(header != NULL, "Expected non-NULL header JSON");
+ RD_UT_ASSERT(payload != NULL, "Expected non-NULL payload JSON");
+
+ /* Verify header contents */
+ cJSON *kid = cJSON_GetObjectItem(header, "kid");
+ RD_UT_ASSERT(kid != NULL, "Expected kid in header");
+ RD_UT_ASSERT(cJSON_IsString(kid), "Expected kid to be string");
+ RD_UT_ASSERT(!strcmp(cJSON_GetStringValue(kid), "test-key-id"),
+ "Incorrect kid value");
+
+ /* Verify payload contents */
+ cJSON *sub = cJSON_GetObjectItem(payload, "sub");
+ RD_UT_ASSERT(sub != NULL, "Expected sub in payload");
+ RD_UT_ASSERT(cJSON_IsString(sub), "Expected sub to be string");
+ RD_UT_ASSERT(!strcmp(cJSON_GetStringValue(sub), "test-subject"),
+ "Incorrect sub value");
+
+ cJSON *aud = cJSON_GetObjectItem(payload, "aud");
+ RD_UT_ASSERT(aud != NULL, "Expected aud in payload");
+ RD_UT_ASSERT(cJSON_IsString(aud), "Expected aud to be string");
+ RD_UT_ASSERT(!strcmp(cJSON_GetStringValue(aud), "test-audience"),
+ "Incorrect aud value");
+
+ /* Test with non-existent file */
+ cJSON_Delete(header);
+ cJSON_Delete(payload);
+ header = NULL;
+ payload = NULL;
+
+ result = rd_kafka_oidc_assertion_parse_template_file(
+ rk, "/non/existent/file", &header, &payload);
+ RD_UT_ASSERT(result == -1, "Expected failure with non-existent file");
+ RD_UT_ASSERT(header == NULL,
+ "Expected NULL header with failed processing");
+ RD_UT_ASSERT(payload == NULL,
+ "Expected NULL payload with failed processing");
+
+ unlink(template_path);
+ rd_free(template_path);
+ rd_free(rk);
+ if (header)
+ cJSON_Delete(header);
+ if (payload)
+ cJSON_Delete(payload);
+
+ RD_UT_PASS();
+}
+
+/**
+ * @brief Test JWT assertion creation with minimal approach.
+ * Creates a simplified test that validates the format of the created
+ * JWT.
+ */
+static int ut_sasl_oauthbearer_oidc_assertion_create(void) {
+ rd_kafka_t *rk;
+ char *private_key_pem;
+ char *jwt;
+ char *header_part, *payload_part, *signature_part;
+ char *dot1, *dot2;
+
+ RD_UT_BEGIN();
+
+ rk = rd_calloc(1, sizeof(*rk));
+
+ /* Random key for signing */
+ private_key_pem =
+ "-----BEGIN PRIVATE KEY-----\n"
+ "MIIEvAIBADANBgkqhkiG9w0BAQEFAASCBKYwggSiAgEAAoIBAQCuBS7qG5Cd2voa\n"
+ "7nSU2xaDbe6QOYU2P4bIY58SKHbFyq1iB517r61ImsWD+UfZuVxCqXRaWdxxnG/D\n"
+ "5VGTQzBOZYlgSYxdJ1KvITXO8kj5i2zBT/LI9R9MTQ7nLFh+vQm1aM8Ts1PmA5t9\n"
+ "zFtR9B8RfqN9kbt+2LnLY57aJxEkFC3D89D0WWT97UJWKo7/vxMqp9K9uAIL2Efo\n"
+ "5rp9qwyPbx9LmTbfZ8Vog6mG6tAQQHSUqw0PnfhADCVCkYtkzYcyDZy3qZQFu1bY\n"
+ "KuuMoMjssyCUL5tTHyNZju0p3Z0bSfOV/nkqHpSSjHKCeQkSKS18/7In6cfY/M4k\n"
+ "8rM4HWkdAgMBAAECggEAFsTo2YrXxj/Dn8h5ioyMCpBUuZw9GNcBDLE0PAz9VW3q\n"
+ "d7wlV+ypkKlnlJgGVa+SKcrARZ4iYN8mJIyZutn8tRVF/0pASmP9xppizvwWnkgm\n"
+ "57hNPQwNl08x1v+PaK3VWl4nUh2RqbPpIXGetT9q3UAjpiduT++Nh9Y2D7cy3/Ro\n"
+ "ritnpBDs1R6y5J3rxiE1s8kLYwhDRCPsgUg/ZtKPDTTFz42ArrFeqM91FmjHYP3t\n"
+ "p9Uh6CIZ80D6CsMX/TnZFfhKe6EvKBSl4W6tcdFlnXW52fm/670iKSmcJ09+fzPO\n"
+ "T1BLrkXGv51bFnlvUyJqQGVEv5+0+HUX/oTpTknMQQKBgQDbYhqip5e8r1f5v32B\n"
+ "k1r3xtEiWU2mZoTHJu6bVeuigzVhz4pTMVZChElJ4QnhwwO0t5Oe4Su1MZtjMRw7\n"
+ "qIE+YM2pXss25LRXbmWItuRWINzpe8omlxQSOj2tNO/67l0P4vmmrT5wkU2cG6TR\n"
+ "ddzorO3NDA4MY4+Xdli+SHXwUQKBgQDLEMqlwyvaGjuZ30l6F13fWnEt9PNCtJsa\n"
+ "nsdKJKyFMThdysY/PK40o2GTRRhgYa2jigN3OCYSSznRRZRlqznqL1bOLlYV6zS8\n"
+ "TGhdLXuApyLAjZYIK4RtZJYGR9+yg8rH13uNektgW8KnHh5Ko/ptRVoEukf3SBsh\n"
+ "f0Fib3ylDQKBgE11Bth0+bMJ6bLpNEPiphSjosVQ6ISe37R8/3Pi0y5uyxM8tqcG\n"
+ "3WDg2gt2pAmM1CsjQcCv2cHAwQ81kLVTmkZO4W4yZOd9ulrARKMPh/EM61KYfVhA\n"
+ "sTp6S7py3WQocr0gM2rw8gHGm7NJY1j9F0EjhVaHMhKXuGQOyehtJw7xAoGAPwuA\n"
+ "jwRQSg+Y74XmbxRwHZcbynPhTpV6DkK7huZp9ZQ5ds0szZdOUqNi+PEbx1isKzj/\n"
+ "KHVzRHy8f5+FmicV/QIjhjHWokl6/vcN89faHzBE1tleejzgiYIQHfUUm3zVaUQa\n"
+ "ZOtSGaGDhpUQPIY6itBcSVl4XGqzmavDpgcNAMUCgYBFFGtG+RbSySzKfRUp3vc5\n"
+ "8YqIdrtXfW9gc9s1+Pw8wfgrY0Rrvy+e3ClSwgGENxgxBvWvhzq2m0S8x2jdLAl1\n"
+ "b+VLGCOpUvS4iN2yrHkoHS7BSW40wLuVooJUAaNOIEPqiv1JC75q2dhTRrANp6WB\n"
+ "bm+7yWVTNlXYuKQqtuOkNQ==\n"
+ "-----END PRIVATE KEY-----\n";
+
+ jwt = rd_kafka_oidc_assertion_create(
+ rk, private_key_pem, NULL, NULL,
+ RD_KAFKA_SASL_OAUTHBEARER_ASSERTION_ALGORITHM_RS256, NULL,
+ "test-subject", "test-issuer", "test-audience", 2, 300, rd_true);
+
+ RD_UT_ASSERT(jwt != NULL, "Failed to create JWT assertion");
+
+ dot1 = strchr(jwt, '.');
+ RD_UT_ASSERT(dot1 != NULL, "JWT missing first dot separator");
+
+ dot2 = strchr(dot1 + 1, '.');
+ RD_UT_ASSERT(dot2 != NULL, "JWT missing second dot separator");
+
+ header_part = rd_strndup(jwt, dot1 - jwt);
+ payload_part = rd_strndup(dot1 + 1, dot2 - (dot1 + 1));
+ signature_part = rd_strdup(dot2 + 1);
+
+ RD_UT_ASSERT(strlen(header_part) > 0, "JWT header part is empty");
+ RD_UT_ASSERT(strlen(payload_part) > 0, "JWT payload part is empty");
+ RD_UT_ASSERT(strlen(signature_part) > 0, "JWT signature part is empty");
+
+ RD_UT_ASSERT(!strchr(header_part, '='),
+ "JWT header contains padding character");
+ RD_UT_ASSERT(!strchr(payload_part, '='),
+ "JWT payload contains padding character");
+ RD_UT_ASSERT(!strchr(signature_part, '='),
+ "JWT signature contains padding character");
+
+ RD_UT_ASSERT(!strchr(header_part, '+'),
+ "JWT header contains '+' character");
+ RD_UT_ASSERT(!strchr(header_part, '/'),
+ "JWT header contains '/' character");
+ RD_UT_ASSERT(!strchr(payload_part, '+'),
+ "JWT payload contains '+' character");
+ RD_UT_ASSERT(!strchr(payload_part, '/'),
+ "JWT payload contains '/' character");
+ RD_UT_ASSERT(!strchr(signature_part, '+'),
+ "JWT signature contains '+' character");
+ RD_UT_ASSERT(!strchr(signature_part, '/'),
+ "JWT signature contains '/' character");
+
+ rd_free(header_part);
+ rd_free(payload_part);
+ rd_free(signature_part);
+ rd_free(jwt);
+ rd_free(rk);
+
+ RD_UT_PASS();
+}
+
+int unittest_sasl_oauthbearer_oidc_jwt_bearer(void) {
+ int fails = 0;
+
+ fails += ut_sasl_oauthbearer_oidc_jwt_bearer_base64url_encode();
+ fails += ut_sasl_oauthbearer_oidc_jwt_bearer_build_request_body();
+
+ return fails;
+}
+
+int unittest_sasl_oauthbearer_oidc_assertion(void) {
+ int fails = 0;
+
+ fails += ut_sasl_oauthbearer_oidc_assertion_parse_from_file();
+ fails += ut_sasl_oauthbearer_oidc_assertion_process_template_file();
+ fails += ut_sasl_oauthbearer_oidc_assertion_create();
+
+ return fails;
+}
diff --git a/src/rdkafka_sasl_oauthbearer_oidc.h b/src/rdkafka_sasl_oauthbearer_oidc.h
index f46bf1beb7..1bfef07b55 100644
--- a/src/rdkafka_sasl_oauthbearer_oidc.h
+++ b/src/rdkafka_sasl_oauthbearer_oidc.h
@@ -28,9 +28,19 @@
#ifndef _RDKAFKA_SASL_OAUTHBEARER_OIDC_H_
#define _RDKAFKA_SASL_OAUTHBEARER_OIDC_H_
-void rd_kafka_oidc_token_refresh_cb(rd_kafka_t *rk,
- const char *oauthbearer_config,
- void *opaque);
+void rd_kafka_oidc_token_jwt_bearer_refresh_cb(rd_kafka_t *rk,
+ const char *oauthbearer_config,
+ void *opaque);
+
+void rd_kafka_oidc_token_client_credentials_refresh_cb(
+ rd_kafka_t *rk,
+ const char *oauthbearer_config,
+ void *opaque);
+
+void rd_kafka_oidc_token_metadata_azure_imds_refresh_cb(
+ rd_kafka_t *rk,
+ const char *oauthbearer_config,
+ void *opaque);
int unittest_sasl_oauthbearer_oidc(void);
diff --git a/src/rdkafka_sasl_plain.c b/src/rdkafka_sasl_plain.c
index cca9957c83..3c817c64e3 100644
--- a/src/rdkafka_sasl_plain.c
+++ b/src/rdkafka_sasl_plain.c
@@ -81,9 +81,9 @@ int rd_kafka_sasl_plain_client_new(rd_kafka_transport_t *rktrans,
cidlen = rk->rk_conf.sasl.username
? (int)strlen(rk->rk_conf.sasl.username)
: 0;
- pwlen = rk->rk_conf.sasl.password
- ? (int)strlen(rk->rk_conf.sasl.password)
- : 0;
+ pwlen = rk->rk_conf.sasl.password
+ ? (int)strlen(rk->rk_conf.sasl.password)
+ : 0;
buf = rd_alloca(zidlen + 1 + cidlen + 1 + pwlen + 1);
diff --git a/src/rdkafka_sasl_scram.c b/src/rdkafka_sasl_scram.c
index 01a6cd75e4..689b91284e 100644
--- a/src/rdkafka_sasl_scram.c
+++ b/src/rdkafka_sasl_scram.c
@@ -54,9 +54,10 @@
* @brief Per-connection state
*/
struct rd_kafka_sasl_scram_state {
- enum { RD_KAFKA_SASL_SCRAM_STATE_CLIENT_FIRST_MESSAGE,
- RD_KAFKA_SASL_SCRAM_STATE_SERVER_FIRST_MESSAGE,
- RD_KAFKA_SASL_SCRAM_STATE_CLIENT_FINAL_MESSAGE,
+ enum {
+ RD_KAFKA_SASL_SCRAM_STATE_CLIENT_FIRST_MESSAGE,
+ RD_KAFKA_SASL_SCRAM_STATE_SERVER_FIRST_MESSAGE,
+ RD_KAFKA_SASL_SCRAM_STATE_CLIENT_FINAL_MESSAGE,
} state;
rd_chariov_t cnonce; /* client c-nonce */
rd_chariov_t first_msg_bare; /* client-first-message-bare */
@@ -253,10 +254,9 @@ static char *rd_kafka_sasl_safe_string(const char *str) {
* @brief Build client-final-message-without-proof
* @remark out->ptr will be allocated and must be freed.
*/
-static void rd_kafka_sasl_scram_build_client_final_message_wo_proof(
- struct rd_kafka_sasl_scram_state *state,
- const char *snonce,
- rd_chariov_t *out) {
+static void
+rd_kafka_sasl_scram_build_client_final_message_wo_proof(const char *snonce,
+ rd_chariov_t *out) {
const char *attr_c = "biws"; /* base64 encode of "n,," */
/*
@@ -264,11 +264,9 @@ static void rd_kafka_sasl_scram_build_client_final_message_wo_proof(
* channel-binding "," nonce [","
* extensions]
*/
- out->size = strlen("c=,r=") + strlen(attr_c) + state->cnonce.size +
- strlen(snonce);
- out->ptr = rd_malloc(out->size + 1);
- rd_snprintf(out->ptr, out->size + 1, "c=%s,r=%.*s%s", attr_c,
- (int)state->cnonce.size, state->cnonce.ptr, snonce);
+ out->size = strlen("c=,r=") + strlen(attr_c) + strlen(snonce);
+ out->ptr = rd_malloc(out->size + 1);
+ rd_snprintf(out->ptr, out->size + 1, "c=%s,r=%s", attr_c, snonce);
}
@@ -338,7 +336,7 @@ static int rd_kafka_sasl_scram_build_client_final_message(
/* client-final-message-without-proof */
rd_kafka_sasl_scram_build_client_final_message_wo_proof(
- state, server_nonce, &client_final_msg_wo_proof);
+ server_nonce, &client_final_msg_wo_proof);
/* AuthMessage := client-first-message-bare + "," +
* server-first-message + "," +
diff --git a/src/rdkafka_ssl.c b/src/rdkafka_ssl.c
index ee5154d4b1..2508ad4021 100644
--- a/src/rdkafka_ssl.c
+++ b/src/rdkafka_ssl.c
@@ -135,11 +135,14 @@ const char *rd_kafka_ssl_last_error_str(void) {
*
* If 'rkb' is non-NULL broker-specific logging will be used,
* else it will fall back on global 'rk' debugging.
+ *
+ * `ctx_identifier` is a string used to customize the log message.
*/
-static char *rd_kafka_ssl_error(rd_kafka_t *rk,
- rd_kafka_broker_t *rkb,
- char *errstr,
- size_t errstr_size) {
+char *rd_kafka_ssl_error0(rd_kafka_t *rk,
+ rd_kafka_broker_t *rkb,
+ const char *ctx_identifier,
+ char *errstr,
+ size_t errstr_size) {
unsigned long l;
const char *file, *data, *func;
int line, flags;
@@ -166,9 +169,11 @@ static char *rd_kafka_ssl_error(rd_kafka_t *rk,
if (cnt++ > 0) {
/* Log last message */
if (rkb)
- rd_rkb_log(rkb, LOG_ERR, "SSL", "%s", errstr);
+ rd_rkb_log(rkb, LOG_ERR, "SSL", "%s: %s",
+ ctx_identifier, errstr);
else
- rd_kafka_log(rk, LOG_ERR, "SSL", "%s", errstr);
+ rd_kafka_log(rk, LOG_ERR, "SSL", "%s: %s",
+ ctx_identifier, errstr);
}
ERR_error_string_n(l, buf, sizeof(buf));
@@ -188,12 +193,18 @@ static char *rd_kafka_ssl_error(rd_kafka_t *rk,
if (cnt == 0)
rd_snprintf(errstr, errstr_size,
- "No further error information available");
+ "%s: No further error information available",
+ ctx_identifier);
return errstr;
}
-
+static char *rd_kafka_ssl_error(rd_kafka_t *rk,
+ rd_kafka_broker_t *rkb,
+ char *errstr,
+ size_t errstr_size) {
+ return rd_kafka_ssl_error0(rk, rkb, "kafka", errstr, errstr_size);
+}
/**
* Set transport IO event polling based on SSL error.
@@ -225,16 +236,22 @@ rd_kafka_transport_ssl_io_update(rd_kafka_transport_t *rktrans,
if (serr2)
rd_kafka_ssl_error(NULL, rktrans->rktrans_rkb, errstr,
errstr_size);
- else if (!rd_socket_errno || rd_socket_errno == ECONNRESET)
- rd_snprintf(errstr, errstr_size, "Disconnected");
- else
+ else if (!rd_socket_errno) {
+ rd_snprintf(errstr, errstr_size,
+ "Disconnected: connection closed by "
+ "peer");
+ } else if (rd_socket_errno == ECONNRESET) {
+ rd_snprintf(errstr, errstr_size,
+ "Disconnected: connection reset by peer");
+ } else
rd_snprintf(errstr, errstr_size,
"SSL transport error: %s",
rd_strerror(rd_socket_errno));
return -1;
case SSL_ERROR_ZERO_RETURN:
- rd_snprintf(errstr, errstr_size, "Disconnected");
+ rd_snprintf(errstr, errstr_size,
+ "Disconnected: SSL connection closed by peer");
return -1;
default:
@@ -656,7 +673,7 @@ int rd_kafka_transport_ssl_handshake(rd_kafka_transport_t *rktrans) {
" (install ca-certificates package)"
#endif
;
- else if (!strcmp(errstr, "Disconnected")) {
+ else if (rd_kafka_transport_error_disconnected(errstr)) {
extra = ": connecting to a PLAINTEXT broker listener?";
/* Disconnects during handshake are most likely
* not due to SSL, but rather at the transport level */
@@ -697,22 +714,92 @@ static EVP_PKEY *rd_kafka_ssl_PKEY_from_string(rd_kafka_t *rk,
return pkey;
}
+/**
+ * Read a PEM formatted cert chain from BIO \p in into \p chainp .
+ *
+ * @param rk rdkafka instance.
+ * @param in BIO to read from.
+ * @param chainp Stack to push the certificates to.
+ *
+ * @return 0 on success, -1 on error.
+ */
+int rd_kafka_ssl_read_cert_chain_from_BIO(BIO *in,
+ STACK_OF(X509) * chainp,
+ pem_password_cb *password_cb,
+ void *password_cb_opaque) {
+ X509 *ca;
+ int r, ret = 0;
+ unsigned long err;
+ while (1) {
+ ca = X509_new();
+ if (ca == NULL) {
+ rd_assert(!*"X509_new() allocation failed");
+ }
+ if (PEM_read_bio_X509(in, &ca, password_cb,
+ password_cb_opaque) != NULL) {
+ r = sk_X509_push(chainp, ca);
+ if (!r) {
+ X509_free(ca);
+ ret = -1;
+ goto end;
+ }
+ } else {
+ X509_free(ca);
+ break;
+ }
+ }
+ /* When the while loop ends, it's usually just EOF. */
+ err = ERR_peek_last_error();
+ if (ERR_GET_LIB(err) == ERR_LIB_PEM &&
+ ERR_GET_REASON(err) == PEM_R_NO_START_LINE)
+ ret = 0;
+ else
+ ret = -1; /* some real error */
+ ERR_clear_error();
+end:
+ return ret;
+}
+
/**
* @brief Parse a PEM-formatted string into an X509 object.
+ * Rest of CA chain is pushed to the \p chainp stack.
*
- * @param str Input PEM string, nul-terminated
+ * @param str Input PEM string, nul-terminated.
+ * @param chainp Stack to push the certificates to.
*
* @returns a new X509 on success or NULL on error.
+ *
+ * @remark When NULL is returned the chainp stack is not modified.
*/
-static X509 *rd_kafka_ssl_X509_from_string(rd_kafka_t *rk, const char *str) {
+static X509 *rd_kafka_ssl_X509_from_string(rd_kafka_t *rk,
+ const char *str,
+ STACK_OF(X509) * chainp) {
BIO *bio = BIO_new_mem_buf((void *)str, -1);
X509 *x509;
x509 =
PEM_read_bio_X509(bio, NULL, rd_kafka_transport_ssl_passwd_cb, rk);
- BIO_free(bio);
+ if (!x509) {
+ BIO_free(bio);
+ return NULL;
+ }
+
+ if (rd_kafka_ssl_read_cert_chain_from_BIO(
+ bio, chainp, rd_kafka_transport_ssl_passwd_cb, rk) != 0) {
+ /* Rest of the certificate is present,
+ * but couldn't be read,
+ * returning NULL as certificate cannot be verified
+ * without its chain. */
+ rd_kafka_log(rk, LOG_WARNING, "SSL",
+ "Failed to read certificate chain from PEM. "
+ "Returning NULL certificate too.");
+ X509_free(x509);
+ BIO_free(bio);
+ return NULL;
+ }
+ BIO_free(bio);
return x509;
}
@@ -723,6 +810,7 @@ static X509 *rd_kafka_ssl_X509_from_string(rd_kafka_t *rk, const char *str) {
* @brief Attempt load CA certificates from a Windows Certificate store.
*/
static int rd_kafka_ssl_win_load_cert_store(rd_kafka_t *rk,
+ const char *ctx_identifier,
SSL_CTX *ctx,
const char *store_name) {
HCERTSTORE w_store;
@@ -737,15 +825,16 @@ static int rd_kafka_ssl_win_load_cert_store(rd_kafka_t *rk,
/* Convert store_name to wide-char */
werr = mbstowcs_s(&wsize, NULL, 0, store_name, strlen(store_name));
if (werr || wsize < 2 || wsize > 1000) {
- rd_kafka_log(rk, LOG_ERR, "CERTSTORE",
- "Invalid Windows certificate store name: %.*s%s",
- 30, store_name,
- wsize < 2 ? " (empty)" : " (truncated)");
+ rd_kafka_log(
+ rk, LOG_ERR, "CERTSTORE",
+ "%s: Invalid Windows certificate store name: %.*s%s",
+ ctx_identifier, 30, store_name,
+ wsize < 2 ? " (empty)" : " (truncated)");
return -1;
}
wstore_name = rd_alloca(sizeof(*wstore_name) * wsize);
werr = mbstowcs_s(NULL, wstore_name, wsize, store_name,
- strlen(store_name));
+ strlen(store_name));
rd_assert(!werr);
w_store = CertOpenStore(CERT_STORE_PROV_SYSTEM, 0, 0,
@@ -756,9 +845,9 @@ static int rd_kafka_ssl_win_load_cert_store(rd_kafka_t *rk,
if (!w_store) {
rd_kafka_log(
rk, LOG_ERR, "CERTSTORE",
- "Failed to open Windows certificate "
+ "%s: Failed to open Windows certificate "
"%s store: %s",
- store_name,
+ ctx_identifier, store_name,
rd_strerror_w32(GetLastError(), errstr, sizeof(errstr)));
return -1;
}
@@ -794,9 +883,9 @@ static int rd_kafka_ssl_win_load_cert_store(rd_kafka_t *rk,
CertCloseStore(w_store, 0);
rd_kafka_dbg(rk, SECURITY, "CERTSTORE",
- "%d certificate(s) successfully added from "
+ "%s: %d certificate(s) successfully added from "
"Windows Certificate %s store, %d failed",
- cnt, store_name, fail_cnt);
+ ctx_identifier, cnt, store_name, fail_cnt);
if (cnt == 0 && fail_cnt > 0)
return -1;
@@ -809,9 +898,10 @@ static int rd_kafka_ssl_win_load_cert_store(rd_kafka_t *rk,
*
* @returns the number of successfully loaded certificates, or -1 on error.
*/
-static int rd_kafka_ssl_win_load_cert_stores(rd_kafka_t *rk,
- SSL_CTX *ctx,
- const char *store_names) {
+int rd_kafka_ssl_win_load_cert_stores(rd_kafka_t *rk,
+ const char *ctx_identifier,
+ SSL_CTX *ctx,
+ const char *store_names) {
char *s;
int cert_cnt = 0, fail_cnt = 0;
@@ -845,7 +935,8 @@ static int rd_kafka_ssl_win_load_cert_stores(rd_kafka_t *rk,
s = "";
}
- r = rd_kafka_ssl_win_load_cert_store(rk, ctx, store_name);
+ r = rd_kafka_ssl_win_load_cert_store(rk, ctx_identifier, ctx,
+ store_name);
if (r != -1)
cert_cnt += r;
else
@@ -859,7 +950,32 @@ static int rd_kafka_ssl_win_load_cert_stores(rd_kafka_t *rk,
}
#endif /* MSC_VER */
+/**
+ * @brief Probe for a single \p path and if found and not an empty directory,
+ * set it on the \p ctx.
+ *
+ * @returns 0 if CA location was set with an error, 1 if it was set correctly,
+ * -1 if path should be skipped.
+ */
+static int rd_kafka_ssl_set_ca_path(rd_kafka_t *rk,
+ const char *ctx_identifier,
+ const char *path,
+ SSL_CTX *ctx,
+ rd_bool_t *is_dir) {
+ if (!rd_file_stat(path, is_dir))
+ return -1;
+
+ if (*is_dir && rd_kafka_dir_is_empty(path))
+ return -1;
+
+ rd_kafka_dbg(rk, SECURITY, "CACERTS",
+ "Setting default CA certificate location for %s "
+ "to \"%s\"",
+ ctx_identifier, path);
+ return SSL_CTX_load_verify_locations(ctx, *is_dir ? NULL : path,
+ *is_dir ? path : NULL);
+}
/**
* @brief Probe for the system's CA certificate location and if found set it
@@ -867,8 +983,9 @@ static int rd_kafka_ssl_win_load_cert_stores(rd_kafka_t *rk,
*
* @returns 0 if CA location was set, else -1.
*/
-static int rd_kafka_ssl_probe_and_set_default_ca_location(rd_kafka_t *rk,
- SSL_CTX *ctx) {
+int rd_kafka_ssl_probe_and_set_default_ca_location(rd_kafka_t *rk,
+ const char *ctx_identifier,
+ SSL_CTX *ctx) {
#if _WIN32
/* No standard location on Windows, CA certs are in the ROOT store. */
return -1;
@@ -920,34 +1037,21 @@ static int rd_kafka_ssl_probe_and_set_default_ca_location(rd_kafka_t *rk,
int i;
for (i = 0; (path = paths[i]); i++) {
- struct stat st;
rd_bool_t is_dir;
- int r;
-
- if (stat(path, &st) != 0)
+ int r = rd_kafka_ssl_set_ca_path(rk, ctx_identifier, path, ctx,
+ &is_dir);
+ if (r == -1)
continue;
- is_dir = S_ISDIR(st.st_mode);
-
- if (is_dir && rd_kafka_dir_is_empty(path))
- continue;
-
- rd_kafka_dbg(rk, SECURITY, "CACERTS",
- "Setting default CA certificate location "
- "to %s, override with ssl.ca.location",
- path);
-
- r = SSL_CTX_load_verify_locations(ctx, is_dir ? NULL : path,
- is_dir ? path : NULL);
if (r != 1) {
char errstr[512];
/* Read error and clear the error stack */
rd_kafka_ssl_error(rk, NULL, errstr, sizeof(errstr));
rd_kafka_dbg(rk, SECURITY, "CACERTS",
"Failed to set default CA certificate "
- "location to %s %s: %s: skipping",
+ "location to %s %s for %s: %s: skipping",
is_dir ? "directory" : "file", path,
- errstr);
+ ctx_identifier, errstr);
continue;
}
@@ -956,11 +1060,108 @@ static int rd_kafka_ssl_probe_and_set_default_ca_location(rd_kafka_t *rk,
rd_kafka_dbg(rk, SECURITY, "CACERTS",
"Unable to find any standard CA certificate"
- "paths: is the ca-certificates package installed?");
+ "paths for %s: is the ca-certificates package installed?",
+ ctx_identifier);
return -1;
#endif
}
+/**
+ * @brief Simple utility function to check if \p ca DN is matching
+ * any of the DNs in the \p ca_dns stack.
+ */
+static int rd_kafka_ssl_cert_issuer_match(STACK_OF(X509_NAME) * ca_dns,
+ X509 *ca) {
+ X509_NAME *issuer_dn = X509_get_issuer_name(ca);
+ X509_NAME *dn;
+ int i;
+
+ for (i = 0; i < sk_X509_NAME_num(ca_dns); i++) {
+ dn = sk_X509_NAME_value(ca_dns, i);
+ if (0 == X509_NAME_cmp(dn, issuer_dn)) {
+ /* match found */
+ return 1;
+ }
+ }
+ return 0;
+}
+
+/**
+ * @brief callback function for SSL_CTX_set_cert_cb, see
+ * https://docs.openssl.org/master/man3/SSL_CTX_set_cert_cb for details
+ * of the callback function requirements.
+ *
+ * According to section 4.2.4 of RFC 8446:
+ * The "certificate_authorities" extension is used to indicate the
+ * certificate authorities (CAs) which an endpoint supports and which
+ * SHOULD be used by the receiving endpoint to guide certificate
+ * selection.
+ *
+ * We avoid sending a client certificate if the issuer doesn't match any DN
+ * of server trusted certificate authorities (SSL_get_client_CA_list).
+ * This is done to avoid sending a client certificate that would almost
+ * certainly be rejected by the peer and would avoid successful
+ * SASL_SSL authentication on the same connection in case
+ * `ssl.client.auth=requested`.
+ */
+static int rd_kafka_ssl_cert_callback(SSL *ssl, void *arg) {
+ rd_kafka_t *rk = arg;
+ STACK_OF(X509_NAME) * ca_list;
+ STACK_OF(X509) *certs = NULL;
+ X509 *cert;
+ int i;
+
+ /* Get client cert from SSL connection */
+ cert = SSL_get_certificate(ssl);
+ if (cert == NULL) {
+ /* If there's no client certificate,
+ * skip certificate issuer verification and
+ * avoid logging a warning. */
+ return 1;
+ }
+
+ /* Get the accepted client CA list from the SSL connection, this
+ * comes from the `certificate_authorities` field. */
+ ca_list = SSL_get_client_CA_list(ssl);
+ if (sk_X509_NAME_num(ca_list) < 1) {
+ /* `certificate_authorities` is supported either
+ * in CertificateRequest (SSL <= 3, TLS <= 1.2)
+ * or as an extension (TLS >= 1.3). This should be always
+ * available, but in case it isn't, just send the certificate
+ * and let the server validate it. */
+ return 1;
+ }
+
+ if (rd_kafka_ssl_cert_issuer_match(ca_list, cert)) {
+ /* A match is found, use the certificate. */
+ return 1;
+ }
+
+ /* Get client cert chain from SSL connection */
+ SSL_get0_chain_certs(ssl, &certs);
+
+ if (certs) {
+ /* Check if there's a match in the CA list for
+ * each cert in the chain. */
+ for (i = 0; i < sk_X509_num(certs); i++) {
+ cert = sk_X509_value(certs, i);
+ if (rd_kafka_ssl_cert_issuer_match(ca_list, cert)) {
+ /* A match is found, use the certificate. */
+ return 1;
+ }
+ }
+ }
+
+ /* No match is found, which means they would almost certainly be
+ * rejected by the peer.
+ * We decide to send no certificates. */
+ rd_kafka_log(rk, LOG_WARNING, "SSL",
+ "No matching issuer found in "
+ "server trusted certificate authorities, "
+ "not sending any client certificates");
+ SSL_certs_clear(ssl);
+ return 1;
+}
/**
* @brief Registers certificates, keys, etc, on the SSL_CTX
@@ -1057,7 +1258,7 @@ static int rd_kafka_ssl_set_certs(rd_kafka_t *rk,
/* Attempt to load CA root certificates from the
* configured Windows certificate stores. */
r = rd_kafka_ssl_win_load_cert_stores(
- rk, ctx, rk->rk_conf.ssl.ca_cert_stores);
+ rk, "kafka", ctx, rk->rk_conf.ssl.ca_cert_stores);
if (r == 0) {
rd_kafka_log(
rk, LOG_NOTICE, "CERTSTORE",
@@ -1088,8 +1289,8 @@ static int rd_kafka_ssl_set_certs(rd_kafka_t *rk,
* of standard CA certificate paths and use the
* first one that is found.
* Ignore failures. */
- r = rd_kafka_ssl_probe_and_set_default_ca_location(rk,
- ctx);
+ r = rd_kafka_ssl_probe_and_set_default_ca_location(
+ rk, "kafka", ctx);
}
if (r == -1) {
@@ -1146,6 +1347,20 @@ static int rd_kafka_ssl_set_certs(rd_kafka_t *rk,
rd_snprintf(errstr, errstr_size, "ssl_cert failed: ");
return -1;
}
+
+ if (rk->rk_conf.ssl.cert->chain) {
+ r = SSL_CTX_set0_chain(ctx,
+ rk->rk_conf.ssl.cert->chain);
+ if (r != 1) {
+ rd_snprintf(errstr, errstr_size,
+ "ssl_cert failed: "
+ "setting certificate chain: ");
+ return -1;
+ } else {
+ /* The chain is now owned by the CTX */
+ rk->rk_conf.ssl.cert->chain = NULL;
+ }
+ }
}
if (rk->rk_conf.ssl.cert_location) {
@@ -1165,16 +1380,21 @@ static int rd_kafka_ssl_set_certs(rd_kafka_t *rk,
if (rk->rk_conf.ssl.cert_pem) {
X509 *x509;
+ STACK_OF(X509) *ca = sk_X509_new_null();
+ if (!ca) {
+ rd_assert(!*"sk_X509_new_null() allocation failed");
+ }
rd_kafka_dbg(rk, SECURITY, "SSL",
"Loading public key from string");
- x509 =
- rd_kafka_ssl_X509_from_string(rk, rk->rk_conf.ssl.cert_pem);
+ x509 = rd_kafka_ssl_X509_from_string(
+ rk, rk->rk_conf.ssl.cert_pem, ca);
if (!x509) {
rd_snprintf(errstr, errstr_size,
"ssl.certificate.pem failed: "
"not in PEM format?: ");
+ sk_X509_pop_free(ca, X509_free);
return -1;
}
@@ -1184,11 +1404,25 @@ static int rd_kafka_ssl_set_certs(rd_kafka_t *rk,
if (r != 1) {
rd_snprintf(errstr, errstr_size,
- "ssl.certificate.pem failed: ");
+ "ssl.certificate.pem failed: "
+ "setting main certificate: ");
+ sk_X509_pop_free(ca, X509_free);
return -1;
}
- }
+ if (sk_X509_num(ca) == 0) {
+ sk_X509_pop_free(ca, X509_free);
+ } else {
+ r = SSL_CTX_set0_chain(ctx, ca);
+ if (r != 1) {
+ rd_snprintf(errstr, errstr_size,
+ "ssl.certificate.pem failed: "
+ "setting certificate chain: ");
+ sk_X509_pop_free(ca, X509_free);
+ return -1;
+ }
+ }
+ }
/*
* ssl_key, ssl.key.location and ssl.key.pem
@@ -1261,8 +1495,8 @@ static int rd_kafka_ssl_set_certs(rd_kafka_t *rk,
* ssl.keystore.location
*/
if (rk->rk_conf.ssl.keystore_location) {
- EVP_PKEY *pkey;
- X509 *cert;
+ EVP_PKEY *pkey = NULL;
+ X509 *cert = NULL;
STACK_OF(X509) *ca = NULL;
BIO *bio;
PKCS12 *p12;
@@ -1290,8 +1524,6 @@ static int rd_kafka_ssl_set_certs(rd_kafka_t *rk,
return -1;
}
- pkey = EVP_PKEY_new();
- cert = X509_new();
if (!PKCS12_parse(p12, rk->rk_conf.ssl.keystore_password, &pkey,
&cert, &ca)) {
EVP_PKEY_free(pkey);
@@ -1307,28 +1539,17 @@ static int rd_kafka_ssl_set_certs(rd_kafka_t *rk,
return -1;
}
- if (ca != NULL)
- sk_X509_pop_free(ca, X509_free);
-
PKCS12_free(p12);
BIO_free(bio);
- r = SSL_CTX_use_certificate(ctx, cert);
- X509_free(cert);
- if (r != 1) {
- EVP_PKEY_free(pkey);
- rd_snprintf(errstr, errstr_size,
- "Failed to use ssl.keystore.location "
- "certificate: ");
- return -1;
- }
-
- r = SSL_CTX_use_PrivateKey(ctx, pkey);
- EVP_PKEY_free(pkey);
+ r = SSL_CTX_use_cert_and_key(ctx, cert, pkey, ca, 1);
+ RD_IF_FREE(cert, X509_free);
+ RD_IF_FREE(pkey, EVP_PKEY_free);
+ if (ca != NULL)
+ sk_X509_pop_free(ca, X509_free);
if (r != 1) {
rd_snprintf(errstr, errstr_size,
- "Failed to use ssl.keystore.location "
- "private key: ");
+ "Failed to use ssl.keystore.location: ");
return -1;
}
@@ -1413,6 +1634,10 @@ static int rd_kafka_ssl_set_certs(rd_kafka_t *rk,
return -1;
}
+ /* Set client certificate callback to control the behaviour
+ * of client certificate selection TLS handshake. */
+ SSL_CTX_set_cert_cb(ctx, rd_kafka_ssl_cert_callback, rk);
+
return 0;
}
@@ -1547,8 +1772,8 @@ static rd_bool_t rd_kafka_ssl_ctx_load_providers(rd_kafka_t *rk,
OSSL_PROVIDER *prov;
const char *buildinfo = NULL;
OSSL_PARAM request[] = {{"buildinfo", OSSL_PARAM_UTF8_PTR,
- (void *)&buildinfo, 0, 0},
- {NULL, 0, NULL, 0, 0}};
+ (void *)&buildinfo, 0, 0},
+ {NULL, 0, NULL, 0, 0}};
prov = OSSL_PROVIDER_load(NULL, provider);
if (!prov) {
@@ -1798,7 +2023,7 @@ void rd_kafka_ssl_init(void) {
if (!CRYPTO_get_locking_callback()) {
rd_kafka_ssl_locks_cnt = CRYPTO_num_locks();
rd_kafka_ssl_locks = rd_malloc(rd_kafka_ssl_locks_cnt *
- sizeof(*rd_kafka_ssl_locks));
+ sizeof(*rd_kafka_ssl_locks));
for (i = 0; i < rd_kafka_ssl_locks_cnt; i++)
mtx_init(&rd_kafka_ssl_locks[i], mtx_plain);
diff --git a/src/rdkafka_ssl.h b/src/rdkafka_ssl.h
index 4dce0b1f87..6307857c1d 100644
--- a/src/rdkafka_ssl.h
+++ b/src/rdkafka_ssl.h
@@ -61,4 +61,26 @@ int rd_kafka_ssl_hmac(rd_kafka_broker_t *rkb,
int itcnt,
rd_chariov_t *out);
+int rd_kafka_ssl_read_cert_chain_from_BIO(BIO *in,
+ STACK_OF(X509) * chainp,
+ pem_password_cb *password_cb,
+ void *password_cb_opaque);
+
+int rd_kafka_ssl_probe_and_set_default_ca_location(rd_kafka_t *rk,
+ const char *ctx_identifier,
+ SSL_CTX *ctx);
+
+char *rd_kafka_ssl_error0(rd_kafka_t *rk,
+ rd_kafka_broker_t *rkb,
+ const char *ctx_identifier,
+ char *errstr,
+ size_t errstr_size);
+
+#ifdef _WIN32
+int rd_kafka_ssl_win_load_cert_stores(rd_kafka_t *rk,
+ const char *ctx_identifier,
+ SSL_CTX *ctx,
+ const char *store_names);
+#endif
+
#endif /* _RDKAFKA_SSL_H_ */
diff --git a/src/rdkafka_sticky_assignor.c b/src/rdkafka_sticky_assignor.c
index 5b7658712c..6141a23a7d 100644
--- a/src/rdkafka_sticky_assignor.c
+++ b/src/rdkafka_sticky_assignor.c
@@ -819,6 +819,7 @@ isBalanced(rd_kafka_t *rk,
* currentAssignment's element we get both the consumer
* and partition list in elem here. */
RD_LIST_FOREACH(elem, sortedCurrentSubscriptions, i) {
+ int j;
const char *consumer = (const char *)elem->key;
const rd_kafka_topic_partition_list_t *potentialTopicPartitions;
const rd_kafka_topic_partition_list_t *consumerPartitions;
@@ -836,9 +837,9 @@ isBalanced(rd_kafka_t *rk,
/* Otherwise make sure it can't get any more partitions */
- for (i = 0; i < potentialTopicPartitions->cnt; i++) {
+ for (j = 0; j < potentialTopicPartitions->cnt; j++) {
const rd_kafka_topic_partition_t *partition =
- &potentialTopicPartitions->elems[i];
+ &potentialTopicPartitions->elems[j];
const char *otherConsumer;
int otherConsumerPartitionCount;
@@ -3026,6 +3027,10 @@ static int ut_testLargeAssignmentWithMultipleConsumersLeaving(
rd_kafka_t *rk,
const rd_kafka_assignor_t *rkas,
rd_kafka_assignor_ut_rack_config_t parametrization) {
+ if (rd_unittest_with_valgrind)
+ RD_UT_SKIP(
+ "Skipping large assignment test when using Valgrind");
+
rd_kafka_resp_err_t err;
char errstr[512];
rd_kafka_metadata_t *metadata;
diff --git a/src/rdkafka_telemetry.c b/src/rdkafka_telemetry.c
index 3f2fece177..b76effa9ed 100644
--- a/src/rdkafka_telemetry.c
+++ b/src/rdkafka_telemetry.c
@@ -33,6 +33,7 @@
#include "nanopb/pb.h"
#include "rdkafka_lz4.h"
#include "snappy.h"
+#include "rdunittest.h"
#if WITH_ZSTD
#include "rdkafka_zstd.h"
@@ -51,10 +52,8 @@
static int
rd_kafka_filter_broker_by_GetTelemetrySubscription(rd_kafka_broker_t *rkb,
void *opaque) {
- int features;
- if (rd_kafka_broker_ApiVersion_supported0(
- rkb, RD_KAFKAP_GetTelemetrySubscriptions, 0, 0, &features,
- rd_false) != -1)
+ if (rd_kafka_broker_ApiVersion_at_least_no_lock(
+ rkb, RD_KAFKAP_GetTelemetrySubscriptions, 0))
return 0;
return 1;
}
@@ -173,6 +172,9 @@ static void update_matched_metrics(rd_kafka_t *rk, size_t j) {
static void rd_kafka_match_requested_metrics(rd_kafka_t *rk) {
size_t metrics_cnt = RD_KAFKA_TELEMETRY_METRIC_CNT(rk), i;
+ rd_bool_t is_metric_included[RD_MAX(
+ (int)RD_KAFKA_TELEMETRY_PRODUCER_METRIC__CNT,
+ (int)RD_KAFKA_TELEMETRY_CONSUMER_METRIC__CNT)] = {0};
const rd_kafka_telemetry_metric_info_t *info =
RD_KAFKA_TELEMETRY_METRIC_INFO(rk);
@@ -193,6 +195,9 @@ static void rd_kafka_match_requested_metrics(rd_kafka_t *rk) {
j;
for (j = 0; j < metrics_cnt; j++) {
+ if (is_metric_included[j])
+ continue;
+
/* Prefix matching the requested metrics with the
* available metrics. */
char full_metric_name
@@ -205,8 +210,10 @@ static void rd_kafka_match_requested_metrics(rd_kafka_t *rk) {
rk->rk_telemetry.requested_metrics[i],
name_len) == 0;
- if (name_matches)
+ if (name_matches) {
update_matched_metrics(rk, j);
+ is_metric_included[j] = rd_true;
+ }
}
}
@@ -266,6 +273,24 @@ rd_kafka_push_telemetry_payload_compress(rd_kafka_t *rk,
rd_slice_t payload_slice;
size_t i;
rd_kafka_resp_err_t r = RD_KAFKA_RESP_ERR_NO_ERROR;
+
+ if (payload->rbuf_len == 0) {
+ /* We can only initialize the slice to compress
+ * if not empty. */
+ rd_kafka_dbg(rk, TELEMETRY, "PUSH",
+ "Empty payload. "
+ "Sending uncompressed payload");
+
+ /* It's not important the payload isn't actually a segment
+ * inside the buffer, as size is 0, we can send any allocated
+ * memory here, but we chose the buffer because it's
+ * freed like the other COMPRESSION_NONE case, without
+ * memory leaks. */
+ *compressed_payload = payload;
+ *compressed_payload_size = 0;
+ return RD_KAFKA_COMPRESSION_NONE;
+ }
+
rd_slice_init_full(&payload_slice, payload);
for (i = 0; i < rk->rk_telemetry.accepted_compression_types_cnt; i++) {
rd_kafka_compression_t compression_type =
@@ -343,33 +368,42 @@ static void rd_kafka_send_push_telemetry(rd_kafka_t *rk,
rd_bool_t terminating) {
rd_buf_t *metrics_payload = rd_kafka_telemetry_encode_metrics(rk);
- size_t compressed_metrics_payload_size = 0;
- void *compressed_metrics_payload = NULL;
- rd_kafka_compression_t compression_used =
- rd_kafka_push_telemetry_payload_compress(
- rk, rkb, metrics_payload, &compressed_metrics_payload,
- &compressed_metrics_payload_size);
- if (compressed_metrics_payload_size >
- (size_t)rk->rk_telemetry.telemetry_max_bytes) {
- rd_kafka_log(rk, LOG_WARNING, "TELEMETRY",
- "Metrics payload size %" PRIusz
- " exceeds telemetry_max_bytes %" PRId32
- "specified by the broker.",
- compressed_metrics_payload_size,
- rk->rk_telemetry.telemetry_max_bytes);
+ size_t compressed_metrics_payload_size = 0;
+ void *compressed_metrics_payload = NULL;
+ rd_kafka_compression_t compression_used = RD_KAFKA_COMPRESSION_NONE;
+ if (metrics_payload) {
+ compression_used = rd_kafka_push_telemetry_payload_compress(
+ rk, rkb, metrics_payload, &compressed_metrics_payload,
+ &compressed_metrics_payload_size);
+ if (compressed_metrics_payload_size >
+ (size_t)rk->rk_telemetry.telemetry_max_bytes) {
+ rd_kafka_log(rk, LOG_WARNING, "TELEMETRY",
+ "Metrics payload size %" PRIusz
+ " exceeds telemetry_max_bytes %" PRId32
+ "specified by the broker.",
+ compressed_metrics_payload_size,
+ rk->rk_telemetry.telemetry_max_bytes);
+ }
+
+ rd_kafka_dbg(
+ rk, TELEMETRY, "PUSH",
+ "Sending PushTelemetryRequest with terminating = %s",
+ RD_STR_ToF(terminating));
+ rd_kafka_PushTelemetryRequest(
+ rkb, &rk->rk_telemetry.client_instance_id,
+ rk->rk_telemetry.subscription_id, terminating,
+ compression_used, compressed_metrics_payload,
+ compressed_metrics_payload_size, NULL, 0,
+ RD_KAFKA_REPLYQ(rk->rk_ops, 0),
+ rd_kafka_handle_PushTelemetry, NULL);
+ } else {
+ rd_kafka_log(rk, LOG_WARNING, "PUSH",
+ "Telemetry metrics encode error, not sending "
+ "metrics");
}
- rd_kafka_dbg(rk, TELEMETRY, "PUSH",
- "Sending PushTelemetryRequest with terminating = %s",
- RD_STR_ToF(terminating));
- rd_kafka_PushTelemetryRequest(
- rkb, &rk->rk_telemetry.client_instance_id,
- rk->rk_telemetry.subscription_id, terminating, compression_used,
- compressed_metrics_payload, compressed_metrics_payload_size, NULL,
- 0, RD_KAFKA_REPLYQ(rk->rk_ops, 0), rd_kafka_handle_PushTelemetry,
- NULL);
-
- rd_buf_destroy_free(metrics_payload);
+ if (metrics_payload)
+ rd_buf_destroy_free(metrics_payload);
if (compression_used != RD_KAFKA_COMPRESSION_NONE)
rd_free(compressed_metrics_payload);
@@ -382,7 +416,7 @@ static void rd_kafka_send_push_telemetry(rd_kafka_t *rk,
* @brief Progress the telemetry state machine.
*
* @locks none
- * @locks_acquired none
+ * @locks_acquired rd_kafka_rdlock()
* @locality main thread
*/
static void rd_kafka_telemetry_fsm(rd_kafka_t *rk) {
@@ -458,7 +492,7 @@ void rd_kafka_telemetry_fsm_tmr_cb(rd_kafka_timers_t *rkts, void *rk) {
* @brief Handles parsed GetTelemetrySubscriptions response.
*
* @locks none
- * @locks_acquired none
+ * @locks_acquired rd_kafka_rdlock()
* @locality main thread
*/
void rd_kafka_handle_get_telemetry_subscriptions(rd_kafka_t *rk,
@@ -495,10 +529,12 @@ void rd_kafka_handle_get_telemetry_subscriptions(rd_kafka_t *rk,
if (rk->rk_telemetry.rk_historic_c.ts_start == 0) {
rk->rk_telemetry.rk_historic_c.ts_start = now_ns;
rk->rk_telemetry.rk_historic_c.ts_last = now_ns;
+ rd_kafka_rdlock(rk);
TAILQ_FOREACH(rkb, &rk->rk_brokers, rkb_link) {
rkb->rkb_telemetry.rkb_historic_c.connects =
rd_atomic32_get(&rkb->rkb_c.connects);
}
+ rd_kafka_rdunlock(rk);
}
} else {
@@ -611,6 +647,7 @@ void rd_kafka_telemetry_await_termination(rd_kafka_t *rk) {
return;
}
+ mtx_lock(&rk->rk_telemetry.lock);
rko = rd_kafka_op_new(RD_KAFKA_OP_TERMINATE_TELEMETRY);
rko->rko_rk = rk;
rd_kafka_q_enq(rk->rk_ops, rko);
@@ -618,12 +655,8 @@ void rd_kafka_telemetry_await_termination(rd_kafka_t *rk) {
/* Await termination sequence completion. */
rd_kafka_dbg(rk, TELEMETRY, "TERM",
"Awaiting termination of telemetry.");
- mtx_lock(&rk->rk_telemetry.lock);
cnd_timedwait_ms(&rk->rk_telemetry.termination_cnd,
&rk->rk_telemetry.lock,
- /* TODO(milind): Evaluate this timeout after completion
- of all metrics push, is it too much, or too less if
- we include serialization? */
1000 /* timeout for waiting */);
mtx_unlock(&rk->rk_telemetry.lock);
rd_kafka_dbg(rk, TELEMETRY, "TERM",
@@ -695,3 +728,33 @@ void rd_kafka_set_telemetry_broker_maybe(rd_kafka_t *rk,
&rk->rk_timers, &rk->rk_telemetry.request_timer, rd_false,
0 /* immediate */, rd_kafka_telemetry_fsm_tmr_cb, (void *)rk);
}
+
+/**
+ * @brief Overlapping prefixes should not match the metrics
+ * multiple times.
+ */
+int unit_test_telemetry_match_requested_metrics_no_duplicates(void) {
+ rd_kafka_t *rk = rd_kafka_new(RD_KAFKA_PRODUCER, NULL, NULL, 0);
+ rk->rk_telemetry.requested_metrics_cnt = 3;
+ rk->rk_telemetry.requested_metrics =
+ rd_calloc(rk->rk_telemetry.requested_metrics_cnt, sizeof(char *));
+ rk->rk_telemetry.requested_metrics[0] = rd_strdup("org");
+ rk->rk_telemetry.requested_metrics[1] = rd_strdup("org.apache");
+ rk->rk_telemetry.requested_metrics[2] = rd_strdup("org.apache.kafka");
+ rd_kafka_match_requested_metrics(rk);
+
+ RD_UT_ASSERT(rk->rk_telemetry.matched_metrics_cnt ==
+ RD_KAFKA_TELEMETRY_PRODUCER_METRIC__CNT,
+ "Expected %d matched metrics, got %" PRIusz,
+ RD_KAFKA_TELEMETRY_PRODUCER_METRIC__CNT,
+ rk->rk_telemetry.matched_metrics_cnt);
+ rd_kafka_destroy(rk);
+ return 0;
+}
+
+
+int unittest_telemetry(void) {
+ int fails = 0;
+ fails += unit_test_telemetry_match_requested_metrics_no_duplicates();
+ return fails;
+}
diff --git a/src/rdkafka_telemetry_decode.c b/src/rdkafka_telemetry_decode.c
index e380b964ff..452e43c378 100644
--- a/src/rdkafka_telemetry_decode.c
+++ b/src/rdkafka_telemetry_decode.c
@@ -37,26 +37,60 @@
#define _NANOPB_STRING_DECODE_MAX_BUFFER_SIZE 1024
+#define MAX_LABELS 10
+#define UNITTEST_MARKER "unittest"
-struct metric_unit_test_data {
+enum unit_test_string_decoding_state {
+ STATE_LABELS,
+ STATE_VERSION,
+ STATE_METRIC_NAME,
+ STATE_METRIC_DESCRIPTION,
+ STATE_COMPLETE
+};
+
+struct unit_test_metric_label {
+ char key[_NANOPB_STRING_DECODE_MAX_BUFFER_SIZE];
+ char value[_NANOPB_STRING_DECODE_MAX_BUFFER_SIZE];
+};
+
+struct unit_test_data {
rd_kafka_telemetry_metric_type_t type;
int32_t current_field;
+ struct unit_test_metric_label labels[MAX_LABELS];
+ int label_count;
+ char version[_NANOPB_STRING_DECODE_MAX_BUFFER_SIZE];
char metric_name[_NANOPB_STRING_DECODE_MAX_BUFFER_SIZE];
char metric_description[_NANOPB_STRING_DECODE_MAX_BUFFER_SIZE];
int64_t metric_value_int;
+ int64_t expected_metric_value_int;
double metric_value_double;
+ double expected_metric_value_double;
+ int64_t int64_value;
uint64_t metric_time;
+ enum unit_test_string_decoding_state state;
+ bool expecting_label_value;
};
-static struct metric_unit_test_data unit_test_data;
+static struct unit_test_data unit_test_data;
-static void clear_unit_test_data(void) {
- unit_test_data.type = RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE;
- unit_test_data.current_field = 0;
- unit_test_data.metric_name[0] = '\0';
- unit_test_data.metric_description[0] = '\0';
- unit_test_data.metric_value_int = 0;
- unit_test_data.metric_time = 0;
+static void clear_unit_test_data(int64_t expected_value_int,
+ double expected_value_double) {
+ memset(&unit_test_data, 0, sizeof(unit_test_data));
+
+ unit_test_data.type = RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE;
+ unit_test_data.state = STATE_LABELS;
+ unit_test_data.expecting_label_value = false;
+ unit_test_data.version[0] = '\0';
+ unit_test_data.metric_name[0] = '\0';
+ unit_test_data.metric_description[0] = '\0';
+ unit_test_data.current_field = 0;
+ unit_test_data.label_count = 0;
+ unit_test_data.metric_value_int = 0;
+ unit_test_data.metric_value_double = 0.0;
+ unit_test_data.metric_time = 0;
+ unit_test_data.int64_value = 0;
+ unit_test_data.expected_metric_value_int = expected_value_int;
+ unit_test_data.expected_metric_value_double = expected_value_double;
}
static bool
@@ -401,20 +435,59 @@ int rd_kafka_telemetry_decode_metrics(
static void unit_test_telemetry_decoded_string(void *opaque,
const uint8_t *decoded) {
- switch (unit_test_data.current_field) {
- case 2:
+
+ switch (unit_test_data.state) {
+ case STATE_LABELS:
+ if (strcmp((const char *)decoded, UNITTEST_MARKER) == 0) {
+ unit_test_data.state = STATE_VERSION;
+ unit_test_data.expecting_label_value = false;
+ } else if (unit_test_data.expecting_label_value) {
+ rd_snprintf(unit_test_data
+ .labels[unit_test_data.label_count - 1]
+ .value,
+ sizeof(unit_test_data.labels[0].value),
+ "%s", decoded);
+ unit_test_data.expecting_label_value = false;
+ } else {
+ if (unit_test_data.label_count < MAX_LABELS) {
+ rd_snprintf(
+ unit_test_data
+ .labels[unit_test_data.label_count]
+ .key,
+ sizeof(unit_test_data.labels[0].key), "%s",
+ decoded);
+ unit_test_data.label_count++;
+ unit_test_data.expecting_label_value = true;
+ }
+ }
+ unit_test_data.current_field++;
+ break;
+
+ case STATE_VERSION:
+ rd_snprintf(unit_test_data.version,
+ sizeof(unit_test_data.version), "%s", decoded);
+ unit_test_data.state = STATE_METRIC_NAME;
+ unit_test_data.current_field++;
+ break;
+
+ case STATE_METRIC_NAME:
rd_snprintf(unit_test_data.metric_name,
sizeof(unit_test_data.metric_name), "%s", decoded);
+ unit_test_data.state = STATE_METRIC_DESCRIPTION;
+ unit_test_data.current_field++;
break;
- case 3:
+
+ case STATE_METRIC_DESCRIPTION:
rd_snprintf(unit_test_data.metric_description,
sizeof(unit_test_data.metric_description), "%s",
decoded);
+ unit_test_data.state = STATE_COMPLETE;
+ unit_test_data.current_field++;
break;
- default:
+
+ case STATE_COMPLETE:
break;
}
- unit_test_data.current_field++;
}
static void unit_test_telemetry_decoded_NumberDataPoint(
@@ -426,6 +499,11 @@ static void unit_test_telemetry_decoded_NumberDataPoint(
unit_test_data.current_field++;
}
+static void unit_test_telemetry_decoded_int64(void *opaque,
+ int64_t int64_value) {
+ unit_test_data.int64_value = int64_value;
+}
+
static void
unit_test_telemetry_decoded_type(void *opaque,
rd_kafka_telemetry_metric_type_t type) {
@@ -444,14 +522,33 @@ unit_test_telemetry_decode_error(void *opaque, const char *error, ...) {
rd_assert(!*"Failure while decoding telemetry data");
}
-bool unit_test_telemetry(rd_kafka_telemetry_producer_metric_name_t metric_name,
- const char *expected_name,
- const char *expected_description,
- rd_kafka_telemetry_metric_type_t expected_type,
- rd_bool_t is_double) {
+int unit_test_telemetry(rd_kafka_type_t rk_type,
+ rd_kafka_telemetry_producer_metric_name_t metric_name,
+ const char *expected_name,
+ const char *expected_description,
+ rd_kafka_telemetry_metric_type_t expected_type,
+ rd_bool_t is_double,
+ rd_bool_t is_per_broker,
+ void (*set_metric_value)(rd_kafka_t *,
+ rd_kafka_broker_t *),
+ int64_t expected_value_int,
+ double expected_value_double) {
rd_kafka_t *rk = rd_calloc(1, sizeof(*rk));
rwlock_init(&rk->rk_lock);
- rk->rk_type = RD_KAFKA_PRODUCER;
+ rd_kafka_conf_t *conf = rd_kafka_conf_new();
+ char *client_rack = "rack1", *transactional_id = "tx-id",
+ *group_id = "group-id", *group_instance_id = "group-instance-id";
+ rd_kafka_conf_set(conf, "client.rack", client_rack, NULL, 0);
+ rd_kafka_conf_set(conf, "transactional.id", transactional_id, NULL, 0);
+ rd_kafka_conf_set(conf, "group.id", group_id, NULL, 0);
+ rd_kafka_conf_set(conf, "group.instance.id", group_instance_id, NULL,
+ 0);
+ rk->rk_conf = *conf;
+ rd_free(conf);
+
+ rk->rk_type = rk_type;
+ rk->rk_cgrp = rd_calloc(1, sizeof(*rk->rk_cgrp));
+ rk->rk_broker_cnt.val = 1;
rk->rk_telemetry.matched_metrics_cnt = 1;
rk->rk_telemetry.matched_metrics =
rd_malloc(sizeof(rd_kafka_telemetry_producer_metric_name_t) *
@@ -461,34 +558,64 @@ bool unit_test_telemetry(rd_kafka_telemetry_producer_metric_name_t metric_name,
(rd_uclock() - 1000 * 1000) * 1000;
rk->rk_telemetry.rk_historic_c.ts_last =
(rd_uclock() - 1000 * 1000) * 1000;
+
+ rd_avg_init(&rk->rk_telemetry.rd_avg_current.rk_avg_poll_idle_ratio,
+ RD_AVG_GAUGE, 0, 500 * 1000, 2, rd_true);
+ rd_avg_init(&rk->rk_telemetry.rd_avg_current.rk_avg_commit_latency,
+ RD_AVG_GAUGE, 0, 500 * 1000, 2, rd_true);
+ rd_avg_init(&rk->rk_telemetry.rd_avg_current.rk_avg_rebalance_latency,
+ RD_AVG_GAUGE, 0, 500 * 1000, 2, rd_true);
+
+ rd_avg_init(&rk->rk_telemetry.rd_avg_rollover.rk_avg_poll_idle_ratio,
+ RD_AVG_GAUGE, 0, 500 * 1000, 2, rd_true);
+ rd_avg_init(&rk->rk_telemetry.rd_avg_rollover.rk_avg_commit_latency,
+ RD_AVG_GAUGE, 0, 500 * 1000, 2, rd_true);
+ rd_avg_init(&rk->rk_telemetry.rd_avg_rollover.rk_avg_rebalance_latency,
+ RD_AVG_GAUGE, 0, 500 * 1000, 2, rd_true);
+
rd_strlcpy(rk->rk_name, "unittest", sizeof(rk->rk_name));
- clear_unit_test_data();
+ clear_unit_test_data(expected_value_int, expected_value_double);
rd_kafka_telemetry_decode_interface_t decode_interface = {
.decoded_string = unit_test_telemetry_decoded_string,
.decoded_NumberDataPoint =
unit_test_telemetry_decoded_NumberDataPoint,
- .decoded_type = unit_test_telemetry_decoded_type,
- .decode_error = unit_test_telemetry_decode_error,
- .opaque = &unit_test_data,
+ .decoded_int64 = unit_test_telemetry_decoded_int64,
+ .decoded_type = unit_test_telemetry_decoded_type,
+ .decode_error = unit_test_telemetry_decode_error,
+ .opaque = &unit_test_data,
};
TAILQ_INIT(&rk->rk_brokers);
- rd_kafka_broker_t *rkb = rd_calloc(1, sizeof(*rkb));
- rkb->rkb_c.connects.val = 1;
+ rd_kafka_broker_t *rkb = rd_calloc(1, sizeof(*rkb));
+ rkb->rkb_nodeid = 1001;
+ mtx_init(&rkb->rkb_lock, mtx_plain);
+
rd_avg_init(&rkb->rkb_telemetry.rd_avg_current.rkb_avg_rtt,
RD_AVG_GAUGE, 0, 500 * 1000, 2, rd_true);
rd_avg_init(&rkb->rkb_telemetry.rd_avg_current.rkb_avg_outbuf_latency,
RD_AVG_GAUGE, 0, 500 * 1000, 2, rd_true);
rd_avg_init(&rkb->rkb_telemetry.rd_avg_current.rkb_avg_throttle,
RD_AVG_GAUGE, 0, 500 * 1000, 2, rd_true);
+ rd_avg_init(&rkb->rkb_telemetry.rd_avg_current.rkb_avg_fetch_latency,
+ RD_AVG_GAUGE, 0, 500 * 1000, 2, rd_true);
+
rd_avg_init(&rkb->rkb_telemetry.rd_avg_rollover.rkb_avg_rtt,
RD_AVG_GAUGE, 0, 500 * 1000, 2, rd_true);
rd_avg_init(&rkb->rkb_telemetry.rd_avg_rollover.rkb_avg_outbuf_latency,
RD_AVG_GAUGE, 0, 500 * 1000, 2, rd_true);
rd_avg_init(&rkb->rkb_telemetry.rd_avg_rollover.rkb_avg_throttle,
RD_AVG_GAUGE, 0, 500 * 1000, 2, rd_true);
+ rd_avg_init(&rkb->rkb_telemetry.rd_avg_rollover.rkb_avg_fetch_latency,
+ RD_AVG_GAUGE, 0, 500 * 1000, 2, rd_true);
+ rd_avg_init(&rkb->rkb_telemetry.rd_avg_current.rkb_avg_produce_latency,
+ RD_AVG_GAUGE, 0, 500 * 1000, 2, rd_true);
+ rd_avg_init(&rkb->rkb_telemetry.rd_avg_rollover.rkb_avg_produce_latency,
+ RD_AVG_GAUGE, 0, 500 * 1000, 2, rd_true);
+
+ set_metric_value(rk, rkb);
+
TAILQ_INSERT_HEAD(&rk->rk_brokers, rkb, rkb_link);
rd_buf_t *rbuf = rd_kafka_telemetry_encode_metrics(rk);
void *metrics_payload = rbuf->rbuf_wpos->seg_p;
@@ -504,18 +631,62 @@ bool unit_test_telemetry(rd_kafka_telemetry_producer_metric_name_t metric_name,
RD_UT_ASSERT(unit_test_data.type == expected_type,
"Metric type mismatch");
RD_UT_ASSERT(strcmp(unit_test_data.metric_name, expected_name) == 0,
- "Metric name mismatch");
+ "Metric name mismatch %s != %s",
+ unit_test_data.metric_name, expected_name);
RD_UT_ASSERT(strcmp(unit_test_data.metric_description,
expected_description) == 0,
"Metric description mismatch");
if (is_double)
RD_UT_ASSERT(
- rd_dbl_eq0(unit_test_data.metric_value_double, 1.0, 0.01),
+ rd_dbl_eq0(unit_test_data.metric_value_double,
+ unit_test_data.expected_metric_value_double,
+ 0.01),
"Metric value mismatch");
else
- RD_UT_ASSERT(unit_test_data.metric_value_int == 1,
+ RD_UT_ASSERT(unit_test_data.metric_value_int ==
+ unit_test_data.expected_metric_value_int,
"Metric value mismatch");
+ if (is_per_broker)
+ RD_UT_ASSERT(unit_test_data.int64_value == 1001,
+ "Expected broker mismatch");
RD_UT_ASSERT(unit_test_data.metric_time != 0, "Metric time mismatch");
+ if (rk_type == RD_KAFKA_PRODUCER) {
+ RD_UT_ASSERT(unit_test_data.label_count == 2,
+ "Label count mismatch");
+ RD_UT_ASSERT(
+ strcmp(unit_test_data.labels[0].key, "client_rack") == 0,
+ "Client rack key mismatch");
+ RD_UT_ASSERT(
+ strcmp(unit_test_data.labels[0].value, client_rack) == 0,
+ "Client rack value mismatch");
+ RD_UT_ASSERT(strcmp(unit_test_data.labels[1].key,
+ "transactional_id") == 0,
+ "Transactional id key mismatch");
+ RD_UT_ASSERT(strcmp(unit_test_data.labels[1].value,
+ transactional_id) == 0,
+ "Transactional id value mismatch");
+ } else {
+ RD_UT_ASSERT(unit_test_data.label_count == 3,
+ "Label count mismatch");
+ RD_UT_ASSERT(
+ strcmp(unit_test_data.labels[0].key, "client_rack") == 0,
+ "Client rack key mismatch");
+ RD_UT_ASSERT(
+ strcmp(unit_test_data.labels[0].value, client_rack) == 0,
+ "Client rack value mismatch");
+ RD_UT_ASSERT(strcmp(unit_test_data.labels[1].key, "group_id") ==
+ 0,
+ "Group id key mismatch");
+ RD_UT_ASSERT(strcmp(unit_test_data.labels[1].value, group_id) ==
+ 0,
+ "Group id value mismatch");
+ RD_UT_ASSERT(strcmp(unit_test_data.labels[2].key,
+ "group_instance_id") == 0,
+ "Group instance id key mismatch");
+ RD_UT_ASSERT(strcmp(unit_test_data.labels[2].value,
+ group_instance_id) == 0,
+ "Group instance id value mismatch");
+ }
rd_free(rk->rk_telemetry.matched_metrics);
rd_buf_destroy_free(rbuf);
@@ -527,28 +698,351 @@ bool unit_test_telemetry(rd_kafka_telemetry_producer_metric_name_t metric_name,
rd_avg_destroy(
&rkb->rkb_telemetry.rd_avg_rollover.rkb_avg_outbuf_latency);
rd_avg_destroy(&rkb->rkb_telemetry.rd_avg_rollover.rkb_avg_throttle);
+
+ rd_avg_destroy(
+ &rkb->rkb_telemetry.rd_avg_current.rkb_avg_fetch_latency);
+ rd_avg_destroy(
+ &rkb->rkb_telemetry.rd_avg_rollover.rkb_avg_fetch_latency);
+
+ rd_avg_destroy(&rk->rk_telemetry.rd_avg_current.rk_avg_poll_idle_ratio);
+ rd_avg_destroy(
+ &rk->rk_telemetry.rd_avg_rollover.rk_avg_poll_idle_ratio);
+
+ rd_avg_destroy(
+ &rk->rk_telemetry.rd_avg_current.rk_avg_rebalance_latency);
+ rd_avg_destroy(
+ &rk->rk_telemetry.rd_avg_rollover.rk_avg_rebalance_latency);
+
+ rd_avg_destroy(&rk->rk_telemetry.rd_avg_current.rk_avg_commit_latency);
+ rd_avg_destroy(&rk->rk_telemetry.rd_avg_rollover.rk_avg_commit_latency);
+
+ rd_avg_destroy(
+ &rkb->rkb_telemetry.rd_avg_current.rkb_avg_produce_latency);
+ rd_avg_destroy(
+ &rkb->rkb_telemetry.rd_avg_rollover.rkb_avg_produce_latency);
+
+ mtx_destroy(&rkb->rkb_lock);
rd_free(rkb);
rwlock_destroy(&rk->rk_lock);
+ rd_free(rk->rk_cgrp);
+ rd_kafka_anyconf_destroy(_RK_GLOBAL, &rk->rk_conf);
rd_free(rk);
RD_UT_PASS();
+ return 0;
+}
+
+void unit_test_telemetry_set_connects(rd_kafka_t *rk, rd_kafka_broker_t *rkb) {
+ rkb->rkb_c.connects.val = 1;
+}
+
+void unit_test_telemetry_set_connects2(rd_kafka_t *rk, rd_kafka_broker_t *rkb) {
+ rkb->rkb_c.connects.val = 2;
+}
+
+void unit_test_telemetry_set_rtt(rd_kafka_t *rk, rd_kafka_broker_t *rkb) {
+ rd_avg_add(&rkb->rkb_telemetry.rd_avg_current.rkb_avg_rtt, 1000);
+ rd_avg_add(&rkb->rkb_telemetry.rd_avg_current.rkb_avg_rtt, 1000);
+}
+
+void unit_test_telemetry_set_throttle_time(rd_kafka_t *rk,
+ rd_kafka_broker_t *rkb) {
+ rd_avg_add(&rkb->rkb_telemetry.rd_avg_current.rkb_avg_throttle, 1);
+ rd_avg_add(&rkb->rkb_telemetry.rd_avg_current.rkb_avg_throttle, 1);
+ rd_avg_add(&rkb->rkb_telemetry.rd_avg_current.rkb_avg_throttle, 1);
+}
+
+void unit_test_telemetry_set_queue_time(rd_kafka_t *rk,
+ rd_kafka_broker_t *rkb) {
+ rd_avg_add(&rkb->rkb_telemetry.rd_avg_current.rkb_avg_outbuf_latency,
+ 1000);
+ rd_avg_add(&rkb->rkb_telemetry.rd_avg_current.rkb_avg_outbuf_latency,
+ 1000);
+ rd_avg_add(&rkb->rkb_telemetry.rd_avg_current.rkb_avg_outbuf_latency,
+ 1000);
+ rd_avg_add(&rkb->rkb_telemetry.rd_avg_current.rkb_avg_outbuf_latency,
+ 1000);
+}
+
+void unit_test_telemetry_set_produce_latency(rd_kafka_t *rk,
+ rd_kafka_broker_t *rkb) {
+ rd_avg_add(&rkb->rkb_telemetry.rd_avg_current.rkb_avg_produce_latency,
+ 1000);
+ rd_avg_add(&rkb->rkb_telemetry.rd_avg_current.rkb_avg_produce_latency,
+ 1000);
+}
+
+void unit_test_telemetry_set_coordinator_assigned_partitions(
+ rd_kafka_t *rk,
+ rd_kafka_broker_t *rkb) {
+ rk->rk_cgrp->rkcg_c.assignment_size = 1;
+}
+
+void unit_test_telemetry_set_rebalance_latency(rd_kafka_t *rk,
+ rd_kafka_broker_t *rkb) {
+ rd_avg_add(&rk->rk_telemetry.rd_avg_current.rk_avg_rebalance_latency,
+ 1000);
+}
+
+void unit_test_telemetry_set_fetch_latency(rd_kafka_t *rk,
+ rd_kafka_broker_t *rkb) {
+ rd_avg_add(&rkb->rkb_telemetry.rd_avg_current.rkb_avg_fetch_latency,
+ 1000);
+ rd_avg_add(&rkb->rkb_telemetry.rd_avg_current.rkb_avg_fetch_latency,
+ 1000);
+}
+
+void unit_test_telemetry_set_poll_idle_ratio(rd_kafka_t *rk,
+ rd_kafka_broker_t *rkb) {
+ rd_avg_add(&rk->rk_telemetry.rd_avg_current.rk_avg_poll_idle_ratio,
+ 1000000);
+ rd_avg_add(&rk->rk_telemetry.rd_avg_current.rk_avg_poll_idle_ratio,
+ 1000000);
+ rd_avg_add(&rk->rk_telemetry.rd_avg_current.rk_avg_poll_idle_ratio,
+ 1000000);
+}
+
+void unit_test_telemetry_set_commit_latency(rd_kafka_t *rk,
+ rd_kafka_broker_t *rkb) {
+ rd_avg_add(&rk->rk_telemetry.rd_avg_current.rk_avg_commit_latency,
+ 1000);
+ rd_avg_add(&rk->rk_telemetry.rd_avg_current.rk_avg_commit_latency,
+ 1000);
}
-bool unit_test_telemetry_gauge(void) {
- return unit_test_telemetry(
+int unit_test_telemetry_gauge(void) {
+ int fails = 0;
+ int64_t default_expected_value_int = 1;
+ double default_expected_value_double = 1.0;
+ /* Producer metrics */
+ fails += unit_test_telemetry(
+ RD_KAFKA_PRODUCER,
RD_KAFKA_TELEMETRY_METRIC_PRODUCER_CONNECTION_CREATION_RATE,
RD_KAFKA_TELEMETRY_METRIC_PREFIX
"producer.connection.creation.rate",
"The rate of connections established per second.",
- RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE, rd_true);
+ RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE, rd_true, rd_false,
+ unit_test_telemetry_set_connects, default_expected_value_int,
+ default_expected_value_double);
+ fails += unit_test_telemetry(
+ RD_KAFKA_PRODUCER,
+ RD_KAFKA_TELEMETRY_METRIC_PRODUCER_NODE_REQUEST_LATENCY_AVG,
+ RD_KAFKA_TELEMETRY_METRIC_PREFIX
+ "producer.node.request.latency.avg",
+ "The average request latency in ms for a node.",
+ RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE, rd_true, rd_true,
+ unit_test_telemetry_set_rtt, default_expected_value_int,
+ default_expected_value_double);
+ fails += unit_test_telemetry(
+ RD_KAFKA_PRODUCER,
+ RD_KAFKA_TELEMETRY_METRIC_PRODUCER_NODE_REQUEST_LATENCY_MAX,
+ RD_KAFKA_TELEMETRY_METRIC_PREFIX
+ "producer.node.request.latency.max",
+ "The maximum request latency in ms for a node.",
+ RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE, rd_false, rd_true,
+ unit_test_telemetry_set_rtt, default_expected_value_int,
+ default_expected_value_double);
+ fails += unit_test_telemetry(
+ RD_KAFKA_PRODUCER,
+ RD_KAFKA_TELEMETRY_METRIC_PRODUCER_PRODUCE_THROTTLE_TIME_AVG,
+ RD_KAFKA_TELEMETRY_METRIC_PREFIX
+ "producer.produce.throttle.time.avg",
+ "The average throttle time in ms for a node.",
+ RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE, rd_true, rd_false,
+ unit_test_telemetry_set_throttle_time, default_expected_value_int,
+ default_expected_value_double);
+ fails += unit_test_telemetry(
+ RD_KAFKA_PRODUCER,
+ RD_KAFKA_TELEMETRY_METRIC_PRODUCER_PRODUCE_THROTTLE_TIME_MAX,
+ RD_KAFKA_TELEMETRY_METRIC_PREFIX
+ "producer.produce.throttle.time.max",
+ "The maximum throttle time in ms for a node.",
+ RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE, rd_false, rd_false,
+ unit_test_telemetry_set_throttle_time, default_expected_value_int,
+ default_expected_value_double);
+ fails += unit_test_telemetry(
+ RD_KAFKA_PRODUCER,
+ RD_KAFKA_TELEMETRY_METRIC_PRODUCER_RECORD_QUEUE_TIME_AVG,
+ RD_KAFKA_TELEMETRY_METRIC_PREFIX "producer.record.queue.time.avg",
+ "The average time in ms a record spends in the producer queue.",
+ RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE, rd_true, rd_false,
+ unit_test_telemetry_set_queue_time, default_expected_value_int,
+ default_expected_value_double);
+ fails += unit_test_telemetry(
+ RD_KAFKA_PRODUCER,
+ RD_KAFKA_TELEMETRY_METRIC_PRODUCER_RECORD_QUEUE_TIME_MAX,
+ RD_KAFKA_TELEMETRY_METRIC_PREFIX "producer.record.queue.time.max",
+ "The maximum time in ms a record spends in the producer queue.",
+ RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE, rd_false, rd_false,
+ unit_test_telemetry_set_queue_time, default_expected_value_int,
+ default_expected_value_double);
+ fails += unit_test_telemetry(
+ RD_KAFKA_PRODUCER,
+ RD_KAFKA_TELEMETRY_METRIC_PRODUCER_PRODUCE_LATENCY_AVG,
+ RD_KAFKA_TELEMETRY_METRIC_PREFIX "producer.request.latency.avg",
+ "The average request latency in ms for produce requests.",
+ RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE, rd_true, rd_false,
+ unit_test_telemetry_set_produce_latency, default_expected_value_int,
+ default_expected_value_double);
+ fails += unit_test_telemetry(
+ RD_KAFKA_PRODUCER,
+ RD_KAFKA_TELEMETRY_METRIC_PRODUCER_PRODUCE_LATENCY_MAX,
+ RD_KAFKA_TELEMETRY_METRIC_PREFIX "producer.request.latency.max",
+ "The maximum request latency in ms for produce requests.",
+ RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE, rd_false, rd_false,
+ unit_test_telemetry_set_produce_latency, default_expected_value_int,
+ default_expected_value_double);
+
+ /* Consumer metrics */
+ fails += unit_test_telemetry(
+ RD_KAFKA_CONSUMER,
+ RD_KAFKA_TELEMETRY_METRIC_CONSUMER_CONNECTION_CREATION_RATE,
+ RD_KAFKA_TELEMETRY_METRIC_PREFIX
+ "consumer.connection.creation.rate",
+ "The rate of connections established per second.",
+ RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE, rd_true, rd_false,
+ unit_test_telemetry_set_connects, default_expected_value_int,
+ default_expected_value_double);
+ fails += unit_test_telemetry(
+ RD_KAFKA_CONSUMER,
+ RD_KAFKA_TELEMETRY_METRIC_CONSUMER_NODE_REQUEST_LATENCY_AVG,
+ RD_KAFKA_TELEMETRY_METRIC_PREFIX
+ "consumer.node.request.latency.avg",
+ "The average request latency in ms for a node.",
+ RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE, rd_true, rd_true,
+ unit_test_telemetry_set_rtt, default_expected_value_int,
+ default_expected_value_double);
+ fails += unit_test_telemetry(
+ RD_KAFKA_CONSUMER,
+ RD_KAFKA_TELEMETRY_METRIC_CONSUMER_NODE_REQUEST_LATENCY_MAX,
+ RD_KAFKA_TELEMETRY_METRIC_PREFIX
+ "consumer.node.request.latency.max",
+ "The maximum request latency in ms for a node.",
+ RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE, rd_false, rd_true,
+ unit_test_telemetry_set_rtt, default_expected_value_int,
+ default_expected_value_double);
+ fails += unit_test_telemetry(
+ RD_KAFKA_CONSUMER,
+ RD_KAFKA_TELEMETRY_METRIC_CONSUMER_COORDINATOR_ASSIGNED_PARTITIONS,
+ RD_KAFKA_TELEMETRY_METRIC_PREFIX
+ "consumer.coordinator.assigned.partitions",
+ "The number of partitions currently assigned to this consumer.",
+ RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE, rd_false, rd_false,
+ unit_test_telemetry_set_coordinator_assigned_partitions,
+ default_expected_value_int, default_expected_value_double);
+ fails += unit_test_telemetry(
+ RD_KAFKA_CONSUMER,
+ RD_KAFKA_TELEMETRY_METRIC_CONSUMER_COORDINATOR_REBALANCE_LATENCY_AVG,
+ RD_KAFKA_TELEMETRY_METRIC_PREFIX
+ "consumer.coordinator.rebalance.latency.avg",
+ "The average rebalance latency in ms for the "
+ "consumer coordinator.",
+ RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE, rd_true, rd_false,
+ unit_test_telemetry_set_rebalance_latency,
+ default_expected_value_int, default_expected_value_double);
+ fails += unit_test_telemetry(
+ RD_KAFKA_CONSUMER,
+ RD_KAFKA_TELEMETRY_METRIC_CONSUMER_COORDINATOR_REBALANCE_LATENCY_MAX,
+ RD_KAFKA_TELEMETRY_METRIC_PREFIX
+ "consumer.coordinator.rebalance.latency.max",
+ "The maximum rebalance latency in ms for the "
+ "consumer coordinator.",
+ RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE, rd_false, rd_false,
+ unit_test_telemetry_set_rebalance_latency,
+ default_expected_value_int, default_expected_value_double);
+ fails += unit_test_telemetry(
+ RD_KAFKA_CONSUMER,
+ RD_KAFKA_TELEMETRY_METRIC_CONSUMER_FETCH_MANAGER_FETCH_LATENCY_AVG,
+ RD_KAFKA_TELEMETRY_METRIC_PREFIX
+ "consumer.fetch.manager.fetch.latency.avg",
+ "The average fetch latency in ms for the fetch manager.",
+ RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE, rd_true, rd_false,
+ unit_test_telemetry_set_fetch_latency, default_expected_value_int,
+ default_expected_value_double);
+ fails += unit_test_telemetry(
+ RD_KAFKA_CONSUMER,
+ RD_KAFKA_TELEMETRY_METRIC_CONSUMER_FETCH_MANAGER_FETCH_LATENCY_MAX,
+ RD_KAFKA_TELEMETRY_METRIC_PREFIX
+ "consumer.fetch.manager.fetch.latency.max",
+ "The maximum fetch latency in ms for the fetch manager.",
+ RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE, rd_false, rd_false,
+ unit_test_telemetry_set_fetch_latency, default_expected_value_int,
+ default_expected_value_double);
+ fails += unit_test_telemetry(
+ RD_KAFKA_CONSUMER,
+ RD_KAFKA_TELEMETRY_METRIC_CONSUMER_POLL_IDLE_RATIO_AVG,
+ RD_KAFKA_TELEMETRY_METRIC_PREFIX "consumer.poll.idle.ratio.avg",
+ "The average ratio of idle to poll for a consumer.",
+ RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE, rd_true, rd_false,
+ unit_test_telemetry_set_poll_idle_ratio, default_expected_value_int,
+ default_expected_value_double);
+ fails += unit_test_telemetry(
+ RD_KAFKA_CONSUMER,
+ RD_KAFKA_TELEMETRY_METRIC_CONSUMER_COORDINATOR_COMMIT_LATENCY_AVG,
+ RD_KAFKA_TELEMETRY_METRIC_PREFIX
+ "consumer.coordinator.commit.latency.avg",
+ "The average commit latency in ms for the consumer coordinator.",
+ RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE, rd_true, rd_false,
+ unit_test_telemetry_set_commit_latency, default_expected_value_int,
+ default_expected_value_double);
+ fails += unit_test_telemetry(
+ RD_KAFKA_CONSUMER,
+ RD_KAFKA_TELEMETRY_METRIC_CONSUMER_COORDINATOR_COMMIT_LATENCY_MAX,
+ RD_KAFKA_TELEMETRY_METRIC_PREFIX
+ "consumer.coordinator.commit.latency.max",
+ "The maximum commit latency in ms for the consumer coordinator.",
+ RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE, rd_false, rd_false,
+ unit_test_telemetry_set_commit_latency, default_expected_value_int,
+ default_expected_value_double);
+ return fails;
}
-bool unit_test_telemetry_sum(void) {
- return unit_test_telemetry(
+int unit_test_telemetry_sum(void) {
+ int fails = 0;
+ int64_t default_expected_value_int = 1;
+ double default_expected_value_double = 1.0;
+
+ /* Producer metrics */
+ fails += unit_test_telemetry(
+ RD_KAFKA_PRODUCER,
RD_KAFKA_TELEMETRY_METRIC_PRODUCER_CONNECTION_CREATION_TOTAL,
RD_KAFKA_TELEMETRY_METRIC_PREFIX
"producer.connection.creation.total",
"The total number of connections established.",
- RD_KAFKA_TELEMETRY_METRIC_TYPE_SUM, rd_false);
+ RD_KAFKA_TELEMETRY_METRIC_TYPE_SUM, rd_false, rd_false,
+ unit_test_telemetry_set_connects, default_expected_value_int,
+ default_expected_value_double);
+
+ /* Consumer metrics */
+ fails += unit_test_telemetry(
+ RD_KAFKA_CONSUMER,
+ RD_KAFKA_TELEMETRY_METRIC_CONSUMER_CONNECTION_CREATION_TOTAL,
+ RD_KAFKA_TELEMETRY_METRIC_PREFIX
+ "consumer.connection.creation.total",
+ "The total number of connections established.",
+ RD_KAFKA_TELEMETRY_METRIC_TYPE_SUM, rd_false, rd_false,
+ unit_test_telemetry_set_connects, default_expected_value_int,
+ default_expected_value_double);
+ /* Test with expected value 2 */
+ fails += unit_test_telemetry(
+ RD_KAFKA_CONSUMER,
+ RD_KAFKA_TELEMETRY_METRIC_CONSUMER_CONNECTION_CREATION_TOTAL,
+ RD_KAFKA_TELEMETRY_METRIC_PREFIX
+ "consumer.connection.creation.total",
+ "The total number of connections established.",
+ RD_KAFKA_TELEMETRY_METRIC_TYPE_SUM, rd_false, rd_false,
+ unit_test_telemetry_set_connects2, 2, 0.0);
+
+ fails += unit_test_telemetry(
+ RD_KAFKA_CONSUMER,
+ RD_KAFKA_TELEMETRY_METRIC_CONSUMER_COORDINATOR_REBALANCE_LATENCY_TOTAL,
+ RD_KAFKA_TELEMETRY_METRIC_PREFIX
+ "consumer.coordinator.rebalance.latency.total",
+ "The total rebalance latency in ms for the "
+ "consumer coordinator.",
+ RD_KAFKA_TELEMETRY_METRIC_TYPE_SUM, rd_false, rd_false,
+ unit_test_telemetry_set_rebalance_latency,
+ default_expected_value_int, default_expected_value_double);
+ return fails;
}
int unittest_telemetry_decode(void) {
diff --git a/src/rdkafka_telemetry_encode.c b/src/rdkafka_telemetry_encode.c
index 5e5a5a3dc1..da1a341d8a 100644
--- a/src/rdkafka_telemetry_encode.c
+++ b/src/rdkafka_telemetry_encode.c
@@ -42,6 +42,48 @@ typedef struct {
size_t count;
} rd_kafka_telemetry_key_values_repeated_t;
+#define calculate_avg(_avg_, _scale_factor_) \
+ ((_avg_).ra_v.avg / (double)_scale_factor_)
+
+#define calculate_max(_avg_, _scale_factor_) \
+ RD_CEIL_INTEGER_DIVISION((_avg_).ra_v.maxv, _scale_factor_)
+
+#define brokers_avg(_rk_, _avg_name_, _scale_factor_, _metric_) \
+ do { \
+ rd_kafka_broker_t *_rkb_; \
+ double avg = 0; \
+ int count = 0; \
+ TAILQ_FOREACH(_rkb_, &(_rk_)->rk_brokers, rkb_link) { \
+ rd_avg_t *rd_avg_rollover = \
+ &_rkb_->rkb_telemetry.rd_avg_rollover._avg_name_; \
+ if (rd_avg_rollover->ra_v.cnt) { \
+ avg = (avg * count + \
+ rd_avg_rollover->ra_v.sum) / \
+ (double)(count + \
+ rd_avg_rollover->ra_v.cnt); \
+ count += rd_avg_rollover->ra_v.cnt; \
+ } \
+ } \
+ if (_scale_factor_ > 1) \
+ (_metric_).double_value = avg / _scale_factor_; \
+ else \
+ (_metric_).double_value = avg; \
+ } while (0)
+
+#define brokers_max(_rk_, _avg_name_, _scale_factor_, _metric_) \
+ do { \
+ rd_kafka_broker_t *_rkb_; \
+ _metric_.int_value = 0; \
+ TAILQ_FOREACH(_rkb_, &(_rk_)->rk_brokers, rkb_link) { \
+ _metric_.int_value = \
+ RD_MAX(_metric_.int_value, \
+ _rkb_->rkb_telemetry.rd_avg_rollover \
+ ._avg_name_.ra_v.maxv); \
+ } \
+ if (_scale_factor_ > 1) \
+ (_metric_).int_value = RD_CEIL_INTEGER_DIVISION( \
+ (_metric_).int_value, _scale_factor_); \
+ } while (0)
static rd_kafka_telemetry_metric_value_t
calculate_connection_creation_total(rd_kafka_t *rk,
@@ -89,16 +131,9 @@ calculate_broker_avg_rtt(rd_kafka_t *rk,
rd_kafka_broker_t *rkb_selected,
rd_ts_t now_ns) {
rd_kafka_telemetry_metric_value_t avg_rtt = RD_ZERO_INIT;
-
- rd_avg_t *rkb_avg_rtt_rollover =
- &rkb_selected->rkb_telemetry.rd_avg_rollover.rkb_avg_rtt;
-
- if (rkb_avg_rtt_rollover->ra_v.cnt) {
- avg_rtt.double_value = rkb_avg_rtt_rollover->ra_v.sum /
- (double)(rkb_avg_rtt_rollover->ra_v.cnt *
- THREE_ORDERS_MAGNITUDE);
- }
-
+ avg_rtt.double_value = calculate_avg(
+ rkb_selected->rkb_telemetry.rd_avg_rollover.rkb_avg_rtt,
+ THREE_ORDERS_MAGNITUDE);
return avg_rtt;
}
@@ -106,35 +141,39 @@ static rd_kafka_telemetry_metric_value_t
calculate_broker_max_rtt(rd_kafka_t *rk,
rd_kafka_broker_t *rkb_selected,
rd_ts_t now_ns) {
- rd_kafka_telemetry_metric_value_t max_rtt;
-
- max_rtt.int_value = RD_CEIL_INTEGER_DIVISION(
- rkb_selected->rkb_telemetry.rd_avg_rollover.rkb_avg_rtt.ra_v.maxv,
+ rd_kafka_telemetry_metric_value_t max_rtt = RD_ZERO_INIT;
+ max_rtt.int_value = calculate_max(
+ rkb_selected->rkb_telemetry.rd_avg_rollover.rkb_avg_rtt,
THREE_ORDERS_MAGNITUDE);
return max_rtt;
}
+static rd_kafka_telemetry_metric_value_t
+calculate_produce_latency_avg(rd_kafka_t *rk,
+ rd_kafka_broker_t *rkb_selected,
+ rd_ts_t now_ns) {
+ rd_kafka_telemetry_metric_value_t avg_rtt = RD_ZERO_INIT;
+ brokers_avg(rk, rkb_avg_produce_latency, THREE_ORDERS_MAGNITUDE,
+ avg_rtt);
+ return avg_rtt;
+}
+
+static rd_kafka_telemetry_metric_value_t
+calculate_produce_latency_max(rd_kafka_t *rk,
+ rd_kafka_broker_t *rkb_selected,
+ rd_ts_t now_ns) {
+ rd_kafka_telemetry_metric_value_t max_rtt = RD_ZERO_INIT;
+ brokers_max(rk, rkb_avg_produce_latency, THREE_ORDERS_MAGNITUDE,
+ max_rtt);
+ return max_rtt;
+}
+
static rd_kafka_telemetry_metric_value_t
calculate_throttle_avg(rd_kafka_t *rk,
rd_kafka_broker_t *rkb_selected,
rd_ts_t now_ns) {
rd_kafka_telemetry_metric_value_t avg_throttle;
- rd_kafka_broker_t *rkb;
- double avg = 0;
- int count = 0;
-
- TAILQ_FOREACH(rkb, &rk->rk_brokers, rkb_link) {
- rd_avg_t *rkb_avg_throttle_rollover =
- &rkb->rkb_telemetry.rd_avg_rollover.rkb_avg_throttle;
- if (rkb_avg_throttle_rollover->ra_v.cnt) {
- avg = (avg * count +
- rkb_avg_throttle_rollover->ra_v.sum) /
- (double)(count +
- rkb_avg_throttle_rollover->ra_v.cnt);
- count += rkb_avg_throttle_rollover->ra_v.cnt;
- }
- }
- avg_throttle.double_value = avg;
+ brokers_avg(rk, rkb_avg_throttle, 1, avg_throttle);
return avg_throttle;
}
@@ -144,14 +183,7 @@ calculate_throttle_max(rd_kafka_t *rk,
rd_kafka_broker_t *rkb_selected,
rd_ts_t now_ns) {
rd_kafka_telemetry_metric_value_t max_throttle;
- rd_kafka_broker_t *rkb;
-
- max_throttle.int_value = 0;
- TAILQ_FOREACH(rkb, &rk->rk_brokers, rkb_link) {
- max_throttle.int_value = RD_MAX(
- max_throttle.int_value, rkb->rkb_telemetry.rd_avg_rollover
- .rkb_avg_throttle.ra_v.maxv);
- }
+ brokers_max(rk, rkb_avg_throttle, 1, max_throttle);
return max_throttle;
}
@@ -160,24 +192,8 @@ calculate_queue_time_avg(rd_kafka_t *rk,
rd_kafka_broker_t *rkb_selected,
rd_ts_t now_ns) {
rd_kafka_telemetry_metric_value_t avg_queue_time;
- rd_kafka_broker_t *rkb;
- double avg = 0;
- int count = 0;
-
- TAILQ_FOREACH(rkb, &rk->rk_brokers, rkb_link) {
- rd_avg_t *rkb_avg_outbuf_latency_rollover =
- &rkb->rkb_telemetry.rd_avg_rollover.rkb_avg_outbuf_latency;
- if (rkb_avg_outbuf_latency_rollover->ra_v.cnt) {
- avg =
- (avg * count +
- rkb_avg_outbuf_latency_rollover->ra_v.sum) /
- (double)(count +
- rkb_avg_outbuf_latency_rollover->ra_v.cnt);
- count += rkb_avg_outbuf_latency_rollover->ra_v.cnt;
- }
- }
-
- avg_queue_time.double_value = avg / THREE_ORDERS_MAGNITUDE;
+ brokers_avg(rk, rkb_avg_outbuf_latency, THREE_ORDERS_MAGNITUDE,
+ avg_queue_time);
return avg_queue_time;
}
@@ -186,17 +202,8 @@ calculate_queue_time_max(rd_kafka_t *rk,
rd_kafka_broker_t *rkb_selected,
rd_ts_t now_ns) {
rd_kafka_telemetry_metric_value_t max_queue_time;
- rd_kafka_broker_t *rkb;
-
- max_queue_time.int_value = 0;
- TAILQ_FOREACH(rkb, &rk->rk_brokers, rkb_link) {
- max_queue_time.int_value =
- RD_MAX(max_queue_time.int_value,
- rkb->rkb_telemetry.rd_avg_rollover
- .rkb_avg_outbuf_latency.ra_v.maxv);
- }
- max_queue_time.int_value = RD_CEIL_INTEGER_DIVISION(
- max_queue_time.int_value, THREE_ORDERS_MAGNITUDE);
+ brokers_max(rk, rkb_avg_outbuf_latency, THREE_ORDERS_MAGNITUDE,
+ max_queue_time);
return max_queue_time;
}
@@ -211,11 +218,104 @@ calculate_consumer_assigned_partitions(rd_kafka_t *rk,
return assigned_partitions;
}
+static rd_kafka_telemetry_metric_value_t
+calculate_consumer_rebalance_latency_avg(rd_kafka_t *rk,
+ rd_kafka_broker_t *rkb_selected,
+ rd_ts_t now_ns) {
+ rd_kafka_telemetry_metric_value_t avg_rebalance_time;
+ avg_rebalance_time.double_value = calculate_avg(
+ rk->rk_telemetry.rd_avg_rollover.rk_avg_rebalance_latency,
+ THREE_ORDERS_MAGNITUDE);
+ return avg_rebalance_time;
+}
+
+static rd_kafka_telemetry_metric_value_t
+calculate_consumer_rebalance_latency_max(rd_kafka_t *rk,
+ rd_kafka_broker_t *rkb_selected,
+ rd_ts_t now_ns) {
+ rd_kafka_telemetry_metric_value_t max_rebalance_time;
+ max_rebalance_time.int_value = calculate_max(
+ rk->rk_telemetry.rd_avg_rollover.rk_avg_rebalance_latency,
+ THREE_ORDERS_MAGNITUDE);
+ return max_rebalance_time;
+}
+
+static rd_kafka_telemetry_metric_value_t
+calculate_consumer_rebalance_latency_total(rd_kafka_t *rk,
+ rd_kafka_broker_t *rkb_selected,
+ rd_ts_t now_ns) {
+ rd_kafka_telemetry_metric_value_t total_rebalance_time;
+ total_rebalance_time.int_value = RD_CEIL_INTEGER_DIVISION(
+ rk->rk_telemetry.rd_avg_rollover.rk_avg_rebalance_latency.ra_v.sum,
+ THREE_ORDERS_MAGNITUDE);
+ if (!rk->rk_telemetry.delta_temporality) {
+ total_rebalance_time.int_value +=
+ rk->rk_telemetry.rk_historic_c.rebalance_latency_total;
+ }
+ return total_rebalance_time;
+}
+
+static rd_kafka_telemetry_metric_value_t
+calculate_consumer_fetch_latency_avg(rd_kafka_t *rk,
+ rd_kafka_broker_t *rkb_selected,
+ rd_ts_t now_ns) {
+ rd_kafka_telemetry_metric_value_t avg_fetch_time;
+ brokers_avg(rk, rkb_avg_fetch_latency, THREE_ORDERS_MAGNITUDE,
+ avg_fetch_time);
+ return avg_fetch_time;
+}
+
+static rd_kafka_telemetry_metric_value_t
+calculate_consumer_fetch_latency_max(rd_kafka_t *rk,
+ rd_kafka_broker_t *rkb_selected,
+ rd_ts_t now_ns) {
+ rd_kafka_telemetry_metric_value_t max_fetch_time;
+ brokers_max(rk, rkb_avg_fetch_latency, THREE_ORDERS_MAGNITUDE,
+ max_fetch_time);
+ return max_fetch_time;
+}
+
+static rd_kafka_telemetry_metric_value_t
+calculate_consumer_poll_idle_ratio_avg(rd_kafka_t *rk,
+ rd_kafka_broker_t *rkb_selected,
+ rd_ts_t now_ns) {
+ rd_kafka_telemetry_metric_value_t avg_poll_idle_avg;
+ avg_poll_idle_avg.double_value = calculate_avg(
+ rk->rk_telemetry.rd_avg_rollover.rk_avg_poll_idle_ratio, 1e6);
+ return avg_poll_idle_avg;
+}
+
+static rd_kafka_telemetry_metric_value_t
+calculate_consumer_commit_latency_avg(rd_kafka_t *rk,
+ rd_kafka_broker_t *rkb_selected,
+ rd_ts_t now_ns) {
+ rd_kafka_telemetry_metric_value_t avg_commit_time;
+ avg_commit_time.double_value = calculate_avg(
+ rk->rk_telemetry.rd_avg_rollover.rk_avg_commit_latency,
+ THREE_ORDERS_MAGNITUDE);
+ return avg_commit_time;
+}
+
+static rd_kafka_telemetry_metric_value_t
+calculate_consumer_commit_latency_max(rd_kafka_t *rk,
+ rd_kafka_broker_t *rkb_selected,
+ rd_ts_t now_ns) {
+ rd_kafka_telemetry_metric_value_t max_commit_time;
+ max_commit_time.int_value = calculate_max(
+ rk->rk_telemetry.rd_avg_rollover.rk_avg_commit_latency,
+ THREE_ORDERS_MAGNITUDE);
+ return max_commit_time;
+}
static void reset_historical_metrics(rd_kafka_t *rk, rd_ts_t now_ns) {
rd_kafka_broker_t *rkb;
rk->rk_telemetry.rk_historic_c.ts_last = now_ns;
+ rk->rk_telemetry.rk_historic_c.rebalance_latency_total +=
+ RD_CEIL_INTEGER_DIVISION(rk->rk_telemetry.rd_avg_rollover
+ .rk_avg_rebalance_latency.ra_v.sum,
+ THREE_ORDERS_MAGNITUDE);
+
TAILQ_FOREACH(rkb, &rk->rk_brokers, rkb_link) {
rkb->rkb_telemetry.rkb_historic_c.connects =
rd_atomic32_get(&rkb->rkb_c.connects);
@@ -241,6 +341,10 @@ static const rd_kafka_telemetry_metric_value_calculator_t
&calculate_queue_time_avg,
[RD_KAFKA_TELEMETRY_METRIC_PRODUCER_RECORD_QUEUE_TIME_MAX] =
&calculate_queue_time_max,
+ [RD_KAFKA_TELEMETRY_METRIC_PRODUCER_PRODUCE_LATENCY_AVG] =
+ &calculate_produce_latency_avg,
+ [RD_KAFKA_TELEMETRY_METRIC_PRODUCER_PRODUCE_LATENCY_MAX] =
+ &calculate_produce_latency_max,
};
static const rd_kafka_telemetry_metric_value_calculator_t
@@ -255,6 +359,22 @@ static const rd_kafka_telemetry_metric_value_calculator_t
&calculate_broker_max_rtt,
[RD_KAFKA_TELEMETRY_METRIC_CONSUMER_COORDINATOR_ASSIGNED_PARTITIONS] =
&calculate_consumer_assigned_partitions,
+ [RD_KAFKA_TELEMETRY_METRIC_CONSUMER_COORDINATOR_REBALANCE_LATENCY_AVG] =
+ &calculate_consumer_rebalance_latency_avg,
+ [RD_KAFKA_TELEMETRY_METRIC_CONSUMER_COORDINATOR_REBALANCE_LATENCY_MAX] =
+ &calculate_consumer_rebalance_latency_max,
+ [RD_KAFKA_TELEMETRY_METRIC_CONSUMER_COORDINATOR_REBALANCE_LATENCY_TOTAL] =
+ &calculate_consumer_rebalance_latency_total,
+ [RD_KAFKA_TELEMETRY_METRIC_CONSUMER_FETCH_MANAGER_FETCH_LATENCY_AVG] =
+ &calculate_consumer_fetch_latency_avg,
+ [RD_KAFKA_TELEMETRY_METRIC_CONSUMER_FETCH_MANAGER_FETCH_LATENCY_MAX] =
+ &calculate_consumer_fetch_latency_max,
+ [RD_KAFKA_TELEMETRY_METRIC_CONSUMER_POLL_IDLE_RATIO_AVG] =
+ &calculate_consumer_poll_idle_ratio_avg,
+ [RD_KAFKA_TELEMETRY_METRIC_CONSUMER_COORDINATOR_COMMIT_LATENCY_AVG] =
+ &calculate_consumer_commit_latency_avg,
+ [RD_KAFKA_TELEMETRY_METRIC_CONSUMER_COORDINATOR_COMMIT_LATENCY_MAX] =
+ &calculate_consumer_commit_latency_max,
};
static const char *get_client_rack(const rd_kafka_t *rk) {
@@ -593,6 +713,10 @@ static void serialize_Metric(
* @brief Encodes the metrics to opentelemetry_proto_metrics_v1_MetricsData and
* returns the serialized data. Currently only supports encoding of connection
* creation total by default
+ *
+ * @locks none
+ * @locks_acquired rd_kafka_rdlock()
+ * @locality main thread
*/
rd_buf_t *rd_kafka_telemetry_encode_metrics(rd_kafka_t *rk) {
rd_buf_t *rbuf = NULL;
@@ -609,6 +733,10 @@ rd_buf_t *rd_kafka_telemetry_encode_metrics(rd_kafka_t *rk) {
RD_KAFKA_TELEMETRY_METRIC_INFO(rk);
size_t total_metrics_count = metrics_to_encode_count;
size_t i, metric_idx = 0;
+
+ if (!metrics_to_encode_count)
+ return rd_buf_new(1, 1);
+
opentelemetry_proto_metrics_v1_MetricsData metrics_data =
opentelemetry_proto_metrics_v1_MetricsData_init_zero;
@@ -635,6 +763,7 @@ rd_buf_t *rd_kafka_telemetry_encode_metrics(rd_kafka_t *rk) {
}
rd_kafka_dbg(rk, TELEMETRY, "PUSH", "Serializing metrics");
+
TAILQ_FOREACH(rkb, &rk->rk_brokers, rkb_link) {
rd_avg_destroy(&rkb->rkb_telemetry.rd_avg_rollover.rkb_avg_rtt);
rd_avg_rollover(&rkb->rkb_telemetry.rd_avg_rollover.rkb_avg_rtt,
@@ -649,6 +778,41 @@ rd_buf_t *rd_kafka_telemetry_encode_metrics(rd_kafka_t *rk) {
rd_avg_rollover(
&rkb->rkb_telemetry.rd_avg_rollover.rkb_avg_throttle,
&rkb->rkb_telemetry.rd_avg_current.rkb_avg_throttle);
+ if (rk->rk_type == RD_KAFKA_CONSUMER) {
+ rd_avg_destroy(&rkb->rkb_telemetry.rd_avg_rollover
+ .rkb_avg_fetch_latency);
+ rd_avg_rollover(&rkb->rkb_telemetry.rd_avg_rollover
+ .rkb_avg_fetch_latency,
+ &rkb->rkb_telemetry.rd_avg_current
+ .rkb_avg_fetch_latency);
+ } else if (rk->rk_type == RD_KAFKA_PRODUCER) {
+ rd_avg_destroy(&rkb->rkb_telemetry.rd_avg_rollover
+ .rkb_avg_produce_latency);
+ rd_avg_rollover(&rkb->rkb_telemetry.rd_avg_rollover
+ .rkb_avg_produce_latency,
+ &rkb->rkb_telemetry.rd_avg_current
+ .rkb_avg_produce_latency);
+ }
+ }
+
+ if (rk->rk_type == RD_KAFKA_CONSUMER) {
+ rd_avg_destroy(
+ &rk->rk_telemetry.rd_avg_rollover.rk_avg_poll_idle_ratio);
+ rd_avg_rollover(
+ &rk->rk_telemetry.rd_avg_rollover.rk_avg_poll_idle_ratio,
+ &rk->rk_telemetry.rd_avg_current.rk_avg_poll_idle_ratio);
+
+ rd_avg_destroy(
+ &rk->rk_telemetry.rd_avg_rollover.rk_avg_rebalance_latency);
+ rd_avg_rollover(
+ &rk->rk_telemetry.rd_avg_rollover.rk_avg_rebalance_latency,
+ &rk->rk_telemetry.rd_avg_current.rk_avg_rebalance_latency);
+
+ rd_avg_destroy(
+ &rk->rk_telemetry.rd_avg_rollover.rk_avg_commit_latency);
+ rd_avg_rollover(
+ &rk->rk_telemetry.rd_avg_rollover.rk_avg_commit_latency,
+ &rk->rk_telemetry.rd_avg_current.rk_avg_commit_latency);
}
int resource_attributes_count =
diff --git a/src/rdkafka_telemetry_encode.h b/src/rdkafka_telemetry_encode.h
index 44445ea2bb..75c6bc896b 100644
--- a/src/rdkafka_telemetry_encode.h
+++ b/src/rdkafka_telemetry_encode.h
@@ -60,6 +60,8 @@ typedef enum {
RD_KAFKA_TELEMETRY_METRIC_PRODUCER_PRODUCE_THROTTLE_TIME_MAX,
RD_KAFKA_TELEMETRY_METRIC_PRODUCER_RECORD_QUEUE_TIME_AVG,
RD_KAFKA_TELEMETRY_METRIC_PRODUCER_RECORD_QUEUE_TIME_MAX,
+ RD_KAFKA_TELEMETRY_METRIC_PRODUCER_PRODUCE_LATENCY_AVG,
+ RD_KAFKA_TELEMETRY_METRIC_PRODUCER_PRODUCE_LATENCY_MAX,
RD_KAFKA_TELEMETRY_PRODUCER_METRIC__CNT
} rd_kafka_telemetry_producer_metric_name_t;
@@ -69,6 +71,14 @@ typedef enum {
RD_KAFKA_TELEMETRY_METRIC_CONSUMER_NODE_REQUEST_LATENCY_AVG,
RD_KAFKA_TELEMETRY_METRIC_CONSUMER_NODE_REQUEST_LATENCY_MAX,
RD_KAFKA_TELEMETRY_METRIC_CONSUMER_COORDINATOR_ASSIGNED_PARTITIONS,
+ RD_KAFKA_TELEMETRY_METRIC_CONSUMER_COORDINATOR_REBALANCE_LATENCY_AVG,
+ RD_KAFKA_TELEMETRY_METRIC_CONSUMER_COORDINATOR_REBALANCE_LATENCY_MAX,
+ RD_KAFKA_TELEMETRY_METRIC_CONSUMER_COORDINATOR_REBALANCE_LATENCY_TOTAL,
+ RD_KAFKA_TELEMETRY_METRIC_CONSUMER_FETCH_MANAGER_FETCH_LATENCY_AVG,
+ RD_KAFKA_TELEMETRY_METRIC_CONSUMER_FETCH_MANAGER_FETCH_LATENCY_MAX,
+ RD_KAFKA_TELEMETRY_METRIC_CONSUMER_POLL_IDLE_RATIO_AVG,
+ RD_KAFKA_TELEMETRY_METRIC_CONSUMER_COORDINATOR_COMMIT_LATENCY_AVG,
+ RD_KAFKA_TELEMETRY_METRIC_CONSUMER_COORDINATOR_COMMIT_LATENCY_MAX,
RD_KAFKA_TELEMETRY_CONSUMER_METRIC__CNT
} rd_kafka_telemetry_consumer_metric_name_t;
@@ -153,62 +163,139 @@ static const rd_kafka_telemetry_metric_info_t
{.name = "producer.record.queue.time.avg",
.description = "The average time in ms a record spends in the "
"producer queue.",
- .unit = "ms",
- .is_int = rd_false,
+ .unit = "ms",
+ .is_int = rd_false,
.is_per_broker = rd_false,
.type = RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE},
[RD_KAFKA_TELEMETRY_METRIC_PRODUCER_RECORD_QUEUE_TIME_MAX] =
{.name = "producer.record.queue.time.max",
.description = "The maximum time in ms a record spends in the "
"producer queue.",
- .unit = "ms",
- .is_int = rd_true,
+ .unit = "ms",
+ .is_int = rd_true,
.is_per_broker = rd_false,
.type = RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE},
-};
-
-static const rd_kafka_telemetry_metric_info_t
- RD_KAFKA_TELEMETRY_CONSUMER_METRICS_INFO
- [RD_KAFKA_TELEMETRY_CONSUMER_METRIC__CNT] = {
- [RD_KAFKA_TELEMETRY_METRIC_CONSUMER_CONNECTION_CREATION_RATE] =
- {.name = "consumer.connection.creation.rate",
+ [RD_KAFKA_TELEMETRY_METRIC_PRODUCER_PRODUCE_LATENCY_AVG] =
+ {.name = "producer.request.latency.avg",
.description =
- "The rate of connections established per second.",
- .unit = "1",
+ "The average request latency in ms for produce requests.",
+ .unit = "ms",
.is_int = rd_false,
.is_per_broker = rd_false,
.type = RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE},
- [RD_KAFKA_TELEMETRY_METRIC_CONSUMER_CONNECTION_CREATION_TOTAL] =
- {.name = "consumer.connection.creation.total",
- .description = "The total number of connections established.",
- .unit = "1",
- .is_int = rd_true,
- .is_per_broker = rd_false,
- .type = RD_KAFKA_TELEMETRY_METRIC_TYPE_SUM},
- [RD_KAFKA_TELEMETRY_METRIC_CONSUMER_NODE_REQUEST_LATENCY_AVG] =
- {.name = "consumer.node.request.latency.avg",
- .description = "The average request latency in ms for a node.",
- .unit = "ms",
- .is_int = rd_false,
- .is_per_broker = rd_true,
- .type = RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE},
- [RD_KAFKA_TELEMETRY_METRIC_CONSUMER_NODE_REQUEST_LATENCY_MAX] =
- {.name = "consumer.node.request.latency.max",
- .description = "The maximum request latency in ms for a node.",
- .unit = "ms",
- .is_int = rd_true,
- .is_per_broker = rd_true,
- .type = RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE},
- [RD_KAFKA_TELEMETRY_METRIC_CONSUMER_COORDINATOR_ASSIGNED_PARTITIONS] =
- {.name = "consumer.coordinator.assigned.partitions",
- .description = "The number of partitions currently assigned "
- "to this consumer.",
- .unit = "1",
+ [RD_KAFKA_TELEMETRY_METRIC_PRODUCER_PRODUCE_LATENCY_MAX] =
+ {.name = "producer.request.latency.max",
+ .description =
+ "The maximum request latency in ms for produce requests.",
+ .unit = "ms",
.is_int = rd_true,
.is_per_broker = rd_false,
.type = RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE},
};
+static const rd_kafka_telemetry_metric_info_t RD_KAFKA_TELEMETRY_CONSUMER_METRICS_INFO
+ [RD_KAFKA_TELEMETRY_CONSUMER_METRIC__CNT] = {
+ [RD_KAFKA_TELEMETRY_METRIC_CONSUMER_CONNECTION_CREATION_RATE] =
+ {.name = "consumer.connection.creation.rate",
+ .description = "The rate of connections established per second.",
+ .unit = "1",
+ .is_int = rd_false,
+ .is_per_broker = rd_false,
+ .type = RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE},
+ [RD_KAFKA_TELEMETRY_METRIC_CONSUMER_CONNECTION_CREATION_TOTAL] =
+ {.name = "consumer.connection.creation.total",
+ .description = "The total number of connections established.",
+ .unit = "1",
+ .is_int = rd_true,
+ .is_per_broker = rd_false,
+ .type = RD_KAFKA_TELEMETRY_METRIC_TYPE_SUM},
+ [RD_KAFKA_TELEMETRY_METRIC_CONSUMER_NODE_REQUEST_LATENCY_AVG] =
+ {.name = "consumer.node.request.latency.avg",
+ .description = "The average request latency in ms for a node.",
+ .unit = "ms",
+ .is_int = rd_false,
+ .is_per_broker = rd_true,
+ .type = RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE},
+ [RD_KAFKA_TELEMETRY_METRIC_CONSUMER_NODE_REQUEST_LATENCY_MAX] =
+ {.name = "consumer.node.request.latency.max",
+ .description = "The maximum request latency in ms for a node.",
+ .unit = "ms",
+ .is_int = rd_true,
+ .is_per_broker = rd_true,
+ .type = RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE},
+ [RD_KAFKA_TELEMETRY_METRIC_CONSUMER_COORDINATOR_ASSIGNED_PARTITIONS] =
+ {.name = "consumer.coordinator.assigned.partitions",
+ .description = "The number of partitions currently assigned "
+ "to this consumer.",
+ .unit = "1",
+ .is_int = rd_true,
+ .is_per_broker = rd_false,
+ .type = RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE},
+ [RD_KAFKA_TELEMETRY_METRIC_CONSUMER_COORDINATOR_REBALANCE_LATENCY_AVG] =
+ {.name = "consumer.coordinator.rebalance.latency.avg",
+ .description = "The average rebalance latency in ms for the "
+ "consumer coordinator.",
+ .unit = "ms",
+ .is_int = rd_false,
+ .is_per_broker = rd_false,
+ .type = RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE},
+ [RD_KAFKA_TELEMETRY_METRIC_CONSUMER_COORDINATOR_REBALANCE_LATENCY_MAX] =
+ {.name = "consumer.coordinator.rebalance.latency.max",
+ .description = "The maximum rebalance latency in ms for the "
+ "consumer coordinator.",
+ .unit = "ms",
+ .is_int = rd_true,
+ .is_per_broker = rd_false,
+ .type = RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE},
+ [RD_KAFKA_TELEMETRY_METRIC_CONSUMER_COORDINATOR_REBALANCE_LATENCY_TOTAL] =
+ {.name = "consumer.coordinator.rebalance.latency.total",
+ .description = "The total rebalance latency in ms for the "
+ "consumer coordinator.",
+ .unit = "ms",
+ .is_int = rd_true,
+ .is_per_broker = rd_false,
+ .type = RD_KAFKA_TELEMETRY_METRIC_TYPE_SUM},
+ [RD_KAFKA_TELEMETRY_METRIC_CONSUMER_FETCH_MANAGER_FETCH_LATENCY_AVG] =
+ {.name = "consumer.fetch.manager.fetch.latency.avg",
+ .description =
+ "The average fetch latency in ms for the fetch manager.",
+ .unit = "ms",
+ .is_int = rd_false,
+ .is_per_broker = rd_false,
+ .type = RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE},
+ [RD_KAFKA_TELEMETRY_METRIC_CONSUMER_FETCH_MANAGER_FETCH_LATENCY_MAX] =
+ {.name = "consumer.fetch.manager.fetch.latency.max",
+ .description =
+ "The maximum fetch latency in ms for the fetch manager.",
+ .unit = "ms",
+ .is_int = rd_true,
+ .is_per_broker = rd_false,
+ .type = RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE},
+ [RD_KAFKA_TELEMETRY_METRIC_CONSUMER_POLL_IDLE_RATIO_AVG] =
+ {.name = "consumer.poll.idle.ratio.avg",
+ .description = "The average ratio of idle to poll for a consumer.",
+ .unit = "1",
+ .is_int = rd_false,
+ .is_per_broker = rd_false,
+ .type = RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE},
+ [RD_KAFKA_TELEMETRY_METRIC_CONSUMER_COORDINATOR_COMMIT_LATENCY_AVG] =
+ {.name = "consumer.coordinator.commit.latency.avg",
+ .description = "The average commit latency in ms for the consumer "
+ "coordinator.",
+ .unit = "ms",
+ .is_int = rd_false,
+ .is_per_broker = rd_false,
+ .type = RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE},
+ [RD_KAFKA_TELEMETRY_METRIC_CONSUMER_COORDINATOR_COMMIT_LATENCY_MAX] =
+ {.name = "consumer.coordinator.commit.latency.max",
+ .description = "The maximum commit latency in ms for the consumer "
+ "coordinator.",
+ .unit = "ms",
+ .is_int = rd_true,
+ .is_per_broker = rd_false,
+ .type = RD_KAFKA_TELEMETRY_METRIC_TYPE_GAUGE},
+};
+
rd_buf_t *rd_kafka_telemetry_encode_metrics(rd_kafka_t *rk);
#endif /* _RDKAFKA_RDKAFKA_TELEMETRY_ENCODE_H */
diff --git a/src/rdkafka_topic.c b/src/rdkafka_topic.c
index fd3a175364..56f63930a8 100644
--- a/src/rdkafka_topic.c
+++ b/src/rdkafka_topic.c
@@ -354,6 +354,7 @@ rd_kafka_topic_t *rd_kafka_topic_new0(rd_kafka_t *rk,
rkt->rkt_rk = rk;
rkt->rkt_ts_create = rd_clock();
+ rkt->rkt_ts_state = rkt->rkt_ts_create;
rkt->rkt_conf = *conf;
rd_free(conf); /* explicitly not rd_kafka_topic_destroy()
@@ -538,7 +539,7 @@ rd_kafka_topic_t *rd_kafka_topic_new(rd_kafka_t *rk,
/* Query for the topic leader (async) */
if (!existing)
- rd_kafka_topic_leader_query(rk, rkt);
+ rd_kafka_topic_fast_leader_query(rk, rd_true /* force */);
/* Drop our reference since there is already/now an app refcnt */
rd_kafka_topic_destroy0(rkt);
@@ -565,7 +566,8 @@ static void rd_kafka_topic_set_state(rd_kafka_topic_t *rkt, int state) {
if (rkt->rkt_state == RD_KAFKA_TOPIC_S_ERROR)
rkt->rkt_err = RD_KAFKA_RESP_ERR_NO_ERROR;
- rkt->rkt_state = state;
+ rkt->rkt_state = state;
+ rkt->rkt_ts_state = rd_clock();
}
/**
@@ -643,6 +645,7 @@ int rd_kafka_toppar_broker_update(rd_kafka_toppar_t *rktp,
* @remark If a toppar is currently delegated to a preferred replica,
* it will not be delegated to the leader broker unless there
* has been a leader change.
+ * @remark The new leader, if present, should not be terminating.
*
* @param leader_id The id of the new leader broker.
* @param leader A reference to the leader broker or NULL if the
@@ -663,7 +666,8 @@ static int rd_kafka_toppar_leader_update(rd_kafka_topic_t *rkt,
int32_t leader_epoch) {
rd_kafka_toppar_t *rktp;
rd_bool_t need_epoch_validation = rd_false;
- int r = 0;
+ rd_bool_t fetching_from_follower;
+ int r = 0;
rktp = rd_kafka_toppar_get(rkt, partition, 0);
if (unlikely(!rktp)) {
@@ -681,7 +685,11 @@ static int rd_kafka_toppar_leader_update(rd_kafka_topic_t *rkt,
rd_kafka_toppar_lock(rktp);
- if (leader_epoch < rktp->rktp_leader_epoch) {
+ /* -1 (null) is excluded to allow to switch back to a
+ * leader not supporting KIP-320 still, for example
+ * during a cluster roll for upgrading brokers to
+ * a version supporting that KIP. */
+ if (leader_epoch != -1 && leader_epoch < rktp->rktp_leader_epoch) {
rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "BROKER",
"%s [%" PRId32
"]: ignoring outdated metadata update with "
@@ -691,81 +699,69 @@ static int rd_kafka_toppar_leader_update(rd_kafka_topic_t *rkt,
rktp->rktp_rkt->rkt_topic->str,
rktp->rktp_partition, leader_epoch,
rktp->rktp_leader_epoch);
- if (rktp->rktp_fetch_state !=
- RD_KAFKA_TOPPAR_FETCH_VALIDATE_EPOCH_WAIT) {
- rd_kafka_toppar_unlock(rktp);
- rd_kafka_toppar_destroy(rktp); /* from get() */
- return 0;
- }
+ rd_kafka_toppar_unlock(rktp);
+ rd_kafka_toppar_destroy(rktp); /* from get() */
+ return 0;
}
- if (rktp->rktp_leader_epoch == -1 ||
- leader_epoch > rktp->rktp_leader_epoch) {
- rd_bool_t fetching_from_follower;
- rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "BROKER",
- "%s [%" PRId32 "]: leader %" PRId32
- " epoch %" PRId32 " -> leader %" PRId32
- " epoch %" PRId32,
- rktp->rktp_rkt->rkt_topic->str,
- rktp->rktp_partition, rktp->rktp_leader_id,
- rktp->rktp_leader_epoch, leader_id, leader_epoch);
- if (leader_epoch > rktp->rktp_leader_epoch)
- rktp->rktp_leader_epoch = leader_epoch;
+ rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "BROKER",
+ "%s [%" PRId32 "]: leader %" PRId32 " epoch %" PRId32
+ " -> leader %" PRId32 " epoch %" PRId32,
+ rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition,
+ rktp->rktp_leader_id, rktp->rktp_leader_epoch, leader_id,
+ leader_epoch);
+
+ if (leader_epoch > rktp->rktp_leader_epoch ||
+ rktp->rktp_fetch_state ==
+ RD_KAFKA_TOPPAR_FETCH_VALIDATE_EPOCH_WAIT) {
+ /* Epoch increased and needs to be validated (leader_epoch > -1)
+ * or we need to complete the validation. */
need_epoch_validation = rd_true;
+ }
+ rktp->rktp_leader_epoch = leader_epoch;
- fetching_from_follower =
- leader != NULL && rktp->rktp_broker != NULL &&
- rktp->rktp_broker->rkb_source != RD_KAFKA_INTERNAL &&
- rktp->rktp_broker != leader;
-
- if (fetching_from_follower &&
- rktp->rktp_leader_id == leader_id) {
- rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "BROKER",
- "Topic %s [%" PRId32 "]: leader %" PRId32
- " unchanged, "
- "not migrating away from preferred "
- "replica %" PRId32,
- rktp->rktp_rkt->rkt_topic->str,
- rktp->rktp_partition, leader_id,
- rktp->rktp_broker_id);
- r = 0;
+ fetching_from_follower =
+ leader != NULL && rktp->rktp_broker != NULL &&
+ rktp->rktp_broker->rkb_source != RD_KAFKA_INTERNAL &&
+ rktp->rktp_broker != leader;
- } else {
+ if (fetching_from_follower && rktp->rktp_leader_id == leader_id) {
+ rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "BROKER",
+ "Topic %s [%" PRId32 "]: leader %" PRId32
+ " unchanged, "
+ "not migrating away from preferred "
+ "replica %" PRId32,
+ rktp->rktp_rkt->rkt_topic->str,
+ rktp->rktp_partition, leader_id,
+ rktp->rktp_broker_id);
+ r = 0;
- if (rktp->rktp_leader_id != leader_id ||
- rktp->rktp_leader != leader) {
- /* Update leader if it has changed */
- rktp->rktp_leader_id = leader_id;
- if (rktp->rktp_leader)
- rd_kafka_broker_destroy(
- rktp->rktp_leader);
- if (leader)
- rd_kafka_broker_keep(leader);
- rktp->rktp_leader = leader;
- }
+ } else {
- /* Update handling broker */
- r = rd_kafka_toppar_broker_update(
- rktp, leader_id, leader, "leader updated");
+ if (rktp->rktp_leader_id != leader_id ||
+ rktp->rktp_leader != leader) {
+ /* Update leader if it has changed */
+ rktp->rktp_leader_id = leader_id;
+ if (rktp->rktp_leader)
+ rd_kafka_broker_destroy(rktp->rktp_leader);
+ if (leader)
+ rd_kafka_broker_keep(leader);
+ rktp->rktp_leader = leader;
}
- } else if (rktp->rktp_fetch_state ==
- RD_KAFKA_TOPPAR_FETCH_VALIDATE_EPOCH_WAIT)
- need_epoch_validation = rd_true;
+ /* Update handling broker */
+ r = rd_kafka_toppar_broker_update(rktp, leader_id, leader,
+ "leader updated");
+ }
if (need_epoch_validation) {
/* Set offset validation position,
* depending it if should continue with current position or
* with next fetch start position. */
- if (rd_kafka_toppar_fetch_decide_start_from_next_fetch_start(
- rktp)) {
- rd_kafka_toppar_set_offset_validation_position(
- rktp, rktp->rktp_next_fetch_start);
- } else {
- rd_kafka_toppar_set_offset_validation_position(
- rktp, rktp->rktp_offsets.fetch_pos);
- }
+ rd_kafka_toppar_set_offset_validation_position(
+ rktp,
+ rd_kafka_toppar_fetch_decide_next_fetch_start_pos(rktp));
rd_kafka_offset_validate(rktp, "epoch updated from metadata");
}
@@ -794,7 +790,12 @@ int rd_kafka_toppar_delegate_to_leader(rd_kafka_toppar_t *rktp) {
rd_kafka_rdlock(rktp->rktp_rkt->rkt_rk);
rd_kafka_toppar_lock(rktp);
- rd_assert(rktp->rktp_leader_id != rktp->rktp_broker_id);
+ if (rktp->rktp_leader_id == rktp->rktp_broker_id) {
+ /* Given lock was released we need to check again */
+ rd_kafka_toppar_unlock(rktp);
+ rd_kafka_rdunlock(rktp->rktp_rkt->rkt_rk);
+ return 0;
+ }
rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "BROKER",
"Topic %s [%" PRId32
@@ -822,6 +823,45 @@ int rd_kafka_toppar_delegate_to_leader(rd_kafka_toppar_t *rktp) {
}
+/**
+ * @brief Forgets current rktp leader, to reduce reference count
+ * and allow the broker to be destroyed.
+ *
+ * @locks none
+ * @locks_acquired rk rdlock, rktp
+ * @locality any
+ */
+void rd_kafka_toppar_forget_leader(rd_kafka_toppar_t *rktp) {
+ rd_kafka_rdlock(rktp->rktp_rkt->rkt_rk);
+ rd_kafka_toppar_lock(rktp);
+
+ if (rktp->rktp_leader) {
+ rd_kafka_broker_destroy(rktp->rktp_leader);
+ rktp->rktp_leader = NULL;
+ rktp->rktp_leader_id = -1;
+ rktp->rktp_leader_epoch = -1;
+ }
+
+ rd_kafka_toppar_unlock(rktp);
+ rd_kafka_rdunlock(rktp->rktp_rkt->rkt_rk);
+}
+
+/**
+ * @brief Revert the topic+partition delegation to the internal broker.
+ *
+ * @locks none
+ * @locks_acquired rk rdlock, rktp
+ * @locality any
+ */
+void rd_kafka_toppar_undelegate(rd_kafka_toppar_t *rktp) {
+ rd_kafka_rdlock(rktp->rktp_rkt->rkt_rk);
+ rd_kafka_toppar_lock(rktp);
+
+ rd_kafka_toppar_broker_delegate(rktp, NULL);
+
+ rd_kafka_toppar_unlock(rktp);
+ rd_kafka_rdunlock(rktp->rktp_rkt->rkt_rk);
+}
/**
* @brief Save idempotent producer state for a partition that is about to
@@ -1188,11 +1228,14 @@ rd_bool_t rd_kafka_topic_set_notexists(rd_kafka_topic_t *rkt,
rd_assert(err != RD_KAFKA_RESP_ERR_NO_ERROR);
remains_us =
- (rkt->rkt_ts_create +
+ (rkt->rkt_ts_state +
(rkt->rkt_rk->rk_conf.metadata_propagation_max_ms * 1000)) -
rkt->rkt_ts_metadata;
- if (!permanent && rkt->rkt_state == RD_KAFKA_TOPIC_S_UNKNOWN &&
+ if (!permanent &&
+ (rkt->rkt_state == RD_KAFKA_TOPIC_S_UNKNOWN ||
+ rkt->rkt_state == RD_KAFKA_TOPIC_S_ERROR ||
+ rkt->rkt_state == RD_KAFKA_TOPIC_S_EXISTS) &&
remains_us > 0) {
/* Still allowing topic metadata to propagate. */
rd_kafka_dbg(
@@ -1220,6 +1263,47 @@ rd_bool_t rd_kafka_topic_set_notexists(rd_kafka_topic_t *rkt,
return rd_true;
}
+/**
+ * @brief Mark topic as existent, unless metadata propagation configuration
+ * disallows it.
+ *
+ * @returns true if the topic was marked as existent, else false.
+ *
+ * @locks topic_wrlock() MUST be held.
+ */
+rd_bool_t rd_kafka_topic_set_exists(rd_kafka_topic_t *rkt,
+ rd_kafka_Uuid_t topic_id) {
+ rd_ts_t remains_us;
+
+ if (unlikely(rd_kafka_terminating(rkt->rkt_rk))) {
+ /* Dont update metadata while terminating. */
+ return rd_false;
+ }
+
+ remains_us =
+ (rkt->rkt_ts_state +
+ (rkt->rkt_rk->rk_conf.metadata_propagation_max_ms * 1000)) -
+ rkt->rkt_ts_metadata;
+
+ if (/* Same topic id */
+ rd_kafka_Uuid_cmp(rkt->rkt_topic_id, topic_id) == 0 &&
+ rkt->rkt_state == RD_KAFKA_TOPIC_S_NOTEXISTS && remains_us > 0) {
+ /* Still allowing topic metadata to propagate. */
+ rd_kafka_dbg(
+ rkt->rkt_rk, TOPIC | RD_KAFKA_DBG_METADATA, "TOPICPROP",
+ "Topic %.*s exists after being deleted, "
+ " allowing %dms for metadata propagation before marking "
+ "topic "
+ "as existent",
+ RD_KAFKAP_STR_PR(rkt->rkt_topic), (int)(remains_us / 1000));
+ return rd_false;
+ }
+
+ rd_kafka_topic_set_state(rkt, RD_KAFKA_TOPIC_S_EXISTS);
+
+ return rd_true;
+}
+
/**
* @brief Mark topic as errored, such as when topic authorization fails.
*
@@ -1327,18 +1411,24 @@ rd_kafka_topic_metadata_update(rd_kafka_topic_t *rkt,
mdt->err == RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART ||
mdt->err == RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_ID)
rd_kafka_topic_set_notexists(rkt, mdt->err);
- else if (mdt->partition_cnt > 0)
- rd_kafka_topic_set_state(rkt, RD_KAFKA_TOPIC_S_EXISTS);
- else if (mdt->err)
+ else if (mdt->err == RD_KAFKA_RESP_ERR_NO_ERROR &&
+ mdt->partition_cnt > 0)
+ rd_kafka_topic_set_exists(rkt, mdit->topic_id);
+ else if (mdt->err == RD_KAFKA_RESP_ERR_TOPIC_AUTHORIZATION_FAILED)
+ /* Only set an error when it's permanent and it needs
+ * to be surfaced to the application. */
rd_kafka_topic_set_error(rkt, mdt->err);
/* Update number of partitions, but not if there are
* (possibly intermittent) errors (e.g., "Leader not available"). */
if (mdt->err == RD_KAFKA_RESP_ERR_NO_ERROR) {
- upd += rd_kafka_topic_partition_cnt_update(rkt,
- mdt->partition_cnt);
- if (rd_kafka_Uuid_cmp(mdit->topic_id, RD_KAFKA_UUID_ZERO) &&
- rd_kafka_Uuid_cmp(mdit->topic_id, rkt->rkt_topic_id)) {
+ rd_bool_t different_topic_id =
+ rd_kafka_Uuid_cmp(mdit->topic_id, rkt->rkt_topic_id) != 0;
+ if (different_topic_id ||
+ mdt->partition_cnt > rkt->rkt_partition_cnt)
+ upd += rd_kafka_topic_partition_cnt_update(
+ rkt, mdt->partition_cnt);
+ if (different_topic_id) {
/* FIXME: an offset reset must be triggered.
* when rkt_topic_id wasn't zero.
* There are no problems
@@ -1364,71 +1454,63 @@ rd_kafka_topic_metadata_update(rd_kafka_topic_t *rkt,
* Issue #1985. */
if (old_state == RD_KAFKA_TOPIC_S_UNKNOWN)
upd++;
- }
- /* Update leader for each partition */
- for (j = 0; j < mdt->partition_cnt; j++) {
- int r = 0;
- rd_kafka_broker_t *leader;
- int32_t leader_epoch = mdit->partitions[j].leader_epoch;
- rd_kafka_toppar_t *rktp =
- rd_kafka_toppar_get(rkt, mdt->partitions[j].id, 0);
+ /* Update leader for each partition
+ * only when topic response has no errors. */
+ for (j = 0;
+ j < mdt->partition_cnt && j < rkt->rkt_partition_cnt;
+ j++) {
+ int r = 0;
+ rd_kafka_broker_t *leader;
+ int32_t leader_epoch = mdit->partitions[j].leader_epoch;
+ rd_kafka_toppar_t *rktp =
+ rd_kafka_toppar_get(rkt, mdt->partitions[j].id, 0);
+
+ rd_kafka_dbg(rk, TOPIC | RD_KAFKA_DBG_METADATA,
+ "METADATA",
+ "Topic %s [%" PRId32 "] Leader %" PRId32
+ " Epoch %" PRId32,
+ rkt->rkt_topic->str, mdt->partitions[j].id,
+ mdt->partitions[j].leader, leader_epoch);
+
+ leader = partbrokers[j];
+ partbrokers[j] = NULL;
- rd_kafka_dbg(rk, TOPIC | RD_KAFKA_DBG_METADATA, "METADATA",
- "Topic %s [%" PRId32 "] Leader %" PRId32
- " Epoch %" PRId32,
- rkt->rkt_topic->str, mdt->partitions[j].id,
- mdt->partitions[j].leader, leader_epoch);
-
- leader = partbrokers[j];
- partbrokers[j] = NULL;
-
- /* If broker does not support leaderEpoch(KIP 320) then it is
- * set to -1, we assume that metadata is not stale. */
- if (leader_epoch == -1)
- partition_exists_with_no_leader_epoch = rd_true;
- else if (leader_epoch < rktp->rktp_leader_epoch)
- partition_exists_with_stale_leader_epoch = rd_true;
-
-
- /* Update leader for partition */
- r = rd_kafka_toppar_leader_update(rkt, mdt->partitions[j].id,
- mdt->partitions[j].leader,
- leader, leader_epoch);
-
- upd += (r != 0 ? 1 : 0);
-
- if (leader) {
- if (r != -1)
- leader_cnt++;
- /* Drop reference to broker (from find()) */
- rd_kafka_broker_destroy(leader);
+ /* If broker does not support leaderEpoch(KIP 320) then
+ * it is set to -1, we assume that metadata is not
+ * stale. */
+ if (leader_epoch == -1)
+ partition_exists_with_no_leader_epoch = rd_true;
+ else if (rktp && leader_epoch < rktp->rktp_leader_epoch)
+ partition_exists_with_stale_leader_epoch =
+ rd_true;
+
+
+ /* Update leader for partition */
+ r = rd_kafka_toppar_leader_update(
+ rkt, mdt->partitions[j].id,
+ mdt->partitions[j].leader, leader, leader_epoch);
+
+ upd += (r != 0 ? 1 : 0);
+
+ if (leader) {
+ if (r != -1)
+ leader_cnt++;
+ /* Drop reference to broker (from find()) */
+ rd_kafka_broker_destroy(leader);
+ }
+ RD_IF_FREE(rktp, rd_kafka_toppar_destroy);
}
- RD_IF_FREE(rktp, rd_kafka_toppar_destroy);
}
/* If all partitions have leaders, and this metadata update was not
* stale, we can turn off fast leader query. */
- if (mdt->partition_cnt > 0 && leader_cnt == mdt->partition_cnt &&
+ if (rkt->rkt_partition_cnt > 0 &&
+ leader_cnt == rkt->rkt_partition_cnt &&
(partition_exists_with_no_leader_epoch ||
!partition_exists_with_stale_leader_epoch))
rkt->rkt_flags &= ~RD_KAFKA_TOPIC_F_LEADER_UNAVAIL;
- if (mdt->err != RD_KAFKA_RESP_ERR_NO_ERROR && rkt->rkt_partition_cnt) {
- /* (Possibly intermittent) topic-wide error:
- * remove leaders for partitions */
-
- for (j = 0; j < rkt->rkt_partition_cnt; j++) {
- rd_kafka_toppar_t *rktp;
- if (!rkt->rkt_p[j])
- continue;
-
- rktp = rkt->rkt_p[j];
- rd_kafka_toppar_lock(rktp);
- rd_kafka_toppar_broker_delegate(rktp, NULL);
- rd_kafka_toppar_unlock(rktp);
- }
- }
/* If there was an update to the partitions try to assign
* unassigned messages to new partitions, or fail them */
@@ -1647,8 +1729,8 @@ void rd_kafka_topic_scan_all(rd_kafka_t *rk, rd_ts_t now) {
/* Check if metadata information has timed out. */
if (rkt->rkt_state != RD_KAFKA_TOPIC_S_UNKNOWN &&
- !rd_kafka_metadata_cache_topic_get(rk, rkt->rkt_topic->str,
- 1 /*only valid*/)) {
+ !rd_kafka_metadata_cache_topic_get(
+ rk, rkt->rkt_topic->str, NULL, 1 /*only valid*/)) {
rd_kafka_dbg(rk, TOPIC, "NOINFO",
"Topic %s metadata information timed out "
"(%" PRId64 "ms old)",
@@ -1766,7 +1848,8 @@ void rd_kafka_topic_scan_all(rd_kafka_t *rk, rd_ts_t now) {
* info exists*/
,
rk->rk_conf.allow_auto_create_topics,
- rd_false /*!cgrp_update*/, "refresh unavailable topics");
+ rd_false /*!cgrp_update*/, -1,
+ "refresh unavailable topics");
rd_list_destroy(&query_topics);
}
@@ -1965,7 +2048,7 @@ rd_kafka_topic_info_t *rd_kafka_topic_info_new_with_rack(
/**
* Destroy/free topic_info
*/
-void rd_kafka_topic_info_destroy(rd_kafka_topic_info_t *ti) {
+void rd_kafka_topic_info_destroy_free(void *ti) {
rd_free(ti);
}
@@ -2013,7 +2096,7 @@ void rd_kafka_topic_leader_query0(rd_kafka_t *rk,
rd_kafka_metadata_refresh_topics(
rk, NULL, &topics, force, rk->rk_conf.allow_auto_create_topics,
- rd_false /*!cgrp_update*/, "leader query");
+ rd_false /*!cgrp_update*/, -1, "leader query");
rd_list_destroy(&topics);
}
@@ -2039,7 +2122,7 @@ void rd_kafka_local_topics_to_list(rd_kafka_t *rk,
rd_list_grow(topics, rk->rk_topic_cnt);
TAILQ_FOREACH(rkt, &rk->rk_topics, rkt_link)
rd_list_add(topics, rd_strdup(rkt->rkt_topic->str));
- cache_cnt = rd_kafka_metadata_cache_topics_to_list(rk, topics);
+ cache_cnt = rd_kafka_metadata_cache_topics_to_list(rk, topics, rd_true);
if (cache_cntp)
*cache_cntp = cache_cnt;
rd_kafka_rdunlock(rk);
@@ -2056,8 +2139,8 @@ void rd_ut_kafka_topic_set_topic_exists(rd_kafka_topic_t *rkt,
rd_kafka_metadata_partition_internal_t *partitions =
rd_calloc(partition_cnt, sizeof(*partitions));
struct rd_kafka_metadata_topic mdt = {.topic =
- (char *)rkt->rkt_topic->str,
- .partition_cnt = partition_cnt};
+ (char *)rkt->rkt_topic->str,
+ .partition_cnt = partition_cnt};
rd_kafka_metadata_topic_internal_t mdit = {.partitions = partitions};
int i;
@@ -2071,7 +2154,7 @@ void rd_ut_kafka_topic_set_topic_exists(rd_kafka_topic_t *rkt,
rd_kafka_wrlock(rkt->rkt_rk);
rd_kafka_metadata_cache_topic_update(rkt->rkt_rk, &mdt, &mdit, rd_true,
- rd_false, NULL, 0, rd_false);
+ rd_false, rd_true);
rd_kafka_topic_metadata_update(rkt, &mdt, &mdit, rd_clock());
rd_kafka_wrunlock(rkt->rkt_rk);
rd_free(partitions);
diff --git a/src/rdkafka_topic.h b/src/rdkafka_topic.h
index 6e25e7f74e..7035b188b0 100644
--- a/src/rdkafka_topic.h
+++ b/src/rdkafka_topic.h
@@ -159,12 +159,14 @@ struct rd_kafka_topic_s {
rd_refcnt_t rkt_app_refcnt; /**< Number of active rkt's new()ed
* by application. */
- enum { RD_KAFKA_TOPIC_S_UNKNOWN, /* No cluster information yet */
- RD_KAFKA_TOPIC_S_EXISTS, /* Topic exists in cluster */
- RD_KAFKA_TOPIC_S_NOTEXISTS, /* Topic is not known in cluster */
- RD_KAFKA_TOPIC_S_ERROR, /* Topic exists but is in an errored
- * state, such as auth failure. */
+ enum {
+ RD_KAFKA_TOPIC_S_UNKNOWN, /* No cluster information yet */
+ RD_KAFKA_TOPIC_S_EXISTS, /* Topic exists in cluster */
+ RD_KAFKA_TOPIC_S_NOTEXISTS, /* Topic is not known in cluster */
+ RD_KAFKA_TOPIC_S_ERROR, /* Topic exists but is in an errored
+ * state, such as auth failure. */
} rkt_state;
+ rd_ts_t rkt_ts_state; /**< State change time. */
int rkt_flags;
#define RD_KAFKA_TOPIC_F_LEADER_UNAVAIL \
@@ -289,7 +291,7 @@ rd_kafka_topic_info_t *rd_kafka_topic_info_new_with_rack(
const char *topic,
int partition_cnt,
const rd_kafka_metadata_partition_internal_t *mdpi);
-void rd_kafka_topic_info_destroy(rd_kafka_topic_info_t *ti);
+void rd_kafka_topic_info_destroy_free(void *ti);
int rd_kafka_topic_match(rd_kafka_t *rk,
const char *pattern,
@@ -302,6 +304,10 @@ int rd_kafka_toppar_broker_update(rd_kafka_toppar_t *rktp,
int rd_kafka_toppar_delegate_to_leader(rd_kafka_toppar_t *rktp);
+void rd_kafka_toppar_undelegate(rd_kafka_toppar_t *rktp);
+
+void rd_kafka_toppar_forget_leader(rd_kafka_toppar_t *rktp);
+
rd_kafka_resp_err_t rd_kafka_topics_leader_query_sync(rd_kafka_t *rk,
int all_topics,
const rd_list_t *topics,
@@ -314,8 +320,8 @@ void rd_kafka_topic_leader_query0(rd_kafka_t *rk,
rd_kafka_topic_leader_query0(rk, rkt, 1 /*lock*/, \
rd_false /*dont force*/)
-#define rd_kafka_topic_fast_leader_query(rk) \
- rd_kafka_metadata_fast_leader_query(rk)
+#define rd_kafka_topic_fast_leader_query(rk, force) \
+ rd_kafka_metadata_fast_leader_query(rk, force)
void rd_kafka_local_topics_to_list(rd_kafka_t *rk,
rd_list_t *topics,
diff --git a/src/rdkafka_transport.c b/src/rdkafka_transport.c
index f133d8fdde..b7c691d762 100644
--- a/src/rdkafka_transport.c
+++ b/src/rdkafka_transport.c
@@ -278,10 +278,17 @@ static ssize_t rd_kafka_transport_socket_recvmsg(rd_kafka_transport_t *rktrans,
if (unlikely(r <= 0)) {
if (r == -1 && rd_socket_errno == EAGAIN)
return 0;
- else if (r == 0 || (r == -1 && rd_socket_errno == ECONNRESET)) {
+ else if (r == 0) {
/* Receive 0 after POLLIN event means
* connection closed. */
- rd_snprintf(errstr, errstr_size, "Disconnected");
+ rd_snprintf(errstr, errstr_size,
+ "Disconnected: connection closed by "
+ "peer: receive 0 after POLLIN");
+ return -1;
+ } else if (r == -1 && rd_socket_errno == ECONNRESET) {
+ rd_snprintf(errstr, errstr_size,
+ "Disconnected: connection "
+ "reset by peer");
return -1;
} else if (r == -1) {
rd_snprintf(errstr, errstr_size, "%s",
@@ -335,7 +342,9 @@ static ssize_t rd_kafka_transport_socket_recv0(rd_kafka_transport_t *rktrans,
} else if (unlikely(r == 0)) {
/* Receive 0 after POLLIN event means
* connection closed. */
- rd_snprintf(errstr, errstr_size, "Disconnected");
+ rd_snprintf(errstr, errstr_size,
+ "Disconnected: connection closed by "
+ "peer: receive 0 after POLLIN");
return -1;
}
@@ -543,33 +552,6 @@ void rd_kafka_transport_post_connect_setup(rd_kafka_transport_t *rktrans) {
rd_kafka_broker_t *rkb = rktrans->rktrans_rkb;
unsigned int slen;
- /* Set socket send & receive buffer sizes if configuerd */
- if (rkb->rkb_rk->rk_conf.socket_sndbuf_size != 0) {
- if (setsockopt(
- rktrans->rktrans_s, SOL_SOCKET, SO_SNDBUF,
- (void *)&rkb->rkb_rk->rk_conf.socket_sndbuf_size,
- sizeof(rkb->rkb_rk->rk_conf.socket_sndbuf_size)) ==
- RD_SOCKET_ERROR)
- rd_rkb_log(rkb, LOG_WARNING, "SNDBUF",
- "Failed to set socket send "
- "buffer size to %i: %s",
- rkb->rkb_rk->rk_conf.socket_sndbuf_size,
- rd_socket_strerror(rd_socket_errno));
- }
-
- if (rkb->rkb_rk->rk_conf.socket_rcvbuf_size != 0) {
- if (setsockopt(
- rktrans->rktrans_s, SOL_SOCKET, SO_RCVBUF,
- (void *)&rkb->rkb_rk->rk_conf.socket_rcvbuf_size,
- sizeof(rkb->rkb_rk->rk_conf.socket_rcvbuf_size)) ==
- RD_SOCKET_ERROR)
- rd_rkb_log(rkb, LOG_WARNING, "RCVBUF",
- "Failed to set socket receive "
- "buffer size to %i: %s",
- rkb->rkb_rk->rk_conf.socket_rcvbuf_size,
- rd_socket_strerror(rd_socket_errno));
- }
-
/* Get send and receive buffer sizes to allow limiting
* the total number of bytes passed with iovecs to sendmsg()
* and recvmsg(). */
@@ -598,19 +580,6 @@ void rd_kafka_transport_post_connect_setup(rd_kafka_transport_t *rktrans) {
} else if (rktrans->rktrans_sndbuf_size < 1024 * 64)
rktrans->rktrans_sndbuf_size =
1024 * 64; /* Use at least 64KB */
-
-
-#ifdef TCP_NODELAY
- if (rkb->rkb_rk->rk_conf.socket_nagle_disable) {
- int one = 1;
- if (setsockopt(rktrans->rktrans_s, IPPROTO_TCP, TCP_NODELAY,
- (void *)&one, sizeof(one)) == RD_SOCKET_ERROR)
- rd_rkb_log(rkb, LOG_WARNING, "NAGLE",
- "Failed to disable Nagle (TCP_NODELAY) "
- "on socket: %s",
- rd_socket_strerror(rd_socket_errno));
- }
-#endif
}
@@ -746,7 +715,9 @@ static void rd_kafka_transport_io_event(rd_kafka_transport_t *rktrans,
if (r == 0 /* handshake still in progress */ &&
(events & POLLHUP)) {
rd_kafka_broker_conn_closed(
- rkb, RD_KAFKA_RESP_ERR__TRANSPORT, "Disconnected");
+ rkb, RD_KAFKA_RESP_ERR__TRANSPORT,
+ "Disconnected: during "
+ "SSL connection handshake");
return;
}
@@ -770,7 +741,8 @@ static void rd_kafka_transport_io_event(rd_kafka_transport_t *rktrans,
if (events & POLLHUP) {
rd_kafka_broker_fail(rkb, LOG_ERR,
RD_KAFKA_RESP_ERR__AUTHENTICATION,
- "Disconnected");
+ "Disconnected: hung up from "
+ "peer in state AUTH_LEGACY");
return;
}
@@ -781,7 +753,6 @@ static void rd_kafka_transport_io_event(rd_kafka_transport_t *rktrans,
case RD_KAFKA_BROKER_STATE_AUTH_HANDSHAKE:
case RD_KAFKA_BROKER_STATE_AUTH_REQ:
case RD_KAFKA_BROKER_STATE_UP:
- case RD_KAFKA_BROKER_STATE_UPDATE:
if (events & POLLIN) {
while (rkb->rkb_state >= RD_KAFKA_BROKER_STATE_UP &&
@@ -795,7 +766,9 @@ static void rd_kafka_transport_io_event(rd_kafka_transport_t *rktrans,
if (events & POLLHUP) {
rd_kafka_broker_conn_closed(
- rkb, RD_KAFKA_RESP_ERR__TRANSPORT, "Disconnected");
+ rkb, RD_KAFKA_RESP_ERR__TRANSPORT,
+ "Disconnected: connection closed by "
+ "peer: POLLHUP");
return;
}
@@ -1081,6 +1054,45 @@ rd_kafka_transport_t *rd_kafka_transport_new(rd_kafka_broker_t *rkb,
}
#endif
+#ifdef TCP_NODELAY
+ if (rkb->rkb_rk->rk_conf.socket_nagle_disable) {
+ int one = 1;
+ if (setsockopt(s, IPPROTO_TCP, TCP_NODELAY, (void *)&one,
+ sizeof(one)) == RD_SOCKET_ERROR)
+ rd_rkb_log(rkb, LOG_WARNING, "NAGLE",
+ "Failed to disable Nagle (TCP_NODELAY) "
+ "on socket: %s",
+ rd_socket_strerror(rd_socket_errno));
+ }
+#endif
+
+ /* Set socket send & receive buffer sizes if configuerd */
+ if (rkb->rkb_rk->rk_conf.socket_sndbuf_size != 0) {
+ if (setsockopt(
+ s, SOL_SOCKET, SO_SNDBUF,
+ (void *)&rkb->rkb_rk->rk_conf.socket_sndbuf_size,
+ sizeof(rkb->rkb_rk->rk_conf.socket_sndbuf_size)) ==
+ RD_SOCKET_ERROR)
+ rd_rkb_log(rkb, LOG_WARNING, "SNDBUF",
+ "Failed to set socket send "
+ "buffer size to %i: %s",
+ rkb->rkb_rk->rk_conf.socket_sndbuf_size,
+ rd_socket_strerror(rd_socket_errno));
+ }
+
+ if (rkb->rkb_rk->rk_conf.socket_rcvbuf_size != 0) {
+ if (setsockopt(
+ s, SOL_SOCKET, SO_RCVBUF,
+ (void *)&rkb->rkb_rk->rk_conf.socket_rcvbuf_size,
+ sizeof(rkb->rkb_rk->rk_conf.socket_rcvbuf_size)) ==
+ RD_SOCKET_ERROR)
+ rd_rkb_log(rkb, LOG_WARNING, "RCVBUF",
+ "Failed to set socket receive "
+ "buffer size to %i: %s",
+ rkb->rkb_rk->rk_conf.socket_rcvbuf_size,
+ rd_socket_strerror(rd_socket_errno));
+ }
+
/* Set the socket to non-blocking */
if ((r = rd_fd_set_nonblocking(s))) {
rd_snprintf(errstr, errstr_size,
diff --git a/src/rdkafka_transport.h b/src/rdkafka_transport.h
index c5f73163f9..d906b44211 100644
--- a/src/rdkafka_transport.h
+++ b/src/rdkafka_transport.h
@@ -88,6 +88,11 @@ void rd_kafka_transport_set_blocked(rd_kafka_transport_t *rktrans,
} while (0)
#endif
+static RD_INLINE RD_UNUSED rd_bool_t
+rd_kafka_transport_error_disconnected(const char *errstr) {
+ return strncmp(errstr, "Disconnected", 12) == 0;
+}
+
void rd_kafka_transport_init(void);
diff --git a/src/rdkafka_txnmgr.c b/src/rdkafka_txnmgr.c
index 90d330146f..019a3b80c6 100644
--- a/src/rdkafka_txnmgr.c
+++ b/src/rdkafka_txnmgr.c
@@ -2029,7 +2029,7 @@ rd_kafka_error_t *rd_kafka_send_offsets_to_transaction(
rd_kafka_topic_partition_list_sort_by_topic(valid_offsets);
rko = rd_kafka_op_new_cb(rk, RD_KAFKA_OP_TXN,
- rd_kafka_txn_op_send_offsets_to_transaction);
+ rd_kafka_txn_op_send_offsets_to_transaction);
rko->rko_u.txn.offsets = valid_offsets;
rko->rko_u.txn.cgmetadata =
rd_kafka_consumer_group_metadata_dup(cgmetadata);
@@ -2958,6 +2958,11 @@ static void rd_kafka_txn_handle_FindCoordinator(rd_kafka_t *rk,
NodeId);
err = RD_KAFKA_RESP_ERR__UNKNOWN_BROKER;
}
+ if (rkb && rkb->rkb_source != RD_KAFKA_LEARNED) {
+ rd_kafka_broker_destroy(rkb);
+ rkb = NULL;
+ err = RD_KAFKA_RESP_ERR__UNKNOWN_BROKER;
+ }
rd_kafka_rdunlock(rk);
if (err)
diff --git a/src/rdmap.c b/src/rdmap.c
index 1e82bcb9a2..522b786c44 100644
--- a/src/rdmap.c
+++ b/src/rdmap.c
@@ -404,12 +404,17 @@ static int unittest_typed_map2(void) {
static int unittest_untyped_map(void) {
rd_map_t rmap;
int pass, i, r;
- int cnt = 100000;
+ int cnt = rd_unittest_with_valgrind ? 1000 : 100000;
int exp_cnt = 0, get_cnt = 0, iter_cnt = 0;
const rd_map_elem_t *elem;
rd_ts_t ts = rd_clock();
rd_ts_t ts_get = 0;
+ if (rd_unittest_with_valgrind)
+ RD_UT_WARN(
+ "Reducing count in "
+ "untyped map test when using Valgrind");
+
rd_map_init(&rmap, cnt, rd_map_str_cmp, rd_map_str_hash, rd_free,
rd_free);
diff --git a/src/rdrand.c b/src/rdrand.c
index bdab002968..104b289d6f 100644
--- a/src/rdrand.c
+++ b/src/rdrand.c
@@ -30,6 +30,7 @@
#include "rdrand.h"
#include "rdtime.h"
#include "tinycthread.h"
+#include "rdmurmur2.h"
int rd_jitter(int low, int high) {
int rand_num;
@@ -40,8 +41,17 @@ int rd_jitter(int low, int high) {
if (unlikely(seed == 0)) {
struct timeval tv;
rd_gettimeofday(&tv, NULL);
- seed = (unsigned int)(tv.tv_usec / 1000);
+ seed = (unsigned int)(tv.tv_usec);
seed ^= (unsigned int)(intptr_t)thrd_current();
+
+ /* When many threads are created at the same time and the
+ * thread id is different only by a few bits it's possible that
+ * `rand_r`, that is initially multiplying by `1103515245`,
+ * truncates the variable bits and uses the same seed for
+ * different threads. By applying `murmur2` we ensure that seed
+ * variability is distributed across various bits at different
+ * positions. */
+ seed = (unsigned int)rd_murmur2(&seed, sizeof(seed));
}
rand_num = rand_r(&seed);
diff --git a/src/rdtime.h b/src/rdtime.h
index 4a7e76d752..2d8207a5a0 100644
--- a/src/rdtime.h
+++ b/src/rdtime.h
@@ -2,6 +2,7 @@
* librd - Rapid Development C library
*
* Copyright (c) 2012-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -164,9 +165,44 @@ static RD_INLINE int rd_timeout_ms(rd_ts_t timeout_us) {
return (int)((timeout_us + 999) / 1000);
}
+/**
+ * @brief Initialize an absolute timeout based on the provided \p timeout_ms
+ * and given clock \p now
+ *
+ * To be used with rd_timeout_remains() or rd_timeout_remains_us().
+ *
+ * Honours RD_POLL_INFINITE, RD_POLL_NOWAIT.
+ *
+ * @returns the absolute timeout.
+ */
+static RD_INLINE rd_ts_t rd_timeout_init0(rd_ts_t now, int timeout_ms) {
+ if (timeout_ms == RD_POLL_INFINITE || timeout_ms == RD_POLL_NOWAIT)
+ return timeout_ms;
+
+ return now + ((rd_ts_t)timeout_ms * 1000);
+}
+
+/**
+ * @brief Initialize an absolute timeout based on the provided \p timeout_us
+ * and given clock \p now
+ *
+ * To be used with rd_timeout_remains() or rd_timeout_remains_us().
+ *
+ * Honours RD_POLL_INFINITE, RD_POLL_NOWAIT.
+ *
+ * @returns the absolute timeout.
+ */
+static RD_INLINE rd_ts_t rd_timeout_init_us0(rd_ts_t now, rd_ts_t timeout_us) {
+ if (timeout_us == RD_POLL_INFINITE || timeout_us == RD_POLL_NOWAIT)
+ return timeout_us;
+
+ return now + timeout_us;
+}
+
/**
* @brief Initialize an absolute timeout based on the provided \p timeout_ms
+ * and current clock.
*
* To be used with rd_timeout_adjust().
*
@@ -176,12 +212,35 @@ static RD_INLINE int rd_timeout_ms(rd_ts_t timeout_us) {
* to rd_timeout_adjust().
*/
static RD_INLINE rd_ts_t rd_timeout_init(int timeout_ms) {
- if (timeout_ms == RD_POLL_INFINITE || timeout_ms == RD_POLL_NOWAIT)
- return timeout_ms;
+ return rd_timeout_init0(rd_clock(), timeout_ms);
+}
- return rd_clock() + ((rd_ts_t)timeout_ms * 1000);
+/**
+ * @brief Initialize an absolute timeout based on the provided \p timeout_us
+ * and current clock.
+ *
+ * To be used with rd_timeout_remains() or rd_timeout_remains_us().
+ *
+ * Honours RD_POLL_INFINITE, RD_POLL_NOWAIT.
+ *
+ * @returns the absolute timeout.
+ */
+static RD_INLINE rd_ts_t rd_timeout_init_us(rd_ts_t timeout_us) {
+ return rd_timeout_init_us0(rd_clock(), timeout_us);
}
+/**
+ * @brief Gets time since epoch (UTC).
+ */
+static RD_INLINE void rd_timespec_get(struct timespec *tspec) {
+#if defined(__APPLE__) || (defined(__ANDROID__) && __ANDROID_API__ < 29)
+ struct timeval tv;
+ gettimeofday(&tv, NULL);
+ TIMEVAL_TO_TIMESPEC(&tv, tspec);
+#else
+ timespec_get(tspec, TIME_UTC);
+#endif
+}
/**
* @brief Initialize an absolute timespec timeout based on the provided
@@ -197,13 +256,7 @@ static RD_INLINE void rd_timeout_init_timespec_us(struct timespec *tspec,
tspec->tv_sec = timeout_us;
tspec->tv_nsec = 0;
} else {
-#if defined(__APPLE__) || (defined(__ANDROID__) && __ANDROID_API__ < 29)
- struct timeval tv;
- gettimeofday(&tv, NULL);
- TIMEVAL_TO_TIMESPEC(&tv, tspec);
-#else
- timespec_get(tspec, TIME_UTC);
-#endif
+ rd_timespec_get(tspec);
tspec->tv_sec += timeout_us / 1000000;
tspec->tv_nsec += (timeout_us % 1000000) * 1000;
if (tspec->tv_nsec >= 1000000000) {
@@ -227,13 +280,7 @@ static RD_INLINE void rd_timeout_init_timespec(struct timespec *tspec,
tspec->tv_sec = timeout_ms;
tspec->tv_nsec = 0;
} else {
-#if defined(__APPLE__) || (defined(__ANDROID__) && __ANDROID_API__ < 29)
- struct timeval tv;
- gettimeofday(&tv, NULL);
- TIMEVAL_TO_TIMESPEC(&tv, tspec);
-#else
- timespec_get(tspec, TIME_UTC);
-#endif
+ rd_timespec_get(tspec);
tspec->tv_sec += timeout_ms / 1000;
tspec->tv_nsec += (timeout_ms % 1000) * 1000000;
if (tspec->tv_nsec >= 1000000000) {
diff --git a/src/rdunittest.c b/src/rdunittest.c
index fc82c242cd..04e85b6468 100644
--- a/src/rdunittest.c
+++ b/src/rdunittest.c
@@ -55,6 +55,7 @@
#include "rdkafka_txnmgr.h"
rd_bool_t rd_unittest_assert_on_failure = rd_false;
+rd_bool_t rd_unittest_with_valgrind = rd_false;
rd_bool_t rd_unittest_on_ci = rd_false;
rd_bool_t rd_unittest_slow = rd_false;
@@ -416,6 +417,7 @@ static int unittest_rdclock(void) {
extern int unittest_string(void);
extern int unittest_cgrp(void);
+extern int unittest_mock_cluster(void);
#if WITH_SASL_SCRAM
extern int unittest_scram(void);
#endif
@@ -426,7 +428,10 @@ extern int unittest_http(void);
#endif
#if WITH_OAUTHBEARER_OIDC
extern int unittest_sasl_oauthbearer_oidc(void);
+extern int unittest_sasl_oauthbearer_oidc_jwt_bearer(void);
+extern int unittest_sasl_oauthbearer_oidc_assertion(void);
#endif
+extern int unittest_telemetry(void);
extern int unittest_telemetry_decode(void);
int rd_unittest(void) {
@@ -435,42 +440,48 @@ int rd_unittest(void) {
const char *name;
int (*call)(void);
} unittests[] = {
- {"sysqueue", unittest_sysqueue},
- {"string", unittest_string},
- {"map", unittest_map},
- {"rdbuf", unittest_rdbuf},
- {"rdvarint", unittest_rdvarint},
- {"crc32c", unittest_rd_crc32c},
- {"msg", unittest_msg},
- {"murmurhash", unittest_murmur2},
- {"fnv1a", unittest_fnv1a},
+ {"sysqueue", unittest_sysqueue},
+ {"string", unittest_string},
+ {"map", unittest_map},
+ {"rdbuf", unittest_rdbuf},
+ {"rdvarint", unittest_rdvarint},
+ {"crc32c", unittest_rd_crc32c},
+ {"msg", unittest_msg},
+ {"murmurhash", unittest_murmur2},
+ {"fnv1a", unittest_fnv1a},
+ {"mock", unittest_mock_cluster},
#if WITH_HDRHISTOGRAM
- {"rdhdrhistogram", unittest_rdhdrhistogram},
+ {"rdhdrhistogram", unittest_rdhdrhistogram},
#endif
#ifdef _WIN32
- {"rdclock", unittest_rdclock},
+ {"rdclock", unittest_rdclock},
#endif
- {"conf", unittest_conf},
- {"broker", unittest_broker},
- {"request", unittest_request},
+ {"conf", unittest_conf},
+ {"broker", unittest_broker},
+ {"request", unittest_request},
#if WITH_SASL_OAUTHBEARER
- {"sasl_oauthbearer", unittest_sasl_oauthbearer},
+ {"sasl_oauthbearer", unittest_sasl_oauthbearer},
#endif
- {"aborted_txns", unittest_aborted_txns},
- {"cgrp", unittest_cgrp},
+ {"aborted_txns", unittest_aborted_txns},
+ {"cgrp", unittest_cgrp},
#if WITH_SASL_SCRAM
- {"scram", unittest_scram},
+ {"scram", unittest_scram},
#endif
- {"assignors", unittest_assignors},
+ {"assignors", unittest_assignors},
#if WITH_CURL
- {"http", unittest_http},
+ {"http", unittest_http},
#endif
#if WITH_OAUTHBEARER_OIDC
- {"sasl_oauthbearer_oidc", unittest_sasl_oauthbearer_oidc},
+ {"sasl_oauthbearer_oidc", unittest_sasl_oauthbearer_oidc},
+ {"sasl_oauthbearer_oidc_jwt_bearer",
+ unittest_sasl_oauthbearer_oidc_jwt_bearer},
+ {"sasl_oauthbearer_oidc_assertion",
+ unittest_sasl_oauthbearer_oidc_assertion},
#endif
- {"telemetry", unittest_telemetry_decode},
- {NULL}
- };
+ {"telemetry", unittest_telemetry},
+ {"telemetry_decode", unittest_telemetry_decode},
+ {"feature", unittest_feature},
+ {NULL}};
int i;
const char *match = rd_getenv("RD_UT_TEST", NULL);
int cnt = 0;
@@ -482,7 +493,13 @@ int rd_unittest(void) {
rd_unittest_on_ci = rd_true;
}
- if (rd_unittest_on_ci || (ENABLE_DEVEL + 0)) {
+ if (rd_strcmp(rd_getenv("TEST_MODE", NULL), "valgrind") == 0) {
+ RD_UT_SAY("Unittests running with valgrind");
+ rd_unittest_with_valgrind = rd_true;
+ }
+
+ if (rd_unittest_on_ci || rd_unittest_with_valgrind ||
+ (ENABLE_DEVEL + 0)) {
RD_UT_SAY("Unittests will not error out on slow CPUs");
rd_unittest_slow = rd_true;
}
diff --git a/src/rdunittest.h b/src/rdunittest.h
index a9e709fa73..c063d151f6 100644
--- a/src/rdunittest.h
+++ b/src/rdunittest.h
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2017-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -34,6 +35,7 @@
extern rd_bool_t rd_unittest_assert_on_failure;
extern rd_bool_t rd_unittest_on_ci;
+extern rd_bool_t rd_unittest_with_valgrind;
extern rd_bool_t rd_unittest_slow;
#define ENABLE_CODECOV ENABLE_DEVEL
diff --git a/src/rdwin32.h b/src/rdwin32.h
index 37c25843ac..40ea43a7ac 100644
--- a/src/rdwin32.h
+++ b/src/rdwin32.h
@@ -42,7 +42,7 @@
#include /* for sockets + struct timeval */
#include
#include
-
+#include
/**
* Types
diff --git a/src/regexp.c b/src/regexp.c
index 603546c478..e9dba92469 100644
--- a/src/regexp.c
+++ b/src/regexp.c
@@ -95,16 +95,17 @@ static Rune canon(Rune c) {
/* Scan */
-enum { L_CHAR = 256,
- L_CCLASS, /* character class */
- L_NCCLASS, /* negative character class */
- L_NC, /* "(?:" no capture */
- L_PLA, /* "(?=" positive lookahead */
- L_NLA, /* "(?!" negative lookahead */
- L_WORD, /* "\b" word boundary */
- L_NWORD, /* "\B" non-word boundary */
- L_REF, /* "\1" back-reference */
- L_COUNT /* {M,N} */
+enum {
+ L_CHAR = 256,
+ L_CCLASS, /* character class */
+ L_NCCLASS, /* negative character class */
+ L_NC, /* "(?:" no capture */
+ L_PLA, /* "(?=" positive lookahead */
+ L_NLA, /* "(?!" negative lookahead */
+ L_WORD, /* "\b" word boundary */
+ L_NWORD, /* "\B" non-word boundary */
+ L_REF, /* "\1" back-reference */
+ L_COUNT /* {M,N} */
};
static int hex(Restate *g, int c) {
@@ -451,21 +452,23 @@ static int lex(Restate *g) {
/* Parse */
-enum { P_CAT,
- P_ALT,
- P_REP,
- P_BOL,
- P_EOL,
- P_WORD,
- P_NWORD,
- P_PAR,
- P_PLA,
- P_NLA,
- P_ANY,
- P_CHAR,
- P_CCLASS,
- P_NCCLASS,
- P_REF };
+enum {
+ P_CAT,
+ P_ALT,
+ P_REP,
+ P_BOL,
+ P_EOL,
+ P_WORD,
+ P_NWORD,
+ P_PAR,
+ P_PLA,
+ P_NLA,
+ P_ANY,
+ P_CHAR,
+ P_CCLASS,
+ P_NCCLASS,
+ P_REF
+};
struct Renode {
unsigned char type;
@@ -662,23 +665,25 @@ static Renode *parsealt(Restate *g) {
/* Compile */
-enum { I_END,
- I_JUMP,
- I_SPLIT,
- I_PLA,
- I_NLA,
- I_ANYNL,
- I_ANY,
- I_CHAR,
- I_CCLASS,
- I_NCCLASS,
- I_REF,
- I_BOL,
- I_EOL,
- I_WORD,
- I_NWORD,
- I_LPAR,
- I_RPAR };
+enum {
+ I_END,
+ I_JUMP,
+ I_SPLIT,
+ I_PLA,
+ I_NLA,
+ I_ANYNL,
+ I_ANY,
+ I_CHAR,
+ I_CCLASS,
+ I_NCCLASS,
+ I_REF,
+ I_BOL,
+ I_EOL,
+ I_WORD,
+ I_NWORD,
+ I_LPAR,
+ I_RPAR
+};
struct Reinst {
unsigned char opcode;
diff --git a/src/tinycthread_extra.c b/src/tinycthread_extra.c
index 11dc0f212f..6f6d0a5957 100644
--- a/src/tinycthread_extra.c
+++ b/src/tinycthread_extra.c
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2018-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -78,27 +79,41 @@ void cnd_wait_exit(cnd_t *cond) {
int cnd_timedwait_ms(cnd_t *cnd, mtx_t *mtx, int timeout_ms) {
- if (timeout_ms == -1 /* INFINITE*/)
+ int ret;
+ rd_ts_t abs_timeout;
+ rd_bool_t continue_timedwait = rd_true;
+
+ if (timeout_ms == RD_POLL_INFINITE)
return cnd_wait(cnd, mtx);
#if defined(_TTHREAD_WIN32_)
return _cnd_timedwait_win32(cnd, mtx, (DWORD)timeout_ms);
#else
- struct timeval tv;
- struct timespec ts;
-
- gettimeofday(&tv, NULL);
- ts.tv_sec = tv.tv_sec;
- ts.tv_nsec = tv.tv_usec * 1000;
-
- ts.tv_sec += timeout_ms / 1000;
- ts.tv_nsec += (timeout_ms % 1000) * 1000000;
-
- if (ts.tv_nsec >= 1000000000) {
- ts.tv_sec++;
- ts.tv_nsec -= 1000000000;
- }
-
- return cnd_timedwait(cnd, mtx, &ts);
+ abs_timeout = rd_timeout_init(timeout_ms);
+ do {
+ struct timeval tv;
+ struct timespec ts;
+
+ gettimeofday(&tv, NULL);
+ ts.tv_sec = tv.tv_sec;
+ ts.tv_nsec = tv.tv_usec * 1000;
+
+ ts.tv_sec += timeout_ms / 1000;
+ ts.tv_nsec += (timeout_ms % 1000) * 1000000;
+
+ if (ts.tv_nsec >= 1000000000) {
+ ts.tv_sec++;
+ ts.tv_nsec -= 1000000000;
+ }
+
+ ret = cnd_timedwait(cnd, mtx, &ts);
+ continue_timedwait = ret == thrd_timedout;
+ if (continue_timedwait) {
+ timeout_ms = rd_timeout_remains(abs_timeout);
+ if (rd_timeout_expired(timeout_ms))
+ continue_timedwait = rd_false;
+ }
+ } while (continue_timedwait);
+ return ret;
#endif
}
@@ -113,13 +128,22 @@ int cnd_timedwait_msp(cnd_t *cnd, mtx_t *mtx, int *timeout_msp) {
return r;
}
-int cnd_timedwait_abs(cnd_t *cnd, mtx_t *mtx, const struct timespec *tspec) {
- if (tspec->tv_sec == RD_POLL_INFINITE)
+int cnd_timedwait_abs(cnd_t *cnd, mtx_t *mtx, rd_ts_t abs_timeout) {
+ int r = thrd_timedout;
+ int timeout_ms;
+ if (abs_timeout == RD_POLL_INFINITE)
return cnd_wait(cnd, mtx);
- else if (tspec->tv_sec == RD_POLL_NOWAIT)
+ else if (abs_timeout == RD_POLL_NOWAIT)
return thrd_timedout;
- return cnd_timedwait(cnd, mtx, tspec);
+ do {
+ timeout_ms = rd_timeout_remains(abs_timeout);
+ if (timeout_ms == RD_POLL_NOWAIT)
+ break;
+ r = cnd_timedwait_ms(cnd, mtx, timeout_ms);
+ } while (r == thrd_timedout);
+
+ return r;
}
diff --git a/src/tinycthread_extra.h b/src/tinycthread_extra.h
index 2207022592..cb6b611ea7 100644
--- a/src/tinycthread_extra.h
+++ b/src/tinycthread_extra.h
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2018-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -40,6 +41,7 @@
#include /* needed for rwlock_t */
#endif
+#include "rdtypes.h"
/**
* @brief Set thread system name if platform supports it (pthreads)
@@ -83,13 +85,15 @@ int cnd_timedwait_ms(cnd_t *cnd, mtx_t *mtx, int timeout_ms);
int cnd_timedwait_msp(cnd_t *cnd, mtx_t *mtx, int *timeout_msp);
/**
- * @brief Same as cnd_timedwait() but honours
+ * @brief Same as cnd_timedwait() but takes an ansolute timeout in microseconds.
+ * Honours
* RD_POLL_INFINITE (uses cnd_wait()),
* and RD_POLL_NOWAIT (return thrd_timedout immediately).
*
- * @remark Set up \p tspec with rd_timeout_init_timespec().
+ * @remark Set up \p abs_timeout with rd_timeout_init() or
+ * rd_timeout_init_us().
*/
-int cnd_timedwait_abs(cnd_t *cnd, mtx_t *mtx, const struct timespec *tspec);
+int cnd_timedwait_abs(cnd_t *cnd, mtx_t *mtx, rd_ts_t abs_timeout);
diff --git a/tests/0001-multiobj.c b/tests/0001-multiobj.c
index 423bd15ae3..9856dce76e 100644
--- a/tests/0001-multiobj.c
+++ b/tests/0001-multiobj.c
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2012-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -92,6 +93,9 @@ int main_0001_multiobj(int argc, char **argv) {
/* Topic is created on the first iteration. */
if (i > 0)
TIMING_ASSERT(&t_full, 0, 999);
+ else
+ /* Allow metadata propagation. */
+ rd_sleep(1);
}
return 0;
diff --git a/tests/0004-conf.c b/tests/0004-conf.c
index 5dbd9f0b1d..e129b707cc 100644
--- a/tests/0004-conf.c
+++ b/tests/0004-conf.c
@@ -511,28 +511,27 @@ int main_0004_conf(int argc, char **argv) {
int i;
const char *topic;
static const char *gconfs[] = {
- "message.max.bytes",
- "12345", /* int property */
- "client.id",
- "my id", /* string property */
- "debug",
- "topic,metadata,interceptor", /* S2F property */
- "topic.blacklist",
- "__.*", /* #778 */
- "auto.offset.reset",
- "earliest", /* Global->Topic fallthru */
+ "message.max.bytes",
+ "12345", /* int property */
+ "client.id",
+ "my id", /* string property */
+ "debug",
+ "topic,metadata,interceptor", /* S2F property */
+ "topic.blacklist",
+ "__.*", /* #778 */
+ "auto.offset.reset",
+ "earliest", /* Global->Topic fallthru */
#if WITH_ZLIB
- "compression.codec",
- "gzip", /* S2I property */
+ "compression.codec",
+ "gzip", /* S2I property */
#endif
#if defined(_WIN32)
- "ssl.ca.certificate.stores",
- "Intermediate ,, Root ,",
+ "ssl.ca.certificate.stores",
+ "Intermediate ,, Root ,",
#endif
- "client.dns.lookup",
- "resolve_canonical_bootstrap_servers_only",
- NULL
- };
+ "client.dns.lookup",
+ "resolve_canonical_bootstrap_servers_only",
+ NULL};
static const char *tconfs[] = {"request.required.acks",
"-1", /* int */
"auto.commit.enable",
@@ -721,6 +720,75 @@ int main_0004_conf(int argc, char **argv) {
rd_kafka_conf_destroy(conf);
}
+#if WITH_OAUTHBEARER_OIDC
+ {
+ TEST_SAY(
+ "Verify that https.ca.location is mutually "
+ "exclusive with https.ca.pem\n");
+
+ conf = rd_kafka_conf_new();
+
+ test_conf_set(conf, "https.ca.pem",
+ "-----BEGIN CERTIFICATE-----");
+ test_conf_set(conf, "https.ca.location",
+ "/path/to/certificate.pem");
+
+ rk = rd_kafka_new(RD_KAFKA_PRODUCER, conf, errstr,
+ sizeof(errstr));
+ TEST_ASSERT(
+ !rk, "Expected rd_kafka_new() to fail, but it succeeded");
+ TEST_ASSERT(!strcmp(errstr,
+ "`https.ca.location` and "
+ "`https.ca.pem` are mutually exclusive"),
+ "Expected rd_kafka_new() to fail with: "
+ "\"`https.ca.location` and `https.ca.pem` "
+ "are mutually exclusive\", got: \"%s\"",
+ errstr);
+ rd_kafka_conf_destroy(conf);
+ }
+ {
+ TEST_SAY(
+ "Verify that https.ca.location gives an error when "
+ "set to an invalid path\n");
+
+ conf = rd_kafka_conf_new();
+
+ test_conf_set(conf, "https.ca.location",
+ "/?/does/!/not/exist!");
+
+ rk = rd_kafka_new(RD_KAFKA_PRODUCER, conf, errstr,
+ sizeof(errstr));
+ TEST_ASSERT(
+ !rk, "Expected rd_kafka_new() to fail, but it succeeded");
+ TEST_ASSERT(!strcmp(errstr,
+ "`https.ca.location` must be "
+ "an existing file or directory"),
+ "Expected rd_kafka_new() to fail with: "
+ "\"`https.ca.location` must be "
+ "an existing file or directory\", got: \"%s\"",
+ errstr);
+ rd_kafka_conf_destroy(conf);
+ }
+ {
+ TEST_SAY(
+ "Verify that https.ca.location doesn't give an error when "
+ "set to `probe`\n");
+
+ conf = rd_kafka_conf_new();
+
+ test_conf_set(conf, "https.ca.location", "probe");
+
+ rk = rd_kafka_new(RD_KAFKA_PRODUCER, conf, errstr,
+ sizeof(errstr));
+ TEST_ASSERT(
+ rk, "Expected rd_kafka_new() not to fail, but it failed");
+
+ rd_kafka_destroy(rk);
+ }
+#endif /* WITH_OAUTHBEARER_OIDC */
+
+ /* Verify that OpenSSL_AppLink is not needed on Windows (#3554) */
+
#ifdef _WIN32
{
FILE *fp;
@@ -774,26 +842,25 @@ int main_0004_conf(int argc, char **argv) {
const char *exp;
int is_global;
} props[] = {
- {"request.required.acks", "0", "0"},
- {"request.required.acks", "-1", "-1"},
- {"request.required.acks", "1", "1"},
- {"acks", "3", "3"}, /* alias test */
- {"request.required.acks", "393", "393"},
- {"request.required.acks", "bad", NULL},
- {"request.required.acks", "all", "-1"},
- {"request.required.acks", "all", "-1", 1 /*fallthru*/},
- {"acks", "0", "0"}, /* alias test */
+ {"request.required.acks", "0", "0"},
+ {"request.required.acks", "-1", "-1"},
+ {"request.required.acks", "1", "1"},
+ {"acks", "3", "3"}, /* alias test */
+ {"request.required.acks", "393", "393"},
+ {"request.required.acks", "bad", NULL},
+ {"request.required.acks", "all", "-1"},
+ {"request.required.acks", "all", "-1", 1 /*fallthru*/},
+ {"acks", "0", "0"}, /* alias test */
#if WITH_SASL
- {"sasl.mechanisms", "GSSAPI", "GSSAPI", 1},
- {"sasl.mechanisms", "PLAIN", "PLAIN", 1},
- {"sasl.mechanisms", "GSSAPI,PLAIN", NULL, 1},
- {"sasl.mechanisms", "", NULL, 1},
+ {"sasl.mechanisms", "GSSAPI", "GSSAPI", 1},
+ {"sasl.mechanisms", "PLAIN", "PLAIN", 1},
+ {"sasl.mechanisms", "GSSAPI,PLAIN", NULL, 1},
+ {"sasl.mechanisms", "", NULL, 1},
#endif
- {"linger.ms", "12555.3", "12555.3", 1},
- {"linger.ms", "1500.000", "1500", 1},
- {"linger.ms", "0.0001", "0.0001", 1},
- {NULL}
- };
+ {"linger.ms", "12555.3", "12555.3", 1},
+ {"linger.ms", "1500.000", "1500", 1},
+ {"linger.ms", "0.0001", "0.0001", 1},
+ {NULL}};
TEST_SAY("Canonical tests\n");
tconf = rd_kafka_topic_conf_new();
diff --git a/tests/0009-mock_cluster.c b/tests/0009-mock_cluster.c
index 07ab0e8864..a40fde2e2c 100644
--- a/tests/0009-mock_cluster.c
+++ b/tests/0009-mock_cluster.c
@@ -64,6 +64,7 @@ int main_0009_mock_cluster(int argc, char **argv) {
c = test_create_consumer(topic, NULL, conf, NULL);
rkt = test_create_producer_topic(p, topic, NULL);
+ test_wait_topic_exists(p, topic, 5000);
/* Produce */
test_produce_msgs(p, rkt, 0, RD_KAFKA_PARTITION_UA, 0, msgcnt, NULL, 0);
diff --git a/tests/0011-produce_batch.c b/tests/0011-produce_batch.c
index f745a6d310..f0c618bf88 100644
--- a/tests/0011-produce_batch.c
+++ b/tests/0011-produce_batch.c
@@ -366,7 +366,8 @@ static void test_per_message_partition_flag(void) {
TEST_SAY("test_per_message_partition_flag: Created kafka instance %s\n",
rd_kafka_name(rk));
topic_name = test_mk_topic_name("0011_per_message_flag", 1);
- test_create_topic(rk, topic_name, topic_num_partitions, 1);
+ test_create_topic_wait_exists(rk, topic_name, topic_num_partitions, 1,
+ 5000);
rkt = rd_kafka_topic_new(rk, topic_name, topic_conf);
if (!rkt)
diff --git a/tests/0016-client_swname.c b/tests/0016-client_swname.c
index 335925e328..6bf00b44ed 100644
--- a/tests/0016-client_swname.c
+++ b/tests/0016-client_swname.c
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2020-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -120,6 +121,11 @@ int main_0016_client_swname(int argc, char **argv) {
const char *jmx_port;
const char *reason = NULL;
+ if (test_needs_auth()) {
+ TEST_SKIP("Cannot run this test with SSL/SASL\n");
+ return 0;
+ }
+
/* If available, use the Kafka JmxTool to query software name
* in broker JMX metrics */
if (!(broker = test_getenv("BROKER_ADDRESS_2", NULL)))
@@ -136,14 +142,23 @@ int main_0016_client_swname(int argc, char **argv) {
reason =
"Env var BROKER_JMX_PORT_2 missing "
"(not running in trivup or trivup too old?)";
- else
+ else {
+ rd_bool_t apache_kafka_4 =
+ test_broker_version >= TEST_BRKVER(4, 0, 0, 0);
rd_snprintf(jmx_cmd, sizeof(jmx_cmd),
- "%s/bin/kafka-run-class.sh kafka.tools.JmxTool "
+ "%s/bin/kafka-run-class.sh %s "
"--jmx-url "
"service:jmx:rmi:///jndi/rmi://:%s/jmxrmi "
+ " --object-name '*:"
+ "clientSoftwareName=*,"
+ "clientSoftwareVersion=*,*' "
" --one-time true | "
"grep clientSoftware",
- kafka_path, jmx_port);
+ kafka_path,
+ apache_kafka_4 ? "org.apache.kafka.tools.JmxTool"
+ : "kafka.tools.JmxTool",
+ jmx_port);
+ }
if (reason)
TEST_WARN("Will not be able to verify JMX metrics: %s\n",
diff --git a/tests/0017-compression.c b/tests/0017-compression.c
index d13bb1bf6c..8cb295f25f 100644
--- a/tests/0017-compression.c
+++ b/tests/0017-compression.c
@@ -44,20 +44,17 @@ int main_0017_compression(int argc, char **argv) {
int msg_base = 0;
uint64_t testid;
#define CODEC_CNT 5
- const char *codecs[CODEC_CNT + 1] = {
- "none",
+ const char *codecs[CODEC_CNT + 1] = {"none",
#if WITH_ZLIB
- "gzip",
+ "gzip",
#endif
#if WITH_SNAPPY
- "snappy",
+ "snappy",
#endif
#if WITH_ZSTD
- "zstd",
+ "zstd",
#endif
- "lz4",
- NULL
- };
+ "lz4", NULL};
char *topics[CODEC_CNT];
const int32_t partition = 0;
int i;
@@ -77,6 +74,7 @@ int main_0017_compression(int argc, char **argv) {
msg_cnt, codecs[i], topics[i]);
rkt_p = test_create_producer_topic(
rk_p, topics[i], "compression.codec", codecs[i], NULL);
+ test_wait_topic_exists(rk_p, topics[i], 5000);
/* Produce small message that will not decrease with
* compression (issue #781) */
diff --git a/tests/0018-cgrp_term.c b/tests/0018-cgrp_term.c
index 85ac5612fb..ed7c2754b0 100644
--- a/tests/0018-cgrp_term.c
+++ b/tests/0018-cgrp_term.c
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2012-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -63,14 +64,16 @@ static void rebalance_cb(rd_kafka_t *rk,
switch (err) {
case RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS:
assign_cnt++;
- rd_kafka_assign(rk, partitions);
+ test_consumer_assign_by_rebalance_protocol("rebalance", rk,
+ partitions);
break;
case RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS:
if (assign_cnt == 0)
TEST_FAIL("asymetric rebalance_cb\n");
assign_cnt--;
- rd_kafka_assign(rk, NULL);
+ test_consumer_unassign_by_rebalance_protocol("rebalance", rk,
+ partitions);
break;
default:
@@ -195,6 +198,7 @@ static void do_test(rd_bool_t with_queue) {
/* Produce messages */
rk_p = test_create_producer();
rkt_p = test_create_producer_topic(rk_p, topic, NULL);
+ test_wait_topic_exists(rk_p, topic, 5000);
for (partition = 0; partition < partition_cnt; partition++) {
test_produce_msgs(rk_p, rkt_p, testid, partition,
@@ -260,7 +264,8 @@ static void do_test(rd_bool_t with_queue) {
/* Let remaining consumers run for a while to take over the now
* lost partitions. */
- if (test_consumer_group_protocol_generic() &&
+ /* Callback count can vary in KIP-848 */
+ if (test_consumer_group_protocol_classic() &&
assign_cnt != _CONS_CNT - 1)
TEST_FAIL("assign_cnt %d, should be %d\n", assign_cnt,
_CONS_CNT - 1);
diff --git a/tests/0021-rkt_destroy.c b/tests/0021-rkt_destroy.c
index f1517b8476..1b90041786 100644
--- a/tests/0021-rkt_destroy.c
+++ b/tests/0021-rkt_destroy.c
@@ -56,6 +56,7 @@ int main_0021_rkt_destroy(int argc, char **argv) {
testid = test_id_generate();
rk = test_create_producer();
rkt = test_create_producer_topic(rk, topic, NULL);
+ test_wait_topic_exists(rk, topic, 5000);
test_produce_msgs_nowait(rk, rkt, testid, RD_KAFKA_PARTITION_UA, 0,
diff --git a/tests/0022-consume_batch.c b/tests/0022-consume_batch.c
index 97d709201b..370877dba1 100644
--- a/tests/0022-consume_batch.c
+++ b/tests/0022-consume_batch.c
@@ -259,7 +259,10 @@ static void do_test_consume_batch_non_existent_topic(void) {
int main_0022_consume_batch(int argc, char **argv) {
do_test_consume_batch();
- if (test_consumer_group_protocol_generic()) {
+ /* Subscribing to a non-existent topic doesn't generate a
+ * UNKNOWN_TOPIC_OR_PART error with `consumer` group rebalance protocol.
+ */
+ if (test_consumer_group_protocol_classic()) {
do_test_consume_batch_non_existent_topic();
}
return 0;
diff --git a/tests/0026-consume_pause.c b/tests/0026-consume_pause.c
index 53f27ce11b..87119ae9c3 100644
--- a/tests/0026-consume_pause.c
+++ b/tests/0026-consume_pause.c
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2012-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -62,9 +63,7 @@ static void consume_pause(void) {
test_conf_set(conf, "enable.partition.eof", "true");
test_topic_conf_set(tconf, "auto.offset.reset", "smallest");
- test_create_topic(NULL, topic, partition_cnt, 1);
-
- test_wait_topic_exists(NULL, topic, 10 * 1000);
+ test_create_topic_wait_exists(NULL, topic, partition_cnt, 1, 10 * 1000);
/* Produce messages */
testid =
@@ -260,9 +259,8 @@ static void consume_pause_resume_after_reassign(void) {
test_conf_init(&conf, NULL, 60);
- test_create_topic(NULL, topic, (int)partition + 1, 1);
-
- test_wait_topic_exists(NULL, topic, 10 * 1000);
+ test_create_topic_wait_exists(NULL, topic, (int)partition + 1, 1,
+ 10 * 1000);
/* Produce messages */
testid = test_produce_msgs_easy(topic, 0, partition, msgcnt);
@@ -380,7 +378,8 @@ static void rebalance_cb(rd_kafka_t *rk,
* while auto.offset.reset is default at `latest`. */
parts->elems[0].offset = RD_KAFKA_OFFSET_BEGINNING;
- test_consumer_assign("rebalance", rk, parts);
+ test_consumer_assign_by_rebalance_protocol("rebalance", rk,
+ parts);
TEST_SAY("Pausing partitions\n");
if ((err2 = rd_kafka_pause_partitions(rk, parts)))
TEST_FAIL("Failed to pause: %s",
@@ -391,7 +390,8 @@ static void rebalance_cb(rd_kafka_t *rk,
rd_kafka_err2str(err2));
break;
default:
- test_consumer_unassign("rebalance", rk);
+ test_consumer_unassign_by_rebalance_protocol("rebalance", rk,
+ parts);
break;
}
}
@@ -419,9 +419,8 @@ static void consume_subscribe_assign_pause_resume(void) {
test_conf_init(&conf, NULL, 20);
- test_create_topic(NULL, topic, (int)partition + 1, 1);
-
- test_wait_topic_exists(NULL, topic, 10 * 1000);
+ test_create_topic_wait_exists(NULL, topic, (int)partition + 1, 1,
+ 10 * 1000);
/* Produce messages */
testid = test_produce_msgs_easy(topic, 0, partition, msgcnt);
@@ -472,9 +471,8 @@ static void consume_seek_pause_resume(void) {
test_conf_init(&conf, NULL, 20);
- test_create_topic(NULL, topic, (int)partition + 1, 1);
-
- test_wait_topic_exists(NULL, topic, 10 * 1000);
+ test_create_topic_wait_exists(NULL, topic, (int)partition + 1, 1,
+ 10 * 1000);
/* Produce messages */
testid = test_produce_msgs_easy(topic, 0, partition, msgcnt);
diff --git a/tests/0028-long_topicnames.c b/tests/0028-long_topicnames.c
index a20f4308b5..3649805ee7 100644
--- a/tests/0028-long_topicnames.c
+++ b/tests/0028-long_topicnames.c
@@ -62,7 +62,7 @@ int main_0028_long_topicnames(int argc, char **argv) {
rk_c = test_create_consumer(topic, NULL, NULL, NULL);
/* Create topic */
- test_create_topic(rk_c, topic, 1, 1);
+ test_create_topic_wait_exists(rk_c, topic, 1, 1, 5000);
test_consumer_subscribe(rk_c, topic);
test_consumer_poll_no_msgs("consume.nomsgs", rk_c, 0, 5000);
diff --git a/tests/0029-assign_offset.c b/tests/0029-assign_offset.c
index 1d1edd114f..f4ab247e53 100644
--- a/tests/0029-assign_offset.c
+++ b/tests/0029-assign_offset.c
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2012-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -89,11 +90,13 @@ static void rebalance_cb(rd_kafka_t *rk,
}
TEST_SAY("Use these offsets:\n");
test_print_partition_list(parts);
- test_consumer_assign("HL.REBALANCE", rk, parts);
+ test_consumer_assign_by_rebalance_protocol("HL.REBALANCE", rk,
+ parts);
break;
case RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS:
- test_consumer_unassign("HL.REBALANCE", rk);
+ test_consumer_unassign_by_rebalance_protocol("HL.REBALANCE", rk,
+ parts);
break;
default:
@@ -111,12 +114,6 @@ int main_0029_assign_offset(int argc, char **argv) {
test_timing_t t_simple, t_hl;
test_msgver_t mv;
- if (!test_consumer_group_protocol_generic()) {
- /* FIXME: this should be fixed when upgrading from generic to
- * new consumer group will be possible. See KAFKA-15989 */
- return 0;
- }
-
test_conf_init(NULL, NULL, 20 + (test_session_timeout_ms * 3 / 1000));
/* Produce X messages to Y partitions so we get a
@@ -125,6 +122,7 @@ int main_0029_assign_offset(int argc, char **argv) {
testid = test_id_generate();
rk = test_create_producer();
rkt = test_create_producer_topic(rk, topic, NULL);
+ test_wait_topic_exists(rk, topic, 5000);
parts = rd_kafka_topic_partition_list_new(partitions);
diff --git a/tests/0030-offset_commit.c b/tests/0030-offset_commit.c
index e53b0aefe4..735021e54c 100644
--- a/tests/0030-offset_commit.c
+++ b/tests/0030-offset_commit.c
@@ -429,7 +429,7 @@ static void do_empty_commit(void) {
TEST_SAY("First commit\n");
expect = RD_KAFKA_RESP_ERR_NO_ERROR;
err = rd_kafka_commit_queue(rk, NULL, NULL, empty_offset_commit_cb,
- &expect);
+ &expect);
if (err != expect)
TEST_FAIL("commit failed: %s", rd_kafka_err2str(err));
else
@@ -438,7 +438,7 @@ static void do_empty_commit(void) {
TEST_SAY("Second commit, should be empty\n");
expect = RD_KAFKA_RESP_ERR__NO_OFFSET;
err = rd_kafka_commit_queue(rk, NULL, NULL, empty_offset_commit_cb,
- &expect);
+ &expect);
if (err != RD_KAFKA_RESP_ERR__NO_OFFSET)
TEST_FAIL("unexpected commit result, wanted NO_OFFSET, got: %s",
rd_kafka_err2str(err));
diff --git a/tests/0033-regex_subscribe.c b/tests/0033-regex_subscribe.c
index 0919f70519..9800ebe7ea 100644
--- a/tests/0033-regex_subscribe.c
+++ b/tests/0033-regex_subscribe.c
@@ -47,13 +47,14 @@ struct expect {
int exp_err; /* expected error from subscribe() */
int stat[4]; /* per exp status */
int fails;
- enum { _EXP_NONE,
- _EXP_FAIL,
- _EXP_OK,
- _EXP_ASSIGN,
- _EXP_REVOKE,
- _EXP_ASSIGNED,
- _EXP_REVOKED,
+ enum {
+ _EXP_NONE,
+ _EXP_FAIL,
+ _EXP_OK,
+ _EXP_ASSIGN,
+ _EXP_REVOKE,
+ _EXP_ASSIGNED,
+ _EXP_REVOKED,
} result;
};
@@ -135,7 +136,8 @@ static void rebalance_cb(rd_kafka_t *rk,
exp->name, exp->result);
}
expect_match(exp, parts);
- test_consumer_assign("rebalance", rk, parts);
+ test_consumer_assign_by_rebalance_protocol("rebalance", rk,
+ parts);
exp->result = _EXP_ASSIGNED;
break;
@@ -147,7 +149,8 @@ static void rebalance_cb(rd_kafka_t *rk,
exp->name, exp->result);
}
- test_consumer_unassign("rebalance", rk);
+ test_consumer_unassign_by_rebalance_protocol("rebalance", rk,
+ parts);
exp->result = _EXP_REVOKED;
break;
@@ -374,7 +377,7 @@ static int do_test(const char *assignor) {
.name = rd_strdup(tsprintf("%s: regex 0&1", assignor)),
.sub = {rd_strdup(tsprintf(
"^.*[tToOpPiIcC]_0+[12]_[^_]+_%s", groupid)),
- NULL},
+ NULL},
.exp = {topics[0], topics[1], NULL}};
fails += test_subscribe(rk, &expect);
@@ -387,7 +390,7 @@ static int do_test(const char *assignor) {
.name = rd_strdup(tsprintf("%s: regex 2", assignor)),
.sub = {rd_strdup(
tsprintf("^.*TOOTHPIC_000._._%s", groupid)),
- NULL},
+ NULL},
.exp = {topics[2], NULL}};
fails += test_subscribe(rk, &expect);
@@ -401,7 +404,7 @@ static int do_test(const char *assignor) {
"nonexistent(not seen)",
assignor)),
.sub = {rd_strdup(tsprintf("^.*_000[34]_..?_%s", groupid)),
- NULL},
+ NULL},
.exp = {topics[2], NULL}};
fails += test_subscribe(rk, &expect);
@@ -421,9 +424,23 @@ static int do_test(const char *assignor) {
rd_free(expect.name);
}
+ {
+ struct expect expect = {
+ .name = rd_strdup(
+ tsprintf("%s: multiple regex 1&2 matches", assignor)),
+ .sub = {"^.*regex_subscribe_to.*",
+ "^.*regex_subscribe_TOO.*", NULL},
+ .exp = {topics[1], topics[2], NULL}};
+
+ fails += test_subscribe(rk, &expect);
+ rd_free(expect.name);
+ }
test_consumer_close(rk);
+ for (i = 0; i < topic_cnt; i++)
+ test_delete_topic(rk, topics[i]);
+
rd_kafka_destroy(rk);
if (fails)
@@ -434,11 +451,10 @@ static int do_test(const char *assignor) {
int main_0033_regex_subscribe(int argc, char **argv) {
- if (test_consumer_group_protocol_generic()) {
- /* FIXME: when regexes will be supported by KIP-848 */
- do_test("range");
- do_test("roundrobin");
- }
+
+ do_test("range");
+ do_test("roundrobin");
+
return 0;
}
diff --git a/tests/0034-offset_reset.c b/tests/0034-offset_reset.c
index 4a6a58f4dc..5efcd35893 100644
--- a/tests/0034-offset_reset.c
+++ b/tests/0034-offset_reset.c
@@ -230,6 +230,7 @@ static void offset_reset_errors(void) {
for (i = 0; i < (int)RD_ARRAYSIZE(test); i++) {
rd_kafka_event_t *ev;
rd_bool_t broker_down = rd_false;
+ rd_ts_t abs_timeout, now;
/* Make sure consumer is connected */
test_wait_topic_exists(c, topic, 5000);
@@ -259,13 +260,32 @@ static void offset_reset_errors(void) {
test_consumer_assign_partition("ASSIGN", c, topic, partition,
test[i].start_offset);
+ /* 5 seconds timeout */
+ abs_timeout = test_clock() + 5 * 1000000;
while (1) {
+ rd_bool_t not_auto_offset_reset_error;
/* Poll until we see an AUTO_OFFSET_RESET error,
* timeout, or a message, depending on what we're
* looking for. */
- ev = rd_kafka_queue_poll(queue, 5000);
+ ev = rd_kafka_queue_poll(queue, 5000);
+ now = test_clock();
+ not_auto_offset_reset_error =
+ ev &&
+ rd_kafka_event_type(ev) == RD_KAFKA_EVENT_ERROR &&
+ rd_kafka_event_error(ev) !=
+ RD_KAFKA_RESP_ERR__AUTO_OFFSET_RESET;
+
+ if (!ev || (not_auto_offset_reset_error &&
+ now > abs_timeout)) {
+ if (ev) {
+ TEST_SAY(
+ "#%d: Ignoring %s event: %s\n", i,
+ rd_kafka_event_name(ev),
+ rd_kafka_event_error_string(ev));
+ rd_kafka_event_destroy(ev);
+ ev = NULL;
+ }
- if (!ev) {
TEST_ASSERT(broker_down,
"#%d: poll timeout, but broker "
"was not down",
@@ -278,12 +298,13 @@ static void offset_reset_errors(void) {
broker_id);
broker_down = rd_false;
+ /* 5 seconds timeout */
+ abs_timeout = test_clock() + 5 * 1000000;
} else if (rd_kafka_event_type(ev) ==
RD_KAFKA_EVENT_ERROR) {
- if (rd_kafka_event_error(ev) !=
- RD_KAFKA_RESP_ERR__AUTO_OFFSET_RESET) {
+ if (not_auto_offset_reset_error) {
TEST_SAY(
"#%d: Ignoring %s event: %s\n", i,
rd_kafka_event_name(ev),
diff --git a/tests/0036-partial_fetch.c b/tests/0036-partial_fetch.c
index 50c64c35c6..6f0d086711 100644
--- a/tests/0036-partial_fetch.c
+++ b/tests/0036-partial_fetch.c
@@ -59,6 +59,7 @@ int main_0036_partial_fetch(int argc, char **argv) {
testid = test_id_generate();
rk = test_create_producer();
rkt = test_create_producer_topic(rk, topic, NULL);
+ test_wait_topic_exists(rk, topic, 5000);
test_produce_msgs(rk, rkt, testid, partition, 0, msgcnt, NULL, msgsize);
diff --git a/tests/0038-performance.c b/tests/0038-performance.c
index c795354637..9642e8352a 100644
--- a/tests/0038-performance.c
+++ b/tests/0038-performance.c
@@ -68,6 +68,7 @@ int main_0038_performance(int argc, char **argv) {
test_conf_set(conf, "linger.ms", "100");
rk = test_create_handle(RD_KAFKA_PRODUCER, conf);
rkt = test_create_producer_topic(rk, topic, "acks", "1", NULL);
+ test_wait_topic_exists(rk, topic, 5000);
/* First produce one message to create the topic, etc, this might take
* a while and we dont want this to affect the throughput timing. */
diff --git a/tests/0040-io_event.c b/tests/0040-io_event.c
index fba8f9d3b9..c7cd44ca21 100644
--- a/tests/0040-io_event.c
+++ b/tests/0040-io_event.c
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2012-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -74,7 +75,8 @@ int main_0040_io_event(int argc, char **argv) {
rk_p = test_create_producer();
rkt_p = test_create_producer_topic(rk_p, topic, NULL);
- err = test_auto_create_topic_rkt(rk_p, rkt_p, tmout_multip(5000));
+ test_wait_topic_exists(rk_p, topic, 5000);
+ err = test_auto_create_topic_rkt(rk_p, rkt_p, tmout_multip(5000));
TEST_ASSERT(!err, "Topic auto creation failed: %s",
rd_kafka_err2str(err));
@@ -168,13 +170,17 @@ int main_0040_io_event(int argc, char **argv) {
"expecting message\n");
if (rd_kafka_event_error(rkev) ==
RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS) {
- rd_kafka_assign(
- rk_c,
+ test_consumer_assign_by_rebalance_protocol(
+ "rebalance event", rk_c,
rd_kafka_event_topic_partition_list(
rkev));
expecting_io = _NOPE;
- } else
- rd_kafka_assign(rk_c, NULL);
+ } else {
+ test_consumer_unassign_by_rebalance_protocol(
+ "rebalance event", rk_c,
+ rd_kafka_event_topic_partition_list(
+ rkev));
+ }
break;
case RD_KAFKA_EVENT_FETCH:
diff --git a/tests/0041-fetch_max_bytes.c b/tests/0041-fetch_max_bytes.c
index 75ea4f80cc..bafa931c24 100644
--- a/tests/0041-fetch_max_bytes.c
+++ b/tests/0041-fetch_max_bytes.c
@@ -61,6 +61,7 @@ int main_0041_fetch_max_bytes(int argc, char **argv) {
testid = test_id_generate();
rk = test_create_producer();
rkt = test_create_producer_topic(rk, topic, NULL);
+ test_wait_topic_exists(rk, topic, 5000);
test_produce_msgs(rk, rkt, testid, partition, 0, msgcnt / 2, NULL,
MAX_BYTES / 10);
diff --git a/tests/0044-partition_cnt.c b/tests/0044-partition_cnt.c
index b4b66bd482..64df57affb 100644
--- a/tests/0044-partition_cnt.c
+++ b/tests/0044-partition_cnt.c
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2012-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -60,10 +61,10 @@ static void test_producer_partition_cnt_change(void) {
rd_kafka_conf_set_dr_msg_cb(conf, test_dr_msg_cb);
rk = test_create_handle(RD_KAFKA_PRODUCER, conf);
- test_create_topic(rk, topic, partition_cnt / 2, 1);
+ test_create_topic_wait_exists(rk, topic, partition_cnt / 2, 1, 5000);
rkt =
- test_create_topic_object(rk, __FUNCTION__, "message.timeout.ms",
+ test_create_topic_object(rk, topic, "message.timeout.ms",
tsprintf("%d", tmout_multip(10000)), NULL);
test_produce_msgs_nowait(rk, rkt, 0, RD_KAFKA_PARTITION_UA, 0,
diff --git a/tests/0045-subscribe_update.c b/tests/0045-subscribe_update.c
index c4daa4780f..6be18d1e0f 100644
--- a/tests/0045-subscribe_update.c
+++ b/tests/0045-subscribe_update.c
@@ -101,7 +101,7 @@ static void await_assignment(const char *pfx,
if (fails > 0)
TEST_FAIL("%s: assignment mismatch: see above", pfx);
- rd_kafka_assign(rk, tps);
+ test_consumer_assign_by_rebalance_protocol("rebalance event", rk, tps);
rd_kafka_event_destroy(rkev);
}
@@ -112,6 +112,7 @@ static void await_assignment(const char *pfx,
static void
await_revoke(const char *pfx, rd_kafka_t *rk, rd_kafka_queue_t *queue) {
rd_kafka_event_t *rkev;
+ rd_kafka_topic_partition_list_t *tps;
TEST_SAY("%s: waiting for revoke\n", pfx);
rkev = test_wait_event(queue, RD_KAFKA_EVENT_REBALANCE, 30000);
@@ -121,7 +122,10 @@ await_revoke(const char *pfx, rd_kafka_t *rk, rd_kafka_queue_t *queue) {
RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS,
"expected REVOKE, got %s",
rd_kafka_err2str(rd_kafka_event_error(rkev)));
- rd_kafka_assign(rk, NULL);
+ tps = rd_kafka_event_topic_partition_list(rkev);
+
+ test_consumer_unassign_by_rebalance_protocol("rebalance event", rk,
+ tps);
rd_kafka_event_destroy(rkev);
}
@@ -231,7 +235,7 @@ static void do_test_non_exist_and_partchange(void) {
await_no_rebalance("#1: empty", rk, queue, 10000);
TEST_SAY("#1: creating topic %s\n", topic_a);
- test_create_topic(NULL, topic_a, 2, 1);
+ test_create_topic_wait_exists(NULL, topic_a, 2, 1, 5000);
await_assignment("#1: proper", rk, queue, 1, topic_a, 2);
@@ -266,6 +270,7 @@ static void do_test_regex(void) {
rd_kafka_t *rk;
rd_kafka_conf_t *conf;
rd_kafka_queue_t *queue;
+ const char *rebalance_protocol;
/**
* Regex test:
@@ -290,8 +295,7 @@ static void do_test_regex(void) {
queue = rd_kafka_queue_get_consumer(rk);
TEST_SAY("Regex: creating topic %s (subscribed)\n", topic_b);
- test_create_topic(NULL, topic_b, 2, 1);
- rd_sleep(1); // FIXME: do check&wait loop instead
+ test_create_topic_wait_exists(NULL, topic_b, 2, 1, 5000);
TEST_SAY("Regex: Subscribing to %s & %s & %s\n", topic_b, topic_d,
topic_e);
@@ -301,18 +305,26 @@ static void do_test_regex(void) {
2);
TEST_SAY("Regex: creating topic %s (not subscribed)\n", topic_c);
- test_create_topic(NULL, topic_c, 4, 1);
+ test_create_topic_wait_exists(NULL, topic_c, 4, 1, 5000);
/* Should not see a rebalance since no topics are matched. */
await_no_rebalance("Regex: empty", rk, queue, 10000);
TEST_SAY("Regex: creating topic %s (subscribed)\n", topic_d);
- test_create_topic(NULL, topic_d, 1, 1);
-
- await_revoke("Regex: rebalance after topic creation", rk, queue);
-
- await_assignment("Regex: two topics exist", rk, queue, 2, topic_b, 2,
- topic_d, 1);
+ test_create_topic_wait_exists(NULL, topic_d, 1, 1, 5000);
+
+ if (test_consumer_group_protocol_classic())
+ await_revoke("Regex: rebalance after topic creation", rk,
+ queue);
+
+ rebalance_protocol = rd_kafka_rebalance_protocol(rk);
+ if (!strcmp(rebalance_protocol, "COOPERATIVE")) {
+ await_assignment("Regex: two topics exist", rk, queue, 1,
+ topic_d, 1);
+ } else {
+ await_assignment("Regex: two topics exist", rk, queue, 2,
+ topic_b, 2, topic_d, 1);
+ }
test_consumer_close(rk);
rd_kafka_queue_destroy(queue);
@@ -364,12 +376,10 @@ static void do_test_topic_remove(void) {
queue = rd_kafka_queue_get_consumer(rk);
TEST_SAY("Topic removal: creating topic %s (subscribed)\n", topic_f);
- test_create_topic(NULL, topic_f, parts_f, 1);
+ test_create_topic_wait_exists(NULL, topic_f, parts_f, 1, 5000);
TEST_SAY("Topic removal: creating topic %s (subscribed)\n", topic_g);
- test_create_topic(NULL, topic_g, parts_g, 1);
-
- rd_sleep(1); // FIXME: do check&wait loop instead
+ test_create_topic_wait_exists(NULL, topic_g, parts_g, 1, 5000);
TEST_SAY("Topic removal: Subscribing to %s & %s\n", topic_f, topic_g);
topics = rd_kafka_topic_partition_list_new(2);
@@ -493,7 +503,7 @@ static void do_test_replica_rack_change_mock(const char *assignment_strategy,
const char *topic = "topic";
const char *test_name = tsprintf(
"Replica rack changes (%s, subscription = \"%s\", %s client.rack, "
- "%s replica.rack)",
+ "%s replica.rack)",
assignment_strategy, subscription,
use_client_rack ? "with" : "without",
use_replica_rack ? "with" : "without");
@@ -677,6 +687,242 @@ static void do_test_replica_rack_change_leader_no_rack_mock(
SUB_TEST_PASS();
}
+/**
+ * Checking assignments and revocation operations for subscribe and
+ * unsubcribe with regular topic names and regex.
+ */
+static void do_test_resubscribe_with_regex() {
+ char *topic1 = rd_strdup(test_mk_topic_name("topic_regex1", 1));
+ char *topic2 = rd_strdup(test_mk_topic_name("topic_regex2", 1));
+ char *topic_a = rd_strdup(test_mk_topic_name("topic_a", 1));
+ char *group = rd_strdup(
+ tsprintf("group_test_sub_regex_%s", test_str_id_generate_tmp()));
+ rd_kafka_t *rk;
+ rd_kafka_conf_t *conf;
+ rd_kafka_queue_t *queue;
+
+ SUB_TEST("Resubscribe with Regex");
+
+ /**
+ * Topic resubscribe with regex test:
+ * - Create topic topic_regex1 & topic_regex2
+ * - Subscribe to topic_regex1
+ * - Verify topic_regex1 assignment
+ * - Unsubscribe
+ * - Verify revocation
+ * - Subscribe to topic_regex2
+ * - Verify topic_regex2 assignment
+ * - Unsubscribe
+ * - Verify revocation
+ * - Subscribe to regex ^.*topic_regex.*
+ * - Verify topic_regex1 & topic_regex2 assignment
+ * - Unsubscribe
+ * - Verify revocation
+ * - Subscribe to regex ^.*topic_regex.* and topic_a
+ * - Verify topic_regex1, topic_regex2 and topic_a assignment
+ * - Unsubscribe
+ * - Verify revocation
+ */
+
+ TEST_SAY("Creating topic %s\n", topic1);
+ test_create_topic_wait_exists(NULL, topic1, 4, 1, 5000);
+
+ TEST_SAY("Creating topic %s\n", topic2);
+ test_create_topic_wait_exists(NULL, topic2, 4, 1, 5000);
+
+ TEST_SAY("Creating topic %s\n", topic_a);
+ test_create_topic_wait_exists(NULL, topic_a, 2, 1, 5000);
+
+ test_conf_init(&conf, NULL, 60);
+
+ rd_kafka_conf_set_events(conf, RD_KAFKA_EVENT_REBALANCE);
+ rk = test_create_consumer(group, NULL, conf, NULL);
+ queue = rd_kafka_queue_get_consumer(rk);
+
+ /* Subscribe to topic1 */
+ TEST_SAY("Subscribing to %s\n", topic1);
+ test_consumer_subscribe(rk, topic1);
+ /* Wait for assignment */
+ await_assignment("Assignment for topic1", rk, queue, 1, topic1, 4);
+
+ /* Unsubscribe from topic1 */
+ TEST_SAY("Unsubscribing from %s\n", topic1);
+ rd_kafka_unsubscribe(rk);
+ /* Wait for revocation */
+ await_revoke("Revocation after unsubscribing", rk, queue);
+
+ /* Subscribe to topic2 */
+ TEST_SAY("Subscribing to %s\n", topic2);
+ test_consumer_subscribe(rk, topic2);
+ /* Wait for assignment */
+ await_assignment("Assignment for topic2", rk, queue, 1, topic2, 4);
+
+ /* Unsubscribe from topic2 */
+ TEST_SAY("Unsubscribing from %s\n", topic2);
+ rd_kafka_unsubscribe(rk);
+ /* Wait for revocation */
+ await_revoke("Revocation after unsubscribing", rk, queue);
+
+ /* Subscribe to regex ^.*topic_regex.* */
+ TEST_SAY("Subscribing to regex ^.*topic_regex.*\n");
+ test_consumer_subscribe(rk, "^.*topic_regex.*");
+ if (!test_consumer_group_protocol_classic()) {
+ /** Regex matching is async on the broker side for KIP-848
+ * protocol. */
+ rd_sleep(5);
+ }
+ /* Wait for assignment */
+ await_assignment("Assignment for topic1 and topic2", rk, queue, 2,
+ topic1, 4, topic2, 4);
+
+ /* Unsubscribe from regex ^.*topic_regex.* */
+ TEST_SAY("Unsubscribing from regex ^.*topic_regex.*\n");
+ rd_kafka_unsubscribe(rk);
+ /* Wait for revocation */
+ await_revoke("Revocation after unsubscribing", rk, queue);
+
+ /* Subscribe to regex ^.*topic_regex.* and topic_a literal */
+ TEST_SAY("Subscribing to regex ^.*topic_regex.* and topic_a\n");
+ test_consumer_subscribe_multi(rk, 2, "^.*topic_regex.*", topic_a);
+ /* Wait for assignment */
+ if (test_consumer_group_protocol_classic()) {
+ await_assignment("Assignment for topic1, topic2 and topic_a",
+ rk, queue, 3, topic1, 4, topic2, 4, topic_a,
+ 2);
+ } else {
+ /* KIP-848 broker is assigning literal topics first in 1 HB call
+ * and all the matched ones in later HB call*/
+ await_assignment("Assignment for topic_a", rk, queue, 1,
+ topic_a, 2);
+ await_assignment("Assignment for topic1 and topic2", rk, queue,
+ 2, topic1, 4, topic2, 4);
+ }
+
+ /* Unsubscribe */
+ TEST_SAY("Unsubscribing\n");
+ rd_kafka_unsubscribe(rk);
+ await_revoke("Revocation after unsubscribing", rk, queue);
+
+ /* Cleanup */
+ test_delete_topic(rk, topic1);
+ test_delete_topic(rk, topic2);
+ test_delete_topic(rk, topic_a);
+
+ test_consumer_close(rk);
+ rd_kafka_queue_destroy(queue);
+
+ rd_kafka_destroy(rk);
+
+ rd_free(topic1);
+ rd_free(topic2);
+ rd_free(topic_a);
+ rd_free(group);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief Create many topics and apply several subscription
+ * updates, unsubscribing and re-subscribing too.
+ * After changing some subscriptions verifies that the assignment
+ * corresponds to last one.
+ *
+ * @param with_rebalance_cb Use a rebalance callback to perform the assignment.
+ * It needs to poll the consumer when awaiting for the
+ * assignment in this case.
+ */
+static void do_test_subscribe_many_updates(rd_bool_t with_rebalance_cb) {
+#define TOPIC_CNT 100
+ char *topics[TOPIC_CNT] = {0};
+ char *topic;
+ char *group;
+ size_t i;
+ rd_kafka_t *rk;
+ rd_kafka_conf_t *conf;
+ const int partition_cnt = 4;
+
+ SUB_TEST("%s", with_rebalance_cb ? "with rebalance callback"
+ : "without rebalance callback");
+
+ RD_ARRAY_FOREACH_INDEX(topic, topics, i) {
+ char topic_i[17];
+ rd_snprintf(topic_i, sizeof(topic_i), "topic%" PRIusz, i);
+ topics[i] = rd_strdup(test_mk_topic_name(topic_i, 1));
+ };
+ group = topics[0];
+
+ test_conf_init(&conf, NULL, 60);
+ if (with_rebalance_cb)
+ rd_kafka_conf_set_rebalance_cb(conf, test_rebalance_cb);
+ rk = test_create_consumer(group, NULL, conf, NULL);
+
+ TEST_SAY("Creating %d topics\n", TOPIC_CNT);
+ TEST_CALL_ERR__(test_CreateTopics_simple(rk, NULL, topics, TOPIC_CNT,
+ partition_cnt, NULL));
+ test_wait_topic_exists(rk, topics[TOPIC_CNT - 1], 5000);
+ /* Give the cluster some more time to propagate metadata
+ * for TOPICS_CNT topics */
+ rd_sleep(1);
+
+ RD_ARRAY_FOREACH_INDEX(topic, topics, i) {
+ const int max_subscription_size = 5;
+ size_t j;
+ int k;
+ int subscription_size =
+ RD_MIN(max_subscription_size, TOPIC_CNT - i);
+ int expected_assignment_cnt = subscription_size * partition_cnt;
+ rd_kafka_topic_partition_list_t *expected_assignment = NULL;
+
+ rd_kafka_topic_partition_list_t *subscription =
+ rd_kafka_topic_partition_list_new(subscription_size);
+ rd_bool_t check_expected_assignment =
+ (i % 5 == 0 || i == TOPIC_CNT - 1);
+ rd_bool_t do_unsubscribe = i % 7 == 0;
+
+ if (check_expected_assignment)
+ expected_assignment = rd_kafka_topic_partition_list_new(
+ expected_assignment_cnt);
+
+ for (j = i; j < i + subscription_size; j++) {
+ rd_kafka_topic_partition_list_add(
+ subscription, topics[j], RD_KAFKA_PARTITION_UA);
+ /* We unsubscribe every 7 iteration and
+ * we check assignments every 5 iteration so
+ * at 7 * 5 we unsubscribe and check that assignment
+ * is empty. */
+ if (check_expected_assignment && !do_unsubscribe)
+ for (k = 0; k < partition_cnt; k++)
+ rd_kafka_topic_partition_list_add(
+ expected_assignment, topics[j], k);
+ }
+ TEST_CALL_ERR__(rd_kafka_subscribe(rk, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ if (do_unsubscribe)
+ TEST_CALL_ERR__(rd_kafka_unsubscribe(rk));
+ if (check_expected_assignment) {
+ test_consumer_wait_assignment_topic_partition_list(
+ rk,
+ /* poll when we have a rebalance callback */
+ with_rebalance_cb, expected_assignment, 30000);
+ rd_kafka_topic_partition_list_destroy(
+ expected_assignment);
+ }
+ };
+
+ TEST_CALL_ERR__(
+ test_DeleteTopics_simple(rk, NULL, topics, TOPIC_CNT, NULL));
+ RD_ARRAY_FOREACH_INDEX(topic, topics, i) {
+ rd_free(topic);
+ };
+
+ test_consumer_close(rk);
+ rd_kafka_destroy(rk);
+
+ SUB_TEST_PASS();
+#undef TOPIC_CNT
+}
+
int main_0045_subscribe_update(int argc, char **argv) {
if (!test_can_create_topics(1))
@@ -706,6 +952,8 @@ int main_0045_subscribe_update_topic_remove(int argc, char **argv) {
int main_0045_subscribe_update_mock(int argc, char **argv) {
+ TEST_SKIP_MOCK_CLUSTER(0);
+
do_test_regex_many_mock("range", rd_false);
do_test_regex_many_mock("cooperative-sticky", rd_false);
do_test_regex_many_mock("cooperative-sticky", rd_true);
@@ -713,6 +961,16 @@ int main_0045_subscribe_update_mock(int argc, char **argv) {
return 0;
}
+int main_0045_resubscribe_with_regex(int argc, char **argv) {
+ do_test_resubscribe_with_regex();
+ return 0;
+}
+
+int main_0045_subscribe_many_updates(int argc, char **argv) {
+ do_test_subscribe_many_updates(rd_false);
+ do_test_subscribe_many_updates(rd_true);
+ return 0;
+}
int main_0045_subscribe_update_racks_mock(int argc, char **argv) {
int use_replica_rack = 0;
@@ -720,6 +978,12 @@ int main_0045_subscribe_update_racks_mock(int argc, char **argv) {
TEST_SKIP_MOCK_CLUSTER(0);
+ /* KIP 848 Mock broker assignor isn't rack-aware. */
+ if (!test_consumer_group_protocol_classic()) {
+ TEST_SKIP("Test meaningful only with classic protocol\n");
+ return 0;
+ }
+
for (use_replica_rack = 0; use_replica_rack < 2; use_replica_rack++) {
for (use_client_rack = 0; use_client_rack < 2;
use_client_rack++) {
@@ -737,9 +1001,9 @@ int main_0045_subscribe_update_racks_mock(int argc, char **argv) {
use_client_rack, use_replica_rack);
}
}
-
- /* Do not test with range assignor (yet) since it does not do rack aware
- * assignment properly with the NULL rack, even for the Java client. */
+ /* Do not test with range assignor (yet) since it does not do
+ * rack aware assignment properly with the NULL rack, even for
+ * the Java client. */
do_test_replica_rack_change_leader_no_rack_mock("cooperative-sticky");
return 0;
diff --git a/tests/0047-partial_buf_tmout.c b/tests/0047-partial_buf_tmout.c
index e999afa367..e91a89863b 100644
--- a/tests/0047-partial_buf_tmout.c
+++ b/tests/0047-partial_buf_tmout.c
@@ -81,6 +81,7 @@ int main_0047_partial_buf_tmout(int argc, char **argv) {
rkt = test_create_producer_topic(rk, topic, "message.timeout.ms", "300",
NULL);
+ test_wait_topic_exists(rk, topic, 5000);
while (got_timeout_err == 0) {
test_produce_msgs_nowait(rk, rkt, 0, RD_KAFKA_PARTITION_UA, 0,
diff --git a/tests/0048-partitioner.c b/tests/0048-partitioner.c
index 63761506c5..638bbf83e8 100644
--- a/tests/0048-partitioner.c
+++ b/tests/0048-partitioner.c
@@ -267,7 +267,7 @@ static void do_test_partitioners(void) {
int pi;
const char *topic = test_mk_topic_name(__FUNCTION__, 1);
- test_create_topic(NULL, topic, part_cnt, 1);
+ test_create_topic_wait_exists(NULL, topic, part_cnt, 1, 5000);
for (pi = 0; ptest[pi].partitioner; pi++) {
do_test_partitioner(topic, ptest[pi].partitioner, _MSG_CNT,
diff --git a/tests/0050-subscribe_adds.c b/tests/0050-subscribe_adds.c
index acde518e47..5802ec8159 100644
--- a/tests/0050-subscribe_adds.c
+++ b/tests/0050-subscribe_adds.c
@@ -74,6 +74,7 @@ test_no_duplicate_messages(const char *partition_assignment_strategy) {
rd_kafka_topic_t *rkt;
rkt = test_create_producer_topic(rk, topic[i], NULL);
+ test_wait_topic_exists(rk, topic[i], 5000);
test_produce_msgs(rk, rkt, testid, RD_KAFKA_PARTITION_UA,
(msgcnt / TOPIC_CNT) * i,
diff --git a/tests/0051-assign_adds.c b/tests/0051-assign_adds.c
index 31866627dd..516cadcab4 100644
--- a/tests/0051-assign_adds.c
+++ b/tests/0051-assign_adds.c
@@ -68,6 +68,7 @@ int main_0051_assign_adds(int argc, char **argv) {
rd_kafka_topic_t *rkt;
rkt = test_create_producer_topic(rk, topic[i], NULL);
+ test_wait_topic_exists(rk, topic[i], 5000);
test_produce_msgs(rk, rkt, testid, 0, (msgcnt / TOPIC_CNT) * i,
(msgcnt / TOPIC_CNT), NULL, 100);
diff --git a/tests/0052-msg_timestamps.c b/tests/0052-msg_timestamps.c
index 7921cd4594..26c2a464b4 100644
--- a/tests/0052-msg_timestamps.c
+++ b/tests/0052-msg_timestamps.c
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2012-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -165,6 +166,7 @@ static void test_timestamps(const char *broker_tstype,
"--replication-factor 1 --partitions 1 "
"--config message.timestamp.type=%s",
topic, broker_tstype);
+ test_wait_topic_exists(NULL, topic, 5000);
TEST_SAY(_C_MAG "Producing %d messages to %s\n", msgcnt, topic);
produce_msgs(topic, 0, testid, msgcnt, broker_version, codec);
@@ -180,6 +182,7 @@ static void test_timestamps(const char *broker_tstype,
int main_0052_msg_timestamps(int argc, char **argv) {
+ rd_bool_t test_with_apache_kafka_since_4_0 = rd_false;
if (!test_can_create_topics(1))
return 0;
@@ -189,6 +192,13 @@ int main_0052_msg_timestamps(int argc, char **argv) {
return 0;
}
+ if (test_broker_version >= TEST_BRKVER(4, 0, 0, 0)) {
+ /* Code using `broker.version.fallback` and
+ * `api.version.request=false` won't work
+ * against Apache Kafka 4.0 */
+ test_with_apache_kafka_since_4_0 = rd_true;
+ }
+
/* Broker version limits the producer's feature set,
* for 0.9.0.0 no timestamp will be transmitted,
* but for 0.10.1.0 (or newer, api.version.request will be true)
@@ -207,13 +217,21 @@ int main_0052_msg_timestamps(int argc, char **argv) {
test_timestamps("CreateTime", "0.10.1.0", "none", &my_timestamp);
test_timestamps("LogAppendTime", "0.10.1.0", "none", &broker_timestamp);
- test_timestamps("CreateTime", "0.9.0.0", "none", &invalid_timestamp);
- test_timestamps("LogAppendTime", "0.9.0.0", "none", &broker_timestamp);
+ if (!test_with_apache_kafka_since_4_0) {
+ test_timestamps("CreateTime", "0.9.0.0", "none",
+ &invalid_timestamp);
+ test_timestamps("LogAppendTime", "0.9.0.0", "none",
+ &broker_timestamp);
+ }
#if WITH_ZLIB
test_timestamps("CreateTime", "0.10.1.0", "gzip", &my_timestamp);
test_timestamps("LogAppendTime", "0.10.1.0", "gzip", &broker_timestamp);
- test_timestamps("CreateTime", "0.9.0.0", "gzip", &invalid_timestamp);
- test_timestamps("LogAppendTime", "0.9.0.0", "gzip", &broker_timestamp);
+ if (!test_with_apache_kafka_since_4_0) {
+ test_timestamps("CreateTime", "0.9.0.0", "gzip",
+ &invalid_timestamp);
+ test_timestamps("LogAppendTime", "0.9.0.0", "gzip",
+ &broker_timestamp);
+ }
#endif
return 0;
diff --git a/tests/0053-stats_cb.cpp b/tests/0053-stats_cb.cpp
index d7254a6ca3..9e4cc77137 100644
--- a/tests/0053-stats_cb.cpp
+++ b/tests/0053-stats_cb.cpp
@@ -324,17 +324,13 @@ static void verify_e2e_stats(const std::string &prod_stats,
exp_tot_rxmsg_bytes += rxbytes;
Test::Say(tostr() << "Producer partition: " << (*pp)["partition"].GetInt()
- << ": "
- << "txmsgs: " << txmsgs << " vs "
- << exp_parts[part].msgcnt << ", "
- << "txbytes: " << txbytes << " vs "
- << exp_parts[part].totsize << "\n");
+ << ": " << "txmsgs: " << txmsgs << " vs "
+ << exp_parts[part].msgcnt << ", " << "txbytes: "
+ << txbytes << " vs " << exp_parts[part].totsize << "\n");
Test::Say(tostr() << "Consumer partition: " << (*cp)["partition"].GetInt()
- << ": "
- << "rxmsgs: " << rxmsgs << " vs "
- << exp_parts[part].msgcnt << ", "
- << "rxbytes: " << rxbytes << " vs "
- << exp_parts[part].totsize << "\n");
+ << ": " << "rxmsgs: " << rxmsgs << " vs "
+ << exp_parts[part].msgcnt << ", " << "rxbytes: "
+ << rxbytes << " vs " << exp_parts[part].totsize << "\n");
}
/* Check top-level total stats */
@@ -349,16 +345,12 @@ static void verify_e2e_stats(const std::string &prod_stats,
int64_t tot_rxmsgs = c["rxmsgs"].GetInt();
int64_t tot_rxmsg_bytes = c["rxmsg_bytes"].GetInt();
- Test::Say(tostr() << "Producer total: "
- << "txmsgs: " << tot_txmsgs << " vs " << exp_tot_txmsgs
- << ", "
- << "txbytes: " << tot_txmsg_bytes << " vs "
- << exp_tot_txmsg_bytes << "\n");
- Test::Say(tostr() << "Consumer total: "
- << "rxmsgs: " << tot_rxmsgs << " vs " << exp_tot_rxmsgs
- << ", "
- << "rxbytes: " << tot_rxmsg_bytes << " vs "
- << exp_tot_rxmsg_bytes << "\n");
+ Test::Say(tostr() << "Producer total: " << "txmsgs: " << tot_txmsgs << " vs "
+ << exp_tot_txmsgs << ", " << "txbytes: " << tot_txmsg_bytes
+ << " vs " << exp_tot_txmsg_bytes << "\n");
+ Test::Say(tostr() << "Consumer total: " << "rxmsgs: " << tot_rxmsgs << " vs "
+ << exp_tot_rxmsgs << ", " << "rxbytes: " << tot_rxmsg_bytes
+ << " vs " << exp_tot_rxmsg_bytes << "\n");
}
/**
diff --git a/tests/0055-producer_latency.c b/tests/0055-producer_latency.c
index a8cbb4efe8..08ccf927b7 100644
--- a/tests/0055-producer_latency.c
+++ b/tests/0055-producer_latency.c
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2012-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -28,6 +29,7 @@
#include "test.h"
#include "rdkafka.h"
+#include "../src/rdkafka_protocol.h"
#define _MSG_COUNT 10
@@ -146,10 +148,10 @@ static int verify_latency(struct latconf *latconf) {
}
latconf->wakeups = tot_wakeups;
- if (latconf->wakeups < 10 || latconf->wakeups > 1000) {
+ if (latconf->wakeups > 1000) {
TEST_FAIL_LATER(
"%s: broker wakeups out of range: %d, "
- "expected 10..1000",
+ "expected 5..1000",
latconf->name, latconf->wakeups);
fails++;
}
@@ -341,7 +343,7 @@ int main_0055_producer_latency(int argc, char **argv) {
}
/* Create topic without replicas to keep broker-side latency down */
- test_create_topic(NULL, topic, 1, 1);
+ test_create_topic_wait_exists(NULL, topic, 1, 1, 5000);
for (latconf = latconfs; latconf->name; latconf++)
test_producer_latency(topic, latconf);
@@ -364,3 +366,174 @@ int main_0055_producer_latency(int argc, char **argv) {
return 0;
}
+
+static void dr_msg_cb_first_message(rd_kafka_t *rk,
+ const rd_kafka_message_t *rkmessage,
+ void *opaque) {
+ test_timing_t *t_produce = (test_timing_t *)rkmessage->_private;
+ TIMING_STOP(t_produce);
+ /* The reason for setting such a low value is that both the mcluster and
+ * the producer are running locally, and there is no linger. This
+ * prevents the test from passing spuriously. */
+ TIMING_ASSERT_LATER(t_produce, 0, 100);
+
+ TEST_ASSERT(rkmessage->err == RD_KAFKA_RESP_ERR_NO_ERROR,
+ "expected no error, got %s",
+ rd_kafka_err2str(rkmessage->err));
+ TEST_ASSERT(rkmessage->offset == 0);
+}
+
+/**
+ * Test producer latency of the first message to 50 different topics.
+ *
+ * Cases:
+ * 0: rd_kafka_produce
+ * 1: rd_kafka_producev
+ * 2: rd_kafka_produceva
+ * 3: rd_kafka_produce_batch
+ */
+static void test_producer_latency_first_message(int case_number) {
+ rd_kafka_t *rk;
+ rd_kafka_conf_t *conf;
+ const char *topic;
+ test_timing_t t_produce;
+ rd_kafka_resp_err_t err;
+ rd_kafka_mock_cluster_t *mcluster;
+ rd_kafka_resp_err_t err_produce;
+ const char *bootstrap_servers;
+ const char *case_name = NULL;
+ int i;
+ size_t cnt;
+ rd_kafka_mock_request_t **reqs = NULL;
+ int metadata_request_cnt = 0;
+ const int iterations = 50;
+
+ if (case_number == 0) {
+ case_name = "rd_kafka_produce";
+ } else if (case_number == 1) {
+ case_name = "rd_kafka_producev";
+ } else if (case_number == 2) {
+ case_name = "rd_kafka_produceva";
+ } else if (case_number == 3) {
+ case_name = "rd_kafka_produce_batch";
+ }
+
+ SUB_TEST("Starting test for %s", case_name);
+ mcluster = test_mock_cluster_new(1, &bootstrap_servers);
+
+ test_conf_init(&conf, NULL, 60);
+ test_conf_set(conf, "bootstrap.servers", bootstrap_servers);
+ test_conf_set(conf, "linger.ms", "0");
+ rd_kafka_conf_set_dr_msg_cb(conf, dr_msg_cb_first_message);
+ rk = test_create_handle(RD_KAFKA_PRODUCER, conf);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+
+ /* Run the case 50 times. */
+ for (i = 0; i < iterations; i++) {
+ topic = test_mk_topic_name("0055_producer_latency_mock", 1);
+ rd_kafka_mock_topic_create(mcluster, topic, 1, 1);
+
+ switch (case_number) {
+ case 0: {
+ char *payload = "value";
+ rd_kafka_topic_t *rkt =
+ rd_kafka_topic_new(rk, topic, NULL);
+ int res;
+ TIMING_START(&t_produce, "Produce message");
+
+ res = rd_kafka_produce(rkt, 0, RD_KAFKA_MSG_F_COPY,
+ payload, 5, NULL, 0, &t_produce);
+ rd_kafka_topic_destroy(rkt);
+ TEST_ASSERT(res == 0, "expected no error");
+ break;
+ }
+ case 1: {
+ TIMING_START(&t_produce, "Produce message");
+ err_produce = rd_kafka_producev(
+ rk, RD_KAFKA_V_TOPIC(topic),
+ RD_KAFKA_V_PARTITION(0),
+ RD_KAFKA_V_VALUE("value", 5),
+ RD_KAFKA_V_OPAQUE((&t_produce)), RD_KAFKA_V_END);
+ TEST_ASSERT(err_produce == RD_KAFKA_RESP_ERR_NO_ERROR,
+ "expected no error, got %s",
+ rd_kafka_err2str(err_produce));
+ break;
+ }
+ case 2: {
+ rd_kafka_vu_t vus[3];
+ vus[0].vtype = RD_KAFKA_VTYPE_TOPIC;
+ vus[0].u.cstr = topic;
+ vus[1].vtype = RD_KAFKA_VTYPE_VALUE;
+ vus[1].u.mem.ptr = "value";
+ vus[1].u.mem.size = 5;
+ vus[2].vtype = RD_KAFKA_VTYPE_OPAQUE;
+ vus[2].u.ptr = &t_produce;
+
+ TIMING_START(&t_produce, "Produce message");
+ TEST_CALL_ERROR__(rd_kafka_produceva(rk, vus, 3));
+ break;
+ }
+ case 3: {
+ rd_kafka_message_t rkmessages[1] = {0};
+ rd_kafka_topic_t *rkt =
+ rd_kafka_topic_new(rk, topic, NULL);
+ int res;
+
+ rkmessages[0].payload = "value";
+ rkmessages[0].len = 5;
+ rkmessages[0]._private = &t_produce;
+
+ TIMING_START(&t_produce, "Produce message");
+ res = rd_kafka_produce_batch(
+ rkt, 0, RD_KAFKA_MSG_F_COPY, rkmessages, 1);
+ rd_kafka_topic_destroy(rkt);
+ TEST_ASSERT(res == 1, "expected 1 msg enqueued, got %d",
+ res);
+ break;
+ }
+ }
+ rd_kafka_poll(rk, 0);
+ err = rd_kafka_flush(rk, 10000);
+ TEST_ASSERT(err == RD_KAFKA_RESP_ERR_NO_ERROR,
+ "expected all messages to be flushed, got %s",
+ rd_kafka_err2str(err));
+
+ /* The topic_scan_all timer runs every 1 second. Originally, the
+ * issue was that the topic metadata was only fetched on the
+ * topic scan rather than when we were queueing the message.
+ * Running a loop and waiting 1ms between iterations means we
+ * will avoid passing tests due to coincidence. */
+ rd_usleep(1 * 1000ll, 0);
+ }
+ rd_kafka_destroy(rk);
+
+ reqs = rd_kafka_mock_get_requests(mcluster, &cnt);
+ for (i = 0; i < (int)cnt; i++) {
+ if (rd_kafka_mock_request_api_key(reqs[i]) ==
+ RD_KAFKAP_Metadata) {
+ metadata_request_cnt++;
+ }
+ }
+ TEST_ASSERT(metadata_request_cnt == iterations,
+ "Expected exactly %d MetadataRequest, got %d", iterations,
+ metadata_request_cnt);
+ rd_kafka_mock_request_destroy_array(reqs, cnt);
+ rd_kafka_mock_stop_request_tracking(mcluster);
+
+ test_mock_cluster_destroy(mcluster);
+
+ TEST_LATER_CHECK();
+ SUB_TEST_PASS();
+}
+
+int main_0055_producer_latency_mock(int argc, char **argv) {
+ TEST_SKIP_MOCK_CLUSTER(0);
+
+ int case_number;
+ for (case_number = 0; case_number < 4; case_number++) {
+ test_producer_latency_first_message(case_number);
+ }
+
+ return 0;
+}
diff --git a/tests/0056-balanced_group_mt.c b/tests/0056-balanced_group_mt.c
index 59dc8691bc..71a4ccc276 100644
--- a/tests/0056-balanced_group_mt.c
+++ b/tests/0056-balanced_group_mt.c
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2012-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -152,7 +153,8 @@ static void rebalance_cb(rd_kafka_t *rk,
case RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS:
assign_cnt++;
- rd_kafka_assign(rk, partitions);
+ test_consumer_assign_by_rebalance_protocol("rebalance", rk,
+ partitions);
mtx_lock(&lock);
consumers_running = 1;
mtx_unlock(&lock);
@@ -177,7 +179,8 @@ static void rebalance_cb(rd_kafka_t *rk,
if (assign_cnt == 0)
TEST_FAIL("asymetric rebalance_cb");
assign_cnt--;
- rd_kafka_assign(rk, NULL);
+ test_consumer_unassign_by_rebalance_protocol("rebalance", rk,
+ partitions);
mtx_lock(&lock);
consumers_running = 0;
mtx_unlock(&lock);
@@ -222,6 +225,7 @@ int main_0056_balanced_group_mt(int argc, char **argv) {
/* Produce messages */
rk_p = test_create_producer();
rkt_p = test_create_producer_topic(rk_p, topic, NULL);
+ test_wait_topic_exists(rk_p, topic, 5000);
for (partition = 0; partition < partition_cnt; partition++) {
test_produce_msgs(rk_p, rkt_p, testid, partition,
diff --git a/tests/0058-log.cpp b/tests/0058-log.cpp
index bf1c97a74e..e099276667 100644
--- a/tests/0058-log.cpp
+++ b/tests/0058-log.cpp
@@ -52,9 +52,9 @@ class myLogCb : public RdKafka::EventCb {
switch (event.type()) {
case RdKafka::Event::EVENT_LOG:
cnt_++;
- Test::Say(tostr() << "Log: "
- << "level " << event.severity() << ", facility "
- << event.fac() << ", str " << event.str() << "\n");
+ Test::Say(tostr() << "Log: " << "level " << event.severity()
+ << ", facility " << event.fac() << ", str "
+ << event.str() << "\n");
if (state_ != _EXP_LOG)
Test::Fail(
"Received unexpected "
diff --git a/tests/0059-bsearch.cpp b/tests/0059-bsearch.cpp
index 18ea216bda..4e4bd4b6de 100644
--- a/tests/0059-bsearch.cpp
+++ b/tests/0059-bsearch.cpp
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2016-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -27,6 +28,7 @@
*/
#include
+#include
#include "testcpp.h"
/**
@@ -110,7 +112,7 @@ class MyDeliveryReportCb : public RdKafka::DeliveryReportCb {
static void do_test_bsearch(void) {
RdKafka::Conf *conf, *tconf;
int msgcnt = 1000;
- int64_t timestamp;
+ int64_t timestamp_ms;
std::string errstr;
RdKafka::ErrorCode err;
MyDeliveryReportCb my_dr;
@@ -128,14 +130,16 @@ static void do_test_bsearch(void) {
delete conf;
delete tconf;
- timestamp = 1000;
+ /* Start with now() - 1h */
+ timestamp_ms = std::time(0) * 1000LL - 3600LL * 1000LL;
+
for (int i = 0; i < msgcnt; i++) {
err = p->produce(topic, partition, RdKafka::Producer::RK_MSG_COPY,
- (void *)topic.c_str(), topic.size(), NULL, 0, timestamp,
+ (void *)topic.c_str(), topic.size(), NULL, 0, timestamp_ms,
i == 357 ? (void *)1 /*golden*/ : NULL);
if (err != RdKafka::ERR_NO_ERROR)
Test::Fail("Produce failed: " + RdKafka::err2str(err));
- timestamp += 100 + (timestamp % 9);
+ timestamp_ms += 100 + (i % 10);
}
if (p->flush(tmout_multip(5000)) != 0)
diff --git a/tests/0061-consumer_lag.cpp b/tests/0061-consumer_lag.cpp
index 10a18afb33..c89aae1e4c 100644
--- a/tests/0061-consumer_lag.cpp
+++ b/tests/0061-consumer_lag.cpp
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2016-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -39,18 +40,33 @@ static std::string topic;
class StatsCb : public RdKafka::EventCb {
public:
- int64_t calc_lag; // calculated lag
- int lag_valid; // number of times lag has been valid
+ int64_t calc_lag; // calculated lag
+ int lag_valid; // number of times lag has been valid
+ bool partitions_assigned; // partitions were assigned
+ bool skip_first; // skip first event after assignment
StatsCb() {
- calc_lag = -1;
- lag_valid = 0;
+ calc_lag = -1;
+ lag_valid = 0;
+ partitions_assigned = false;
+ skip_first = true;
}
/**
* @brief Event callback
*/
void event_cb(RdKafka::Event &event) {
+ if (!partitions_assigned) {
+ /* It means we won't find the topic in the stats JSON. */
+ return;
+ }
+
+ if (skip_first) {
+ skip_first = false;
+ /* First JSON after assignment could be created before the assignment. */
+ return;
+ }
+
if (event.type() == RdKafka::Event::EVENT_LOG) {
Test::Say(tostr() << "LOG-" << event.severity() << "-" << event.fac()
<< ": " << event.str() << "\n");
@@ -200,19 +216,23 @@ static void do_test_consumer_lag(bool with_txns) {
Test::conf_set(conf, "group.id", topic);
Test::conf_set(conf, "enable.auto.commit", "false");
Test::conf_set(conf, "auto.offset.reset", "earliest");
- Test::conf_set(conf, "statistics.interval.ms", "100");
+ Test::conf_set(conf, "statistics.interval.ms", "500");
RdKafka::KafkaConsumer *c = RdKafka::KafkaConsumer::create(conf, errstr);
if (!c)
Test::Fail("Failed to create KafkaConsumer: " + errstr);
delete conf;
+ /* Execute callbacks for statistics enqueued before assignment. */
+ c->poll(0);
+
/* Assign partitions */
std::vector parts;
parts.push_back(RdKafka::TopicPartition::create(topic, 0));
if ((err = c->assign(parts)))
Test::Fail("assign failed: " + RdKafka::err2str(err));
RdKafka::TopicPartition::destroy(parts);
+ stats.partitions_assigned = true;
/* Start consuming */
Test::Say("Consuming topic " + topic + "\n");
diff --git a/tests/0067-empty_topic.cpp b/tests/0067-empty_topic.cpp
index 2db9ee8735..c2a1c39277 100644
--- a/tests/0067-empty_topic.cpp
+++ b/tests/0067-empty_topic.cpp
@@ -70,6 +70,9 @@ static void do_test_empty_topic_consumer() {
RdKafka::err2str(err));
delete md;
+ /* Await propagation to the whole cluster */
+ test_wait_topic_exists(consumer->c_ptr(), topic.c_str(), 1 * 1000);
+
/* Start consumer */
err = consumer->start(rkt, partition, RdKafka::Topic::OFFSET_BEGINNING);
if (err)
diff --git a/tests/0068-produce_timeout.c b/tests/0068-produce_timeout.c
index 7f19506888..667bd94bf1 100644
--- a/tests/0068-produce_timeout.c
+++ b/tests/0068-produce_timeout.c
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2012-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -28,26 +29,11 @@
#include "test.h"
-#if WITH_SOCKEM
#include "rdkafka.h"
+#include "../src/rdkafka_protocol.h"
#include
-/**
- * Force produce requests to timeout to test error handling.
- */
-
-/**
- * @brief Sockem connect, called from **internal librdkafka thread** through
- * librdkafka's connect_cb
- */
-static int connect_cb(struct test *test, sockem_t *skm, const char *id) {
-
- /* Let delay be high to trigger the local timeout */
- sockem_set(skm, "delay", 10000, NULL);
- return 0;
-}
-
static int
is_fatal_cb(rd_kafka_t *rk, rd_kafka_resp_err_t err, const char *reason) {
/* Ignore connectivity errors since we'll be bringing down
@@ -95,17 +81,25 @@ int main_0068_produce_timeout(int argc, char **argv) {
rd_kafka_conf_t *conf;
rd_kafka_topic_t *rkt;
int msgcounter = 0;
+ const char *bootstraps;
+
+ TEST_SKIP_MOCK_CLUSTER(0);
+
+ rd_kafka_mock_cluster_t *mcluster =
+ test_mock_cluster_new(3, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 1, 3);
+ rd_kafka_mock_partition_set_leader(mcluster, topic, 0, 1);
testid = test_id_generate();
test_conf_init(&conf, NULL, 60);
+ test_conf_set(conf, "bootstrap.servers", bootstraps);
rd_kafka_conf_set_dr_msg_cb(conf, dr_msg_cb);
- test_socket_enable(conf);
- test_curr->connect_cb = connect_cb;
test_curr->is_fatal_cb = is_fatal_cb;
- rk = test_create_handle(RD_KAFKA_PRODUCER, conf);
+ rk = test_create_handle(RD_KAFKA_PRODUCER, conf);
+ /* message timeout should be less that rtt */
rkt = test_create_producer_topic(rk, topic, "message.timeout.ms",
"2000", NULL);
@@ -113,10 +107,13 @@ int main_0068_produce_timeout(int argc, char **argv) {
test_auto_create_topic_rkt(rk, rkt, tmout_multip(5000));
TEST_SAY("Producing %d messages that should timeout\n", msgcnt);
+ rd_kafka_mock_broker_push_request_error_rtts(
+ mcluster, 1, RD_KAFKAP_Produce, 1, RD_KAFKA_RESP_ERR_NO_ERROR,
+ 6000);
+
test_produce_msgs_nowait(rk, rkt, testid, 0, 0, msgcnt, NULL, 0, 0,
&msgcounter);
-
TEST_SAY("Flushing..\n");
rd_kafka_flush(rk, 10000);
@@ -131,8 +128,9 @@ int main_0068_produce_timeout(int argc, char **argv) {
rd_kafka_topic_destroy(rkt);
rd_kafka_destroy(rk);
- return 0;
-}
+ test_mock_cluster_destroy(mcluster);
+ TEST_LATER_CHECK();
-#endif
+ return 0;
+}
diff --git a/tests/0069-consumer_add_parts.c b/tests/0069-consumer_add_parts.c
index b43c4c3a69..d8c4e444e0 100644
--- a/tests/0069-consumer_add_parts.c
+++ b/tests/0069-consumer_add_parts.c
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2012-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -58,10 +59,7 @@ static void rebalance_cb(rd_kafka_t *rk,
rd_kafka_err2str(err));
test_print_partition_list(parts);
- if (err == RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS)
- rd_kafka_assign(rk, parts);
- else if (err == RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS)
- rd_kafka_assign(rk, NULL);
+ test_rebalance_cb(rk, err, parts, opaque);
*statep = err;
}
@@ -79,9 +77,7 @@ int main_0069_consumer_add_parts(int argc, char **argv) {
c2 = test_create_consumer(topic, rebalance_cb, NULL, NULL);
TEST_SAY("Creating topic %s with 2 partitions\n", topic);
- test_create_topic(c1, topic, 2, 1);
-
- test_wait_topic_exists(c1, topic, 10 * 1000);
+ test_create_topic_wait_exists(c1, topic, 2, 1, 10 * 5000);
TEST_SAY("Subscribing\n");
test_consumer_subscribe(c1, topic);
diff --git a/tests/0075-retry.c b/tests/0075-retry.c
index c3ce353abf..958e0b7078 100644
--- a/tests/0075-retry.c
+++ b/tests/0075-retry.c
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2012-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -57,6 +58,10 @@ static struct {
} next;
int term;
+ /* Number of created sockets */
+ int num_sockets;
+ /* Current test, available to all threads as it's not TLS */
+ struct test *test_curr;
} ctrl;
static int ctrl_thrd_main(void *arg) {
@@ -67,6 +72,10 @@ static int ctrl_thrd_main(void *arg) {
int64_t now;
cnd_timedwait_ms(&ctrl.cnd, &ctrl.lock, 10);
+ if (!test_socket_find(ctrl.test_curr, ctrl.skm))
+ /* Socket was closed and destroyed when
+ * releasing the lock. */
+ break;
if (ctrl.cmd.ts_at) {
ctrl.next.ts_at = ctrl.cmd.ts_at;
@@ -106,8 +115,16 @@ static int ctrl_thrd_main(void *arg) {
static int connect_cb(struct test *test, sockem_t *skm, const char *id) {
mtx_lock(&ctrl.lock);
+ if (ctrl.num_sockets < 1) {
+ /* Since librdkafka is decommissioning brokers,
+ * first connection is with a bootstrap broker,
+ * next one is with a learned one. */
+ ctrl.num_sockets++;
+ mtx_unlock(&ctrl.lock);
+ return 0;
+ }
if (ctrl.skm) {
- /* Reject all but the first connect */
+ /* Reject all but the first two connects */
mtx_unlock(&ctrl.lock);
return ECONNREFUSED;
}
@@ -170,14 +187,26 @@ static void do_test_low_socket_timeout(const char *topic) {
mtx_init(&ctrl.lock, mtx_plain);
cnd_init(&ctrl.cnd);
+ ctrl.test_curr = test_curr;
TEST_SAY("Test Metadata request retries on timeout\n");
+ test_create_topic_wait_exists(NULL, topic, 1, -1, 5000);
+
test_conf_init(&conf, NULL, 60);
test_conf_set(conf, "socket.timeout.ms", "1000");
test_conf_set(conf, "socket.max.fails", "12345");
test_conf_set(conf, "retry.backoff.ms", "5000");
test_conf_set(conf, "retry.backoff.max.ms", "5000");
+ /* Sparse re-connect interval depends on `reconnect.backoff.ms` / 2
+ * increasing it allows to saturate the sparse re-connect interval
+ * and avoid to concurrent connects: the first for setting the bootstrap
+ * brokers and the second for adding a new topic to get metadata for.
+ * This way the test is more reliable especially when using valgrind
+ * as it needs to keep the last connection only,
+ * to add a delay later. */
+ test_conf_set(conf, "reconnect.backoff.ms", "2000");
+
/* Avoid api version requests (with their own timeout) to get in
* the way of our test */
test_conf_set(conf, "api.version.request", "false");
@@ -244,6 +273,14 @@ static void do_test_low_socket_timeout(const char *topic) {
int main_0075_retry(int argc, char **argv) {
const char *topic = test_mk_topic_name("0075_retry", 1);
+ if (test_needs_auth()) {
+ /* When authentication is involved there's the need
+ * for additional SASL calls. These are delayed too and
+ * it changes test timing. */
+ TEST_SKIP("Cannot run this test with SSL/SASL\n");
+ return 0;
+ }
+
do_test_low_socket_timeout(topic);
return 0;
diff --git a/tests/0076-produce_retry.c b/tests/0076-produce_retry.c
index 2ea9dfa4fd..c4e07ca471 100644
--- a/tests/0076-produce_retry.c
+++ b/tests/0076-produce_retry.c
@@ -128,6 +128,7 @@ static void do_test_produce_retries(const char *topic,
rk = test_create_handle(RD_KAFKA_PRODUCER, conf);
rkt = test_create_producer_topic(rk, topic, NULL);
+ test_wait_topic_exists(rk, topic, 5000);
/* Create the topic to make sure connections are up and ready. */
err = test_auto_create_topic_rkt(rk, rkt, tmout_multip(5000));
@@ -277,6 +278,7 @@ static void do_test_produce_retries_disconnect(const char *topic,
rk = test_create_handle(RD_KAFKA_PRODUCER, conf);
rkt = test_create_producer_topic(rk, topic, NULL);
+ test_wait_topic_exists(rk, topic, 5000);
err = test_produce_sync(rk, rkt, testid, 0);
diff --git a/tests/0077-compaction.c b/tests/0077-compaction.c
index 623461b7f8..433c249b00 100644
--- a/tests/0077-compaction.c
+++ b/tests/0077-compaction.c
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2012-2022, Magnus Edenhill
+ * 2023, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -193,6 +194,7 @@ static void do_test_compaction(int msgs_per_key, const char *compression) {
"--config file.delete.delay.ms=10000 "
"--config max.compaction.lag.ms=100",
topic, partition + 1);
+ test_wait_topic_exists(NULL, topic, 5000);
test_conf_init(&conf, NULL, 120);
rd_kafka_conf_set_dr_msg_cb(conf, test_dr_msg_cb);
@@ -300,8 +302,12 @@ static void do_test_compaction(int msgs_per_key, const char *compression) {
* this doesn't really work because the low watermark offset
* is not updated on compaction if the first segment is not deleted.
* But it serves as a pause to let compaction kick in
- * which is triggered by the dummy produce above. */
- wait_compaction(rk, topic, partition, 0, 20 * 1000);
+ * which is triggered by the dummy produce above.
+ * Compaction timer is every 15 seconds and
+ * with a large number of segments it can
+ * take the same time. */
+ wait_compaction(rk, topic, partition, 0,
+ msgcnt > 50 ? 30 * 1000 : 20 * 1000);
TEST_SAY(_C_YEL "Verify messages after compaction\n");
/* After compaction we expect the following messages:
diff --git a/tests/0080-admin_ut.c b/tests/0080-admin_ut.c
index 3a3b980f0a..a9f0e1181f 100644
--- a/tests/0080-admin_ut.c
+++ b/tests/0080-admin_ut.c
@@ -528,22 +528,44 @@ static void do_test_ListConsumerGroups(const char *what,
q = useq ? useq : rd_kafka_queue_new(rk);
if (with_options) {
- rd_kafka_consumer_group_state_t duplicate[2] = {
+ rd_kafka_error_t *error;
+ rd_kafka_consumer_group_state_t duplicate_states[2] = {
RD_KAFKA_CONSUMER_GROUP_STATE_EMPTY,
RD_KAFKA_CONSUMER_GROUP_STATE_EMPTY};
+ rd_kafka_consumer_group_type_t duplicate_types[2] = {
+ RD_KAFKA_CONSUMER_GROUP_TYPE_CLASSIC,
+ RD_KAFKA_CONSUMER_GROUP_TYPE_CLASSIC};
+ rd_kafka_consumer_group_type_t unknown_type[1] = {
+ RD_KAFKA_CONSUMER_GROUP_TYPE_UNKNOWN};
options = rd_kafka_AdminOptions_new(
rk, RD_KAFKA_ADMIN_OP_LISTCONSUMERGROUPS);
/* Test duplicate error on match states */
- rd_kafka_error_t *error =
- rd_kafka_AdminOptions_set_match_consumer_group_states(
- options, duplicate, 2);
+ error = rd_kafka_AdminOptions_set_match_consumer_group_states(
+ options, duplicate_states, 2);
TEST_ASSERT(error && rd_kafka_error_code(error), "%s",
"Expected error on duplicate states,"
" got no error");
rd_kafka_error_destroy(error);
+ /* Test duplicate error on match group types */
+ error = rd_kafka_AdminOptions_set_match_consumer_group_types(
+ options, duplicate_types, 2);
+ TEST_ASSERT(error && rd_kafka_error_code(error), "%s",
+ "Expected error on duplicate group types,"
+ " got no error");
+ rd_kafka_error_destroy(error);
+
+ /* Test invalid args error on setting UNKNOWN group type in
+ * match group types */
+ error = rd_kafka_AdminOptions_set_match_consumer_group_types(
+ options, unknown_type, 1);
+ TEST_ASSERT(error && rd_kafka_error_code(error), "%s",
+ "Expected error on Unknown group type,"
+ " got no error");
+ rd_kafka_error_destroy(error);
+
exp_timeout = MY_SOCKET_TIMEOUT_MS * 2;
TEST_CALL_ERR__(rd_kafka_AdminOptions_set_request_timeout(
options, exp_timeout, errstr, sizeof(errstr)));
@@ -1184,8 +1206,8 @@ static void do_test_AclBinding() {
char errstr[512];
rd_kafka_AclBinding_t *new_acl;
- rd_bool_t valid_resource_types[] = {rd_false, rd_false, rd_true,
- rd_true, rd_true, rd_false};
+ rd_bool_t valid_resource_types[] = {
+ rd_false, rd_false, rd_true, rd_true, rd_true, rd_true, rd_false};
rd_bool_t valid_resource_pattern_types[] = {
rd_false, rd_false, rd_false, rd_true, rd_true, rd_false};
rd_bool_t valid_acl_operation[] = {
@@ -1316,7 +1338,7 @@ static void do_test_AclBindingFilter() {
char errstr[512];
rd_kafka_AclBindingFilter_t *new_acl_filter;
- rd_bool_t valid_resource_types[] = {rd_false, rd_true, rd_true,
+ rd_bool_t valid_resource_types[] = {rd_false, rd_true, rd_true, rd_true,
rd_true, rd_true, rd_false};
rd_bool_t valid_resource_pattern_types[] = {
rd_false, rd_true, rd_true, rd_true, rd_true, rd_false};
@@ -2355,6 +2377,137 @@ static void do_test_AlterUserScramCredentials(const char *what,
SUB_TEST_PASS();
}
+static void do_test_ElectLeaders(const char *what,
+ rd_kafka_t *rk,
+ rd_kafka_queue_t *useq,
+ int with_options,
+ rd_kafka_ElectionType_t election_type) {
+ rd_kafka_queue_t *q;
+ rd_kafka_AdminOptions_t *options = NULL;
+ rd_kafka_event_t *rkev;
+ rd_kafka_resp_err_t err;
+ const rd_kafka_ElectLeaders_result_t *res;
+ rd_kafka_ElectLeaders_t *duplicate_elect_leaders;
+ rd_kafka_ElectLeaders_t *elect_leaders;
+ int exp_timeout = MY_SOCKET_TIMEOUT_MS;
+ test_timing_t timing;
+ rd_kafka_topic_partition_list_t *partitions;
+ char errstr[512];
+ void *my_opaque = NULL, *opaque;
+
+ SUB_TEST_QUICK("%s ElectLeaders with %s, timeout %dms",
+ rd_kafka_name(rk), what, exp_timeout);
+
+ q = useq ? useq : rd_kafka_queue_new(rk);
+
+ partitions = rd_kafka_topic_partition_list_new(3);
+ rd_kafka_topic_partition_list_add(partitions, "topic1", 9);
+ rd_kafka_topic_partition_list_add(partitions, "topic3", 15);
+ rd_kafka_topic_partition_list_add(partitions, "topic1", 1);
+ elect_leaders = rd_kafka_ElectLeaders_new(election_type, partitions);
+ rd_kafka_topic_partition_list_destroy(partitions);
+
+ partitions = rd_kafka_topic_partition_list_new(3);
+ rd_kafka_topic_partition_list_add(partitions, "topic1", 9);
+ rd_kafka_topic_partition_list_add(partitions, "topic3", 15);
+ rd_kafka_topic_partition_list_add(partitions, "topic1", 9);
+ duplicate_elect_leaders =
+ rd_kafka_ElectLeaders_new(election_type, partitions);
+ rd_kafka_topic_partition_list_destroy(partitions);
+
+ if (with_options) {
+ options = rd_kafka_AdminOptions_new(
+ rk, RD_KAFKA_ADMIN_OP_ELECTLEADERS);
+
+ exp_timeout = MY_SOCKET_TIMEOUT_MS * 2;
+
+ err = rd_kafka_AdminOptions_set_request_timeout(
+ options, exp_timeout, errstr, sizeof(errstr));
+ TEST_ASSERT(!err, "%s", rd_kafka_err2str(err));
+
+ if (useq) {
+ my_opaque = (void *)99981;
+ rd_kafka_AdminOptions_set_opaque(options, my_opaque);
+ }
+ }
+
+ /*Duplicate topic-partition list*/
+ TIMING_START(&timing, "ElectLeaders");
+ TEST_SAY("Call ElectLeaders, timeout is %dms\n", exp_timeout);
+ rd_kafka_ElectLeaders(rk, duplicate_elect_leaders, options, q);
+ TIMING_ASSERT_LATER(&timing, 0, 10);
+ rd_kafka_ElectLeaders_destroy(duplicate_elect_leaders);
+
+ /* Poll result queue */
+ TIMING_START(&timing, "ElectLeaders.queue_poll");
+ rkev = rd_kafka_queue_poll(q, exp_timeout + 1000);
+ TIMING_ASSERT(&timing, 0, exp_timeout + 100);
+ TEST_ASSERT(rkev != NULL, "expected result in %dms", exp_timeout);
+ TEST_SAY("ElectLeaders: got %s in %.3fs\n", rd_kafka_event_name(rkev),
+ TIMING_DURATION(&timing) / 1000.0f);
+
+ /* Convert event to proper result */
+ res = rd_kafka_event_ElectLeaders_result(rkev);
+ TEST_ASSERT(res, "expected ElectLeaders_result, not %s",
+ rd_kafka_event_name(rkev));
+ /*Expecting error*/
+ err = rd_kafka_event_error(rkev);
+ const char *event_errstr_duplicate = rd_kafka_event_error_string(rkev);
+ TEST_ASSERT(err, "expected ElectLeaders to fail");
+ TEST_ASSERT(err == RD_KAFKA_RESP_ERR__INVALID_ARG,
+ "expected RD_KAFKA_RESP_ERR__INVALID_ARG, not %s",
+ rd_kafka_err2name(err));
+ TEST_ASSERT(strcmp(event_errstr_duplicate,
+ "Duplicate partitions specified") == 0,
+ "expected \"Duplicate partitions specified\", not \"%s\"",
+ event_errstr_duplicate);
+ rd_kafka_event_destroy(rkev);
+
+ /*Correct topic-partition list*/
+ TIMING_START(&timing, "ElectLeaders");
+ TEST_SAY("Call ElectLeaders, timeout is %dms\n", exp_timeout);
+ rd_kafka_ElectLeaders(rk, elect_leaders, options, q);
+ TIMING_ASSERT_LATER(&timing, 0, 10);
+ rd_kafka_ElectLeaders_destroy(elect_leaders);
+
+ /* Poll result queue */
+ TIMING_START(&timing, "ElectLeaders.queue_poll");
+ rkev = rd_kafka_queue_poll(q, exp_timeout + 1000);
+ TIMING_ASSERT(&timing, exp_timeout - 100, exp_timeout + 100);
+ TEST_ASSERT(rkev != NULL, "expected result in %dms", exp_timeout);
+ TEST_SAY("ElectLeaders: got %s in %.3fs\n", rd_kafka_event_name(rkev),
+ TIMING_DURATION(&timing) / 1000.0f);
+
+ /* Convert event to proper result */
+ res = rd_kafka_event_ElectLeaders_result(rkev);
+ TEST_ASSERT(res, "expected ElectLeaders_result, not %s",
+ rd_kafka_event_name(rkev));
+ opaque = rd_kafka_event_opaque(rkev);
+ TEST_ASSERT(opaque == my_opaque, "expected opaque to be %p, not %p",
+ my_opaque, opaque);
+ /*Expecting error*/
+ err = rd_kafka_event_error(rkev);
+ const char *event_err = rd_kafka_event_error_string(rkev);
+ TEST_ASSERT(err, "expected ElectLeaders to fail");
+ TEST_ASSERT(err == RD_KAFKA_RESP_ERR__TIMED_OUT,
+ "expected RD_KAFKA_RESP_ERR__TIMED_OUT, not %s",
+ rd_kafka_err2name(err));
+ TEST_ASSERT(strcmp(event_err,
+ "Failed while waiting for controller: "
+ "Local: Timed out") == 0,
+ "expected \"Failed while waiting for controller: "
+ "Local: Timed out\", not \"%s\"",
+ event_err);
+ rd_kafka_event_destroy(rkev);
+
+ if (options)
+ rd_kafka_AdminOptions_destroy(options);
+ if (!useq)
+ rd_kafka_queue_destroy(q);
+
+ SUB_TEST_PASS();
+}
+
/**
* @brief Test a mix of APIs using the same replyq.
*
@@ -2555,6 +2708,10 @@ static void do_test_unclean_destroy(rd_kafka_type_t cltype, int with_mainq) {
* rely on the controller not being found. */
test_conf_set(conf, "bootstrap.servers", "");
test_conf_set(conf, "socket.timeout.ms", "60000");
+ if (test_consumer_group_protocol()) {
+ test_conf_set(conf, "group.protocol",
+ test_consumer_group_protocol());
+ }
rk = rd_kafka_new(cltype, conf, errstr, sizeof(errstr));
TEST_ASSERT(rk, "kafka_new(%d): %s", cltype, errstr);
@@ -2612,17 +2769,18 @@ static void do_test_options(rd_kafka_t *rk) {
RD_KAFKA_ADMIN_OP_LISTCONSUMERGROUPOFFSETS, \
RD_KAFKA_ADMIN_OP_ALTERCONSUMERGROUPOFFSETS, \
RD_KAFKA_ADMIN_OP_DELETECONSUMERGROUPOFFSETS, \
+ RD_KAFKA_ADMIN_OP_ELECTLEADERS, \
RD_KAFKA_ADMIN_OP_ANY /* Must be last */ \
}
struct {
const char *setter;
- const rd_kafka_admin_op_t valid_apis[16];
+ const rd_kafka_admin_op_t valid_apis[17];
} matrix[] = {
{"request_timeout", _all_apis},
{"operation_timeout",
{RD_KAFKA_ADMIN_OP_CREATETOPICS, RD_KAFKA_ADMIN_OP_DELETETOPICS,
RD_KAFKA_ADMIN_OP_CREATEPARTITIONS,
- RD_KAFKA_ADMIN_OP_DELETERECORDS}},
+ RD_KAFKA_ADMIN_OP_DELETERECORDS, RD_KAFKA_ADMIN_OP_ELECTLEADERS}},
{"validate_only",
{RD_KAFKA_ADMIN_OP_CREATETOPICS,
RD_KAFKA_ADMIN_OP_CREATEPARTITIONS,
@@ -2761,6 +2919,10 @@ static rd_kafka_t *create_admin_client(rd_kafka_type_t cltype) {
* rely on the controller not being found. */
test_conf_set(conf, "bootstrap.servers", "");
test_conf_set(conf, "socket.timeout.ms", MY_SOCKET_TIMEOUT_MS_STR);
+ if (test_consumer_group_protocol()) {
+ test_conf_set(conf, "group.protocol",
+ test_consumer_group_protocol());
+ }
/* For use with the background queue */
rd_kafka_conf_set_background_event_cb(conf, background_event_cb);
@@ -2876,6 +3038,23 @@ static void do_test_apis(rd_kafka_type_t cltype) {
do_test_AlterUserScramCredentials("main queue", rk, mainq);
do_test_AlterUserScramCredentials("temp queue", rk, NULL);
+ do_test_ElectLeaders("main queue, options, Preffered Elections", rk,
+ mainq, 1, RD_KAFKA_ELECTION_TYPE_PREFERRED);
+ do_test_ElectLeaders("main queue, options, Unclean Elections", rk,
+ mainq, 1, RD_KAFKA_ELECTION_TYPE_UNCLEAN);
+ do_test_ElectLeaders("main queue, no options, Preffered Elections", rk,
+ mainq, 0, RD_KAFKA_ELECTION_TYPE_PREFERRED);
+ do_test_ElectLeaders("main queue, no options, Unclean Elections", rk,
+ mainq, 0, RD_KAFKA_ELECTION_TYPE_UNCLEAN);
+ do_test_ElectLeaders("temp queue, options, Preffered Elections", rk,
+ NULL, 1, RD_KAFKA_ELECTION_TYPE_PREFERRED);
+ do_test_ElectLeaders("temp queue, options, Unclean Elections", rk, NULL,
+ 1, RD_KAFKA_ELECTION_TYPE_UNCLEAN);
+ do_test_ElectLeaders("temp queue, no options, Preffered Elections", rk,
+ NULL, 0, RD_KAFKA_ELECTION_TYPE_PREFERRED);
+ do_test_ElectLeaders("temp queue, no options, Unclean Elections", rk,
+ NULL, 0, RD_KAFKA_ELECTION_TYPE_UNCLEAN);
+
do_test_mix(rk, mainq);
do_test_configs(rk, mainq);
diff --git a/tests/0081-admin.c b/tests/0081-admin.c
index 0690217a3c..f16f958e58 100644
--- a/tests/0081-admin.c
+++ b/tests/0081-admin.c
@@ -39,6 +39,9 @@
static int32_t *avail_brokers;
static size_t avail_broker_cnt;
+#define group_configs_supported() \
+ (test_broker_version >= TEST_BRKVER(4, 0, 0, 0))
+
static void do_test_CreateTopics(const char *what,
@@ -781,7 +784,7 @@ static void do_test_AlterConfigs(rd_kafka_t *rk, rd_kafka_queue_t *rkqu) {
TEST_ASSERT(!err, "%s", rd_kafka_err2str(err));
err = rd_kafka_ConfigResource_set_config(
- configs[ci], "offset.metadata.max.bytes", "12345");
+ configs[ci], "max.compaction.lag.ms", "3600000");
TEST_ASSERT(!err, "%s", rd_kafka_err2str(err));
if (test_broker_version >= TEST_BRKVER(2, 7, 0, 0))
@@ -902,7 +905,7 @@ static void do_test_AlterConfigs(rd_kafka_t *rk, rd_kafka_queue_t *rkqu) {
*/
static void do_test_IncrementalAlterConfigs(rd_kafka_t *rk,
rd_kafka_queue_t *rkqu) {
-#define MY_CONFRES_CNT 3
+#define MY_CONFRES_CNT 4
char *topics[MY_CONFRES_CNT];
rd_kafka_ConfigResource_t *configs[MY_CONFRES_CNT];
rd_kafka_AdminOptions_t *options;
@@ -935,6 +938,7 @@ static void do_test_IncrementalAlterConfigs(rd_kafka_t *rk,
/** Test the test helper, for use in other tests. */
do {
const char *broker_id = tsprintf("%d", avail_brokers[0]);
+ const char *group_id = topics[0];
const char *confs_set_append[] = {
"compression.type", "SET", "lz4",
"cleanup.policy", "APPEND", "compact"};
@@ -947,6 +951,10 @@ static void do_test_IncrementalAlterConfigs(rd_kafka_t *rk,
const char *confs_delete_subtract_broker[] = {
"background.threads", "DELETE", "",
"log.cleanup.policy", "SUBTRACT", "compact"};
+ const char *confs_set_group[] = {"consumer.session.timeout.ms",
+ "SET", "50000"};
+ const char *confs_delete_group[] = {
+ "consumer.session.timeout.ms", "DELETE", ""};
TEST_SAY("Testing test helper with SET and APPEND\n");
test_IncrementalAlterConfigs_simple(rk, RD_KAFKA_RESOURCE_TOPIC,
@@ -969,6 +977,16 @@ static void do_test_IncrementalAlterConfigs(rd_kafka_t *rk,
test_IncrementalAlterConfigs_simple(
rk, RD_KAFKA_RESOURCE_BROKER, broker_id,
confs_delete_subtract_broker, 2);
+ TEST_SAY(
+ "Testing test helper with SET with GROUP resource type\n");
+ test_IncrementalAlterConfigs_simple(
+ rk, RD_KAFKA_RESOURCE_GROUP, group_id, confs_set_group, 1);
+ TEST_SAY(
+ "Testing test helper with DELETE with GROUP resource "
+ "type\n");
+ test_IncrementalAlterConfigs_simple(rk, RD_KAFKA_RESOURCE_GROUP,
+ group_id,
+ confs_delete_group, 1);
TEST_SAY("End testing test helper\n");
} while (0);
@@ -1025,8 +1043,8 @@ static void do_test_IncrementalAlterConfigs(rd_kafka_t *rk,
TEST_ASSERT(!error, "%s", rd_kafka_error_string(error));
error = rd_kafka_ConfigResource_add_incremental_config(
- configs[ci], "offset.metadata.max.bytes",
- RD_KAFKA_ALTER_CONFIG_OP_TYPE_SET, "12345");
+ configs[ci], "max.compaction.lag.ms",
+ RD_KAFKA_ALTER_CONFIG_OP_TYPE_SET, "3600000");
TEST_ASSERT(!error, "%s", rd_kafka_error_string(error));
if (test_broker_version >= TEST_BRKVER(2, 7, 0, 0))
@@ -1035,6 +1053,23 @@ static void do_test_IncrementalAlterConfigs(rd_kafka_t *rk,
exp_err[ci] = RD_KAFKA_RESP_ERR_UNKNOWN;
ci++;
+ /**
+ * ConfigResource #3: valid group config
+ */
+ configs[ci] =
+ rd_kafka_ConfigResource_new(RD_KAFKA_RESOURCE_GROUP, "my-group");
+
+ error = rd_kafka_ConfigResource_add_incremental_config(
+ configs[ci], "consumer.session.timeout.ms",
+ RD_KAFKA_ALTER_CONFIG_OP_TYPE_SET, "50000");
+ TEST_ASSERT(!error, "%s", rd_kafka_error_string(error));
+ if (group_configs_supported()) {
+ exp_err[ci] = RD_KAFKA_RESP_ERR_NO_ERROR;
+ } else {
+ exp_err[ci] = RD_KAFKA_RESP_ERR_INVALID_REQUEST;
+ }
+ ci++;
+
/*
* Timeout options
*/
@@ -1334,6 +1369,148 @@ static void do_test_DescribeConfigs(rd_kafka_t *rk, rd_kafka_queue_t *rkqu) {
SUB_TEST_PASS();
}
+/**
+ * @brief Test DescribeConfigs for groups
+ */
+static void do_test_DescribeConfigs_groups(rd_kafka_t *rk,
+ rd_kafka_queue_t *rkqu) {
+#define MY_CONFRES_CNT 1
+ rd_kafka_ConfigResource_t *configs[MY_CONFRES_CNT];
+ rd_kafka_AdminOptions_t *options;
+ rd_kafka_resp_err_t exp_err[MY_CONFRES_CNT];
+ rd_kafka_event_t *rkev;
+ rd_kafka_resp_err_t err;
+ const rd_kafka_DescribeConfigs_result_t *res;
+ const rd_kafka_ConfigResource_t **rconfigs;
+ char *group;
+ size_t rconfig_cnt;
+ char errstr[128];
+ const char *errstr2;
+ int ci = 0;
+ int i;
+ int fails = 0;
+
+ SUB_TEST_QUICK();
+
+ group = rd_strdup(test_mk_topic_name(__FUNCTION__, 1));
+
+ /*
+ * ConfigResource #0: group config, for a non-existent group.
+ */
+ configs[ci] =
+ rd_kafka_ConfigResource_new(RD_KAFKA_RESOURCE_GROUP, group);
+ if (group_configs_supported()) {
+ exp_err[ci] = RD_KAFKA_RESP_ERR_NO_ERROR;
+ } else {
+ exp_err[ci] = RD_KAFKA_RESP_ERR_INVALID_REQUEST;
+ }
+ ci++;
+
+ /*
+ * Timeout options
+ */
+ options = rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_ANY);
+ err = rd_kafka_AdminOptions_set_request_timeout(options, 10000, errstr,
+ sizeof(errstr));
+ TEST_ASSERT(!err, "%s", errstr);
+
+ /*
+ * Fire off request
+ */
+ rd_kafka_DescribeConfigs(rk, configs, ci, options, rkqu);
+
+ /*
+ * Wait for result
+ */
+ rkev = test_wait_admin_result(
+ rkqu, RD_KAFKA_EVENT_DESCRIBECONFIGS_RESULT, 10000 + 1000);
+
+ /*
+ * Extract result
+ */
+ res = rd_kafka_event_DescribeConfigs_result(rkev);
+ TEST_ASSERT(res, "Expected DescribeConfigs result, not %s",
+ rd_kafka_event_name(rkev));
+
+ err = rd_kafka_event_error(rkev);
+ errstr2 = rd_kafka_event_error_string(rkev);
+ TEST_ASSERT(!err, "Expected success, not %s: %s",
+ rd_kafka_err2name(err), errstr2);
+
+ rconfigs = rd_kafka_DescribeConfigs_result_resources(res, &rconfig_cnt);
+ TEST_ASSERT((int)rconfig_cnt == ci,
+ "Expected %d result resources, got %" PRIusz "\n", ci,
+ rconfig_cnt);
+
+ /*
+ * Verify status per resource
+ */
+ for (i = 0; i < (int)rconfig_cnt; i++) {
+ const rd_kafka_ConfigEntry_t **entries;
+ size_t entry_cnt;
+
+ err = rd_kafka_ConfigResource_error(rconfigs[i]);
+ errstr2 = rd_kafka_ConfigResource_error_string(rconfigs[i]);
+
+ entries =
+ rd_kafka_ConfigResource_configs(rconfigs[i], &entry_cnt);
+
+ TEST_SAY(
+ "ConfigResource #%d: type %s (%d), \"%s\": "
+ "%" PRIusz " ConfigEntries, error %s (%s)\n",
+ i,
+ rd_kafka_ResourceType_name(
+ rd_kafka_ConfigResource_type(rconfigs[i])),
+ rd_kafka_ConfigResource_type(rconfigs[i]),
+ rd_kafka_ConfigResource_name(rconfigs[i]), entry_cnt,
+ rd_kafka_err2name(err), errstr2 ? errstr2 : "");
+
+ test_print_ConfigEntry_array(entries, entry_cnt, 1);
+
+ if (rd_kafka_ConfigResource_type(rconfigs[i]) !=
+ rd_kafka_ConfigResource_type(configs[i]) ||
+ strcmp(rd_kafka_ConfigResource_name(rconfigs[i]),
+ rd_kafka_ConfigResource_name(configs[i]))) {
+ TEST_FAIL_LATER(
+ "ConfigResource #%d: "
+ "expected type %s name %s, "
+ "got type %s name %s",
+ i,
+ rd_kafka_ResourceType_name(
+ rd_kafka_ConfigResource_type(configs[i])),
+ rd_kafka_ConfigResource_name(configs[i]),
+ rd_kafka_ResourceType_name(
+ rd_kafka_ConfigResource_type(rconfigs[i])),
+ rd_kafka_ConfigResource_name(rconfigs[i]));
+ fails++;
+ }
+
+ if (err != exp_err[i]) {
+ TEST_FAIL_LATER(
+ "ConfigResource #%d: "
+ "expected %s (%d), got %s (%s)",
+ i, rd_kafka_err2name(exp_err[i]), exp_err[i],
+ rd_kafka_err2name(err), errstr2 ? errstr2 : "");
+ fails++;
+ }
+ }
+
+ TEST_ASSERT(!fails, "See %d previous failure(s)", fails);
+
+ rd_kafka_event_destroy(rkev);
+
+ rd_kafka_ConfigResource_destroy_array(configs, ci);
+
+ rd_kafka_AdminOptions_destroy(options);
+
+ rd_free(group);
+
+ TEST_LATER_CHECK();
+#undef MY_CONFRES_CNT
+
+ SUB_TEST_PASS();
+}
+
/**
* @brief Test CreateAcls
*/
@@ -1515,6 +1692,9 @@ do_test_DescribeAcls(rd_kafka_t *rk, rd_kafka_queue_t *useq, int version) {
create_err =
test_CreateAcls_simple(rk, NULL, acl_bindings_create, 2, NULL);
+ /* Wait for ACL propagation. */
+ rd_sleep(tmout_multip(2));
+
TEST_ASSERT(!create_err, "create error: %s",
rd_kafka_err2str(create_err));
@@ -1927,6 +2107,9 @@ do_test_DeleteAcls(rd_kafka_t *rk, rd_kafka_queue_t *useq, int version) {
create_err =
test_CreateAcls_simple(rk, NULL, acl_bindings_create, 3, NULL);
+ /* Wait for ACL propagation. */
+ rd_sleep(tmout_multip(2));
+
TEST_ASSERT(!create_err, "create error: %s",
rd_kafka_err2str(create_err));
@@ -1946,6 +2129,9 @@ do_test_DeleteAcls(rd_kafka_t *rk, rd_kafka_queue_t *useq, int version) {
q);
TIMING_ASSERT_LATER(&timing, 0, 50);
+ /* Wait for ACL propagation. */
+ rd_sleep(tmout_multip(2));
+
/*
* Wait for result
*/
@@ -2062,6 +2248,9 @@ do_test_DeleteAcls(rd_kafka_t *rk, rd_kafka_queue_t *useq, int version) {
q);
TIMING_ASSERT_LATER(&timing, 0, 50);
+ /* Wait for ACL propagation. */
+ rd_sleep(1);
+
/*
* Wait for result
*/
@@ -2634,135 +2823,63 @@ static void do_test_DeleteGroups(const char *what,
}
/**
- * @brief Test list groups, creating consumers for a set of groups,
- * listing and deleting them at the end.
+ * @brief Helper for do_test_ListConsumerGroups, makes ListConsumerGroups call
+ * and checks returned group count equals to \p exp_found.
+ *
+ * Parameter \p exp_type, if not UNKNOWN, needs to match
+ * returned groups type.
+ *
+ * If \p match_states is true, then the state of the
+ * returned groups is checked and expected to be `EMPTY`.
*/
-static void do_test_ListConsumerGroups(const char *what,
- rd_kafka_t *rk,
- rd_kafka_queue_t *useq,
- int request_timeout,
- rd_bool_t match_states) {
-#define TEST_LIST_CONSUMER_GROUPS_CNT 4
- rd_kafka_queue_t *q;
- rd_kafka_AdminOptions_t *options = NULL;
- rd_kafka_event_t *rkev = NULL;
- rd_kafka_resp_err_t err;
- size_t valid_cnt, error_cnt;
- rd_bool_t is_simple_consumer_group;
- rd_kafka_consumer_group_state_t state;
- char errstr[512];
- const char *errstr2, *group_id;
- char *list_consumer_groups[TEST_LIST_CONSUMER_GROUPS_CNT];
- const int partitions_cnt = 1;
- const int msgs_cnt = 100;
- size_t i, found;
- char *topic;
- rd_kafka_metadata_topic_t exp_mdtopic = {0};
- int64_t testid = test_id_generate();
- test_timing_t timing;
- rd_kafka_resp_err_t exp_err = RD_KAFKA_RESP_ERR_NO_ERROR;
- const rd_kafka_ListConsumerGroups_result_t *res;
- const rd_kafka_ConsumerGroupListing_t **groups;
- rd_bool_t has_match_states =
- test_broker_version >= TEST_BRKVER(2, 7, 0, 0);
-
- SUB_TEST_QUICK(
- "%s ListConsumerGroups with %s, request_timeout %d"
- ", match_states %s",
- rd_kafka_name(rk), what, request_timeout, RD_STR_ToF(match_states));
-
- q = useq ? useq : rd_kafka_queue_new(rk);
-
- if (request_timeout != -1) {
- options = rd_kafka_AdminOptions_new(
- rk, RD_KAFKA_ADMIN_OP_LISTCONSUMERGROUPS);
-
- if (match_states) {
- rd_kafka_consumer_group_state_t empty =
- RD_KAFKA_CONSUMER_GROUP_STATE_EMPTY;
-
- TEST_CALL_ERROR__(
- rd_kafka_AdminOptions_set_match_consumer_group_states(
- options, &empty, 1));
- }
-
- TEST_CALL_ERR__(rd_kafka_AdminOptions_set_request_timeout(
- options, request_timeout, errstr, sizeof(errstr)));
- }
-
-
- topic = rd_strdup(test_mk_topic_name(__FUNCTION__, 1));
- exp_mdtopic.topic = topic;
-
- /* Create the topics first. */
- test_CreateTopics_simple(rk, NULL, &topic, 1, partitions_cnt, NULL);
-
- /* Verify that topics are reported by metadata */
- test_wait_metadata_update(rk, &exp_mdtopic, 1, NULL, 0, 15 * 1000);
-
- /* Produce 100 msgs */
- test_produce_msgs_easy(topic, testid, 0, msgs_cnt);
-
- for (i = 0; i < TEST_LIST_CONSUMER_GROUPS_CNT; i++) {
- char *group = rd_strdup(test_mk_topic_name(__FUNCTION__, 1));
- test_consume_msgs_easy(group, topic, testid, -1, msgs_cnt,
- NULL);
- list_consumer_groups[i] = group;
- }
+static void
+test_ListConsumerGroups_helper(rd_kafka_t *rk,
+ rd_kafka_AdminOptions_t *option,
+ rd_kafka_queue_t *q,
+ char **list_consumer_groups,
+ size_t list_consumer_groups_cnt,
+ size_t exp_found,
+ rd_kafka_consumer_group_type_t exp_type,
+ rd_bool_t match_states) {
+ rd_kafka_event_t *rkev = NULL;
+ const rd_kafka_ListConsumerGroups_result_t *result = NULL;
+ size_t group_cnt;
+ size_t error_cnt;
+ size_t found;
+ size_t i, j;
+ const rd_kafka_ConsumerGroupListing_t **groups = NULL;
- TIMING_START(&timing, "ListConsumerGroups");
TEST_SAY("Call ListConsumerGroups\n");
- rd_kafka_ListConsumerGroups(rk, options, q);
- TIMING_ASSERT_LATER(&timing, 0, 50);
-
- TIMING_START(&timing, "ListConsumerGroups.queue_poll");
-
+ rd_kafka_ListConsumerGroups(rk, option, q);
/* Poll result queue for ListConsumerGroups result.
* Print but otherwise ignore other event types
* (typically generic Error events). */
while (1) {
rkev = rd_kafka_queue_poll(q, tmout_multip(20 * 1000));
- TEST_SAY("ListConsumerGroups: got %s in %.3fms\n",
- rd_kafka_event_name(rkev),
- TIMING_DURATION(&timing) / 1000.0f);
if (rkev == NULL)
continue;
+
if (rd_kafka_event_error(rkev))
TEST_SAY("%s: %s\n", rd_kafka_event_name(rkev),
rd_kafka_event_error_string(rkev));
if (rd_kafka_event_type(rkev) ==
- RD_KAFKA_EVENT_LISTCONSUMERGROUPS_RESULT) {
+ RD_KAFKA_EVENT_LISTCONSUMERGROUPS_RESULT)
break;
- }
rd_kafka_event_destroy(rkev);
}
/* Convert event to proper result */
- res = rd_kafka_event_ListConsumerGroups_result(rkev);
- TEST_ASSERT(res, "expected ListConsumerGroups_result, got %s",
+ result = rd_kafka_event_ListConsumerGroups_result(rkev);
+ TEST_ASSERT(result, "expected ListConsumerGroups_result, got %s",
rd_kafka_event_name(rkev));
- /* Expecting error */
- err = rd_kafka_event_error(rkev);
- errstr2 = rd_kafka_event_error_string(rkev);
- TEST_ASSERT(err == exp_err,
- "expected ListConsumerGroups to return %s, got %s (%s)",
- rd_kafka_err2str(exp_err), rd_kafka_err2str(err),
- err ? errstr2 : "n/a");
-
- TEST_SAY("ListConsumerGroups: returned %s (%s)\n",
- rd_kafka_err2str(err), err ? errstr2 : "n/a");
-
- groups = rd_kafka_ListConsumerGroups_result_valid(res, &valid_cnt);
- rd_kafka_ListConsumerGroups_result_errors(res, &error_cnt);
+ if (rd_kafka_event_error(rkev))
+ TEST_FAIL("ListConsumerGroups failed with %s",
+ rd_kafka_event_error_string(rkev));
- /* Other tests could be running */
- TEST_ASSERT(valid_cnt >= TEST_LIST_CONSUMER_GROUPS_CNT,
- "expected ListConsumerGroups to return at least %" PRId32
- " valid groups,"
- " got %zu",
- TEST_LIST_CONSUMER_GROUPS_CNT, valid_cnt);
+ groups = rd_kafka_ListConsumerGroups_result_valid(result, &group_cnt);
+ rd_kafka_ListConsumerGroups_result_errors(result, &error_cnt);
TEST_ASSERT(error_cnt == 0,
"expected ListConsumerGroups to return 0 errors,"
@@ -2770,23 +2887,32 @@ static void do_test_ListConsumerGroups(const char *what,
error_cnt);
found = 0;
- for (i = 0; i < valid_cnt; i++) {
- int j;
- const rd_kafka_ConsumerGroupListing_t *group;
- group = groups[i];
- group_id = rd_kafka_ConsumerGroupListing_group_id(group);
- is_simple_consumer_group =
+ for (i = 0; i < group_cnt; i++) {
+ const rd_kafka_ConsumerGroupListing_t *group = groups[i];
+ const char *group_id =
+ rd_kafka_ConsumerGroupListing_group_id(group);
+ int is_simple_consumer_group =
rd_kafka_ConsumerGroupListing_is_simple_consumer_group(
group);
- state = rd_kafka_ConsumerGroupListing_state(group);
- for (j = 0; j < TEST_LIST_CONSUMER_GROUPS_CNT; j++) {
+ rd_kafka_consumer_group_state_t state =
+ rd_kafka_ConsumerGroupListing_state(group);
+ rd_kafka_consumer_group_type_t type =
+ rd_kafka_ConsumerGroupListing_type(group);
+ for (j = 0; j < list_consumer_groups_cnt; j++) {
if (!strcmp(list_consumer_groups[j], group_id)) {
found++;
TEST_ASSERT(!is_simple_consumer_group,
"expected a normal group,"
" got a simple group");
+ if (exp_type !=
+ RD_KAFKA_CONSUMER_GROUP_TYPE_UNKNOWN) {
+ TEST_ASSERT(
+ type == exp_type,
+ "Expected the Consumer Group Type "
+ "to be set by the Broker.");
+ }
- if (!has_match_states)
+ if (!match_states)
break;
TEST_ASSERT(
@@ -2799,13 +2925,137 @@ static void do_test_ListConsumerGroups(const char *what,
}
}
}
- TEST_ASSERT(found == TEST_LIST_CONSUMER_GROUPS_CNT,
- "expected to find %d"
- " started groups,"
+
+ TEST_ASSERT(found == exp_found,
+ "expected to find %" PRIusz
+ " consumer groups,"
" got %" PRIusz,
- TEST_LIST_CONSUMER_GROUPS_CNT, found);
+ exp_found, found);
rd_kafka_event_destroy(rkev);
+}
+
+/**
+ * @brief Test list groups, creating consumers for a set of groups,
+ * listing and deleting them at the end.
+ */
+static void do_test_ListConsumerGroups(const char *what,
+ rd_kafka_t *rk,
+ rd_kafka_queue_t *useq,
+ int request_timeout,
+ rd_bool_t match_states,
+ rd_bool_t match_types) {
+#define TEST_LIST_CONSUMER_GROUPS_CNT 4
+ rd_kafka_queue_t *q;
+ rd_kafka_AdminOptions_t *options;
+ char errstr[512];
+ char *list_consumer_groups[TEST_LIST_CONSUMER_GROUPS_CNT];
+ const int partitions_cnt = 1;
+ const int msgs_cnt = 100;
+ char *topic;
+ size_t i;
+ rd_kafka_metadata_topic_t exp_mdtopic = {0};
+ int64_t testid = test_id_generate();
+ rd_bool_t has_match_states =
+ test_broker_version >= TEST_BRKVER(2, 7, 0, 0);
+ rd_bool_t has_match_types =
+ test_broker_version >= TEST_BRKVER(3, 8, 0, 0);
+
+ rd_kafka_consumer_group_type_t group_protocol_in_use =
+ RD_KAFKA_CONSUMER_GROUP_TYPE_UNKNOWN;
+ rd_kafka_consumer_group_type_t group_protocol_not_in_use =
+ RD_KAFKA_CONSUMER_GROUP_TYPE_UNKNOWN;
+
+ SUB_TEST_QUICK(
+ "%s ListConsumerGroups with %s, request_timeout %d"
+ ", match_states %s, match_types %s",
+ rd_kafka_name(rk), what, request_timeout, RD_STR_ToF(match_states),
+ RD_STR_ToF(match_types));
+
+ /* match_states would not work if broker version is below 2.7.0 */
+ if (!has_match_states)
+ match_states = rd_false;
+
+ /* match_types would not work if broker version is below 3.8.0 */
+ if (!has_match_types)
+ match_types = rd_false;
+
+ q = useq ? useq : rd_kafka_queue_new(rk);
+
+ options =
+ rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_LISTCONSUMERGROUPS);
+
+ if (request_timeout != -1) {
+ TEST_CALL_ERR__(rd_kafka_AdminOptions_set_request_timeout(
+ options, request_timeout, errstr, sizeof(errstr)));
+ }
+
+ if (match_states) {
+ rd_kafka_consumer_group_state_t empty =
+ RD_KAFKA_CONSUMER_GROUP_STATE_EMPTY;
+
+ TEST_CALL_ERROR__(
+ rd_kafka_AdminOptions_set_match_consumer_group_states(
+ options, &empty, 1));
+ }
+
+ if (match_types) {
+ if (test_consumer_group_protocol_classic()) {
+ group_protocol_in_use =
+ RD_KAFKA_CONSUMER_GROUP_TYPE_CLASSIC;
+ group_protocol_not_in_use =
+ RD_KAFKA_CONSUMER_GROUP_TYPE_CONSUMER;
+ } else {
+ group_protocol_in_use =
+ RD_KAFKA_CONSUMER_GROUP_TYPE_CONSUMER;
+ group_protocol_not_in_use =
+ RD_KAFKA_CONSUMER_GROUP_TYPE_CLASSIC;
+ }
+ TEST_CALL_ERROR__(
+ rd_kafka_AdminOptions_set_match_consumer_group_types(
+ options, &group_protocol_in_use, 1));
+ }
+
+ topic = rd_strdup(test_mk_topic_name(__FUNCTION__, 1));
+ exp_mdtopic.topic = topic;
+
+ /* Create the topics first. */
+ test_CreateTopics_simple(rk, NULL, &topic, 1, partitions_cnt, NULL);
+
+ /* Verify that topics are reported by metadata */
+ test_wait_metadata_update(rk, &exp_mdtopic, 1, NULL, 0, 15 * 1000);
+
+ /* Produce 100 msgs */
+ test_produce_msgs_easy(topic, testid, 0, msgs_cnt);
+
+ for (i = 0; i < TEST_LIST_CONSUMER_GROUPS_CNT; i++) {
+ char *group = rd_strdup(test_mk_topic_name(__FUNCTION__, 1));
+ test_consume_msgs_easy(group, topic, testid, -1, msgs_cnt,
+ NULL);
+ list_consumer_groups[i] = group;
+ }
+
+ test_ListConsumerGroups_helper(rk, options, q, list_consumer_groups,
+ TEST_LIST_CONSUMER_GROUPS_CNT,
+ TEST_LIST_CONSUMER_GROUPS_CNT,
+ group_protocol_in_use, has_match_states);
+
+ if (match_types) {
+ /* Simply test with the option of protocol not in use */
+ rd_kafka_AdminOptions_t *option_group_protocol_not_in_use =
+ rd_kafka_AdminOptions_new(
+ rk, RD_KAFKA_ADMIN_OP_LISTCONSUMERGROUPS);
+ rd_kafka_AdminOptions_set_match_consumer_group_types(
+ option_group_protocol_not_in_use,
+ &group_protocol_not_in_use, 1);
+
+ test_ListConsumerGroups_helper(
+ rk, option_group_protocol_not_in_use, q,
+ list_consumer_groups, TEST_LIST_CONSUMER_GROUPS_CNT, 0,
+ group_protocol_not_in_use, rd_false);
+
+ rd_kafka_AdminOptions_destroy(option_group_protocol_not_in_use);
+ }
test_DeleteGroups_simple(rk, NULL, (char **)list_consumer_groups,
TEST_LIST_CONSUMER_GROUPS_CNT, NULL);
@@ -2816,8 +3066,7 @@ static void do_test_ListConsumerGroups(const char *what,
rd_free(topic);
- if (options)
- rd_kafka_AdminOptions_destroy(options);
+ rd_kafka_AdminOptions_destroy(options);
if (!useq)
rd_kafka_queue_destroy(q);
@@ -3363,6 +3612,9 @@ static void do_test_DescribeTopics(const char *what,
test_CreateAcls_simple(rk, NULL, acl_bindings, 1, NULL));
rd_kafka_AclBinding_destroy(acl_bindings[0]);
+ /* Wait for ACL propagation. */
+ rd_sleep(tmout_multip(2));
+
/* Call DescribeTopics. */
options =
rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DESCRIBETOPICS);
@@ -3422,19 +3674,21 @@ static void do_test_DescribeTopics(const char *what,
rd_kafka_event_destroy(rkev);
/*
- * Allow RD_KAFKA_ACL_OPERATION_DELETE to allow deletion
- * of the created topic as currently our principal only has read
- * and describe.
+ * Remove create ACLs to allow deletion
+ * of the created topic.
*/
acl_bindings[0] = rd_kafka_AclBinding_new(
RD_KAFKA_RESOURCE_TOPIC, topic_names[0],
RD_KAFKA_RESOURCE_PATTERN_LITERAL, principal, "*",
- RD_KAFKA_ACL_OPERATION_DELETE, RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW,
+ RD_KAFKA_ACL_OPERATION_READ, RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW,
NULL, 0);
TEST_CALL_ERR__(
- test_CreateAcls_simple(rk, NULL, acl_bindings, 1, NULL));
+ test_DeleteAcls_simple(rk, NULL, acl_bindings, 1, NULL));
rd_kafka_AclBinding_destroy(acl_bindings[0]);
+ /* Wait for ACL propagation. */
+ rd_sleep(tmout_multip(2));
+
done:
test_DeleteTopics_simple(rk, NULL, topic_names, 1, NULL);
if (!rkqu)
@@ -3585,6 +3839,9 @@ static void do_test_DescribeCluster(const char *what,
test_CreateAcls_simple(rk, NULL, acl_bindings, 1, NULL);
rd_kafka_AclBinding_destroy(acl_bindings[0]);
+ /* Wait for ACL propagation. */
+ rd_sleep(tmout_multip(2));
+
/* Call DescribeCluster. */
options =
rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER);
@@ -3646,6 +3903,9 @@ static void do_test_DescribeCluster(const char *what,
test_DeleteAcls_simple(rk, NULL, &acl_bindings_delete, 1, NULL);
rd_kafka_AclBinding_destroy(acl_bindings_delete);
+ /* Wait for ACL propagation. */
+ rd_sleep(tmout_multip(2));
+
done:
TEST_LATER_CHECK();
@@ -3752,16 +4012,27 @@ do_test_DescribeConsumerGroups_with_authorized_ops(const char *what,
rd_kafka_error_string(error));
{
- const rd_kafka_AclOperation_t expected[] = {
+ const rd_kafka_AclOperation_t expected_ak3[] = {
RD_KAFKA_ACL_OPERATION_DELETE,
RD_KAFKA_ACL_OPERATION_DESCRIBE,
RD_KAFKA_ACL_OPERATION_READ};
+ const rd_kafka_AclOperation_t expected_ak4[] = {
+ RD_KAFKA_ACL_OPERATION_DELETE,
+ RD_KAFKA_ACL_OPERATION_DESCRIBE,
+ RD_KAFKA_ACL_OPERATION_READ,
+ RD_KAFKA_ACL_OPERATION_DESCRIBE_CONFIGS,
+ RD_KAFKA_ACL_OPERATION_ALTER_CONFIGS};
authorized_operations =
rd_kafka_ConsumerGroupDescription_authorized_operations(
results[0], &authorized_operations_cnt);
- test_match_authorized_operations(expected, 3,
- authorized_operations,
- authorized_operations_cnt);
+ if (test_broker_version < TEST_BRKVER(4, 0, 0, 0))
+ test_match_authorized_operations(
+ expected_ak3, 3, authorized_operations,
+ authorized_operations_cnt);
+ else
+ test_match_authorized_operations(
+ expected_ak4, 5, authorized_operations,
+ authorized_operations_cnt);
}
rd_kafka_event_destroy(rkev);
@@ -3835,13 +4106,12 @@ do_test_DescribeConsumerGroups_with_authorized_ops(const char *what,
acl_bindings[0] = rd_kafka_AclBinding_new(
RD_KAFKA_RESOURCE_GROUP, group_id,
RD_KAFKA_RESOURCE_PATTERN_LITERAL, principal, "*",
- RD_KAFKA_ACL_OPERATION_DELETE, RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW,
+ RD_KAFKA_ACL_OPERATION_READ, RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW,
NULL, 0);
- test_CreateAcls_simple(rk, NULL, acl_bindings, 1, NULL);
+ test_DeleteAcls_simple(rk, NULL, acl_bindings, 1, NULL);
rd_kafka_AclBinding_destroy(acl_bindings[0]);
- /* It seems to be taking some time on the cluster for the ACLs to
- * propagate for a group.*/
+ /* Wait for ACL propagation. */
rd_sleep(tmout_multip(2));
test_DeleteGroups_simple(rk, NULL, &group_id, 1, NULL);
@@ -3938,8 +4208,6 @@ static void do_test_DeleteConsumerGroupOffsets(const char *what,
test_wait_metadata_update(rk, exp_mdtopics, exp_mdtopic_cnt, NULL, 0,
15 * 1000);
- rd_sleep(1); /* Additional wait time for cluster propagation */
-
consumer = test_create_consumer(groupid, NULL, NULL, NULL);
if (sub_consumer) {
@@ -4215,8 +4483,6 @@ static void do_test_AlterConsumerGroupOffsets(const char *what,
test_wait_metadata_update(rk, exp_mdtopics, exp_mdtopic_cnt,
NULL, 0, 15 * 1000);
- rd_sleep(1); /* Additional wait time for cluster propagation */
-
consumer = test_create_consumer(group_id, NULL, NULL, NULL);
if (sub_consumer) {
@@ -4499,8 +4765,6 @@ static void do_test_ListConsumerGroupOffsets(const char *what,
test_wait_metadata_update(rk, exp_mdtopics, exp_mdtopic_cnt, NULL, 0,
15 * 1000);
- rd_sleep(1); /* Additional wait time for cluster propagation */
-
consumer = test_create_consumer(group_id, NULL, NULL, NULL);
if (sub_consumer) {
@@ -4681,7 +4945,7 @@ static void do_test_UserScramCredentials(const char *what,
size_t password_size = 8;
rd_kafka_queue_t *queue;
const char *users[1];
- users[0] = "testuserforscram";
+ users[0] = test_mk_topic_name("testuserforscram", 1);
if (null_bytes) {
salt[1] = '\0';
@@ -4801,6 +5065,9 @@ static void do_test_UserScramCredentials(const char *what,
rd_kafka_event_destroy(event);
#endif
+ /* Wait for user propagation. */
+ rd_sleep(tmout_multip(2));
+
/* Credential should be retrieved */
options = rd_kafka_AdminOptions_new(
rk, RD_KAFKA_ADMIN_OP_DESCRIBEUSERSCRAMCREDENTIALS);
@@ -4913,6 +5180,9 @@ static void do_test_UserScramCredentials(const char *what,
final_checks:
#endif
+ /* Wait for user propagation. */
+ rd_sleep(tmout_multip(2));
+
/* Credential doesn't exist anymore for this user */
options = rd_kafka_AdminOptions_new(
@@ -5013,6 +5283,8 @@ static void do_test_ListOffsets(const char *what,
test_CreateTopics_simple(rk, NULL, (char **)&topic, 1, 1, NULL);
+ test_wait_topic_exists(rk, topic, 5000);
+
p = test_create_producer();
for (i = 0; i < RD_ARRAY_SIZE(timestamps); i++) {
rd_kafka_producev(
@@ -5081,6 +5353,15 @@ static void do_test_ListOffsets(const char *what,
test_fixture.query);
continue;
}
+ if (test_fixture.query == RD_KAFKA_OFFSET_SPEC_MAX_TIMESTAMP &&
+ test_broker_version >= TEST_BRKVER(3, 7, 0, 0) &&
+ test_broker_version < TEST_BRKVER(3, 8, 0, 0)) {
+ /* 3.8.0 released fix for KAFKA-16310 */
+ TEST_SAY(
+ "Skipping offset MAX_TIMESTAMP,"
+ " as there was a regression in this version\n");
+ continue;
+ }
TEST_SAY("Testing offset %" PRId64 "\n", test_fixture.query);
@@ -5171,7 +5452,9 @@ static void do_test_apis(rd_kafka_type_t cltype) {
mainq = rd_kafka_queue_get_main(rk);
/* Create topics */
- do_test_CreateTopics("temp queue, op timeout 0", rk, NULL, 0, 0);
+ /* FIXME: KRaft async CreateTopics is working differently than
+ * wth Zookeeper
+ * do_test_CreateTopics("temp queue, op timeout 0", rk, NULL, 0, 0); */
do_test_CreateTopics("temp queue, op timeout 15000", rk, NULL, 15000,
0);
do_test_CreateTopics(
@@ -5183,15 +5466,20 @@ static void do_test_apis(rd_kafka_type_t cltype) {
do_test_CreateTopics("main queue, options", rk, mainq, -1, 0);
/* Delete topics */
- do_test_DeleteTopics("temp queue, op timeout 0", rk, NULL, 0);
+ /* FIXME: KRaft async DeleteTopics is working differently than
+ * with Zookeeper
+ * do_test_DeleteTopics("temp queue, op timeout 0", rk, NULL, 0); */
do_test_DeleteTopics("main queue, op timeout 15000", rk, mainq, 1500);
if (test_broker_version >= TEST_BRKVER(1, 0, 0, 0)) {
/* Create Partitions */
do_test_CreatePartitions("temp queue, op timeout 6500", rk,
NULL, 6500);
- do_test_CreatePartitions("main queue, op timeout 0", rk, mainq,
- 0);
+ /* FIXME: KRaft async CreatePartitions is working differently
+ * than with Zookeeper
+ * do_test_CreatePartitions("main queue, op timeout 0", rk,
+ * mainq, 0);
+ */
}
/* CreateAcls */
@@ -5216,18 +5504,29 @@ static void do_test_apis(rd_kafka_type_t cltype) {
/* DescribeConfigs */
do_test_DescribeConfigs(rk, mainq);
+ do_test_DescribeConfigs_groups(rk, mainq);
/* Delete records */
do_test_DeleteRecords("temp queue, op timeout 0", rk, NULL, 0);
do_test_DeleteRecords("main queue, op timeout 1500", rk, mainq, 1500);
/* List groups */
- do_test_ListConsumerGroups("temp queue", rk, NULL, -1, rd_false);
- do_test_ListConsumerGroups("main queue", rk, mainq, 1500, rd_true);
-
- /* Describe groups */
- do_test_DescribeConsumerGroups("temp queue", rk, NULL, -1);
- do_test_DescribeConsumerGroups("main queue", rk, mainq, 1500);
+ do_test_ListConsumerGroups("temp queue", rk, NULL, -1, rd_false,
+ rd_true);
+ do_test_ListConsumerGroups("temp queue", rk, NULL, -1, rd_false,
+ rd_false);
+ do_test_ListConsumerGroups("main queue", rk, mainq, 1500, rd_true,
+ rd_true);
+ do_test_ListConsumerGroups("main queue", rk, mainq, 1500, rd_true,
+ rd_false);
+
+ /* TODO: check this test after KIP-848 admin operation
+ * implementation */
+ if (test_consumer_group_protocol_classic()) {
+ /* Describe groups */
+ do_test_DescribeConsumerGroups("temp queue", rk, NULL, -1);
+ do_test_DescribeConsumerGroups("main queue", rk, mainq, 1500);
+ }
/* Describe topics */
do_test_DescribeTopics("temp queue", rk, NULL, 15000, rd_false);
@@ -5279,6 +5578,7 @@ static void do_test_apis(rd_kafka_type_t cltype) {
do_test_AlterConsumerGroupOffsets(
"main queue, nonexistent topics", rk, mainq, 1500, rd_false,
rd_false /* don't create topics */);
+
do_test_AlterConsumerGroupOffsets(
"main queue", rk, mainq, 1500,
rd_true, /*with subscribing consumer*/
diff --git a/tests/0082-fetch_max_bytes.cpp b/tests/0082-fetch_max_bytes.cpp
index 4ecb370f75..8d857dbfe2 100644
--- a/tests/0082-fetch_max_bytes.cpp
+++ b/tests/0082-fetch_max_bytes.cpp
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2016-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -87,6 +88,9 @@ static void do_test_fetch_max_bytes(void) {
Test::Fail("Failed to create KafkaConsumer: " + errstr);
delete conf;
+ /* For next consumer */
+ test_wait_topic_exists(c->c_ptr(), topic.c_str(), 5000);
+
/* Subscribe */
std::vector topics;
topics.push_back(topic);
diff --git a/tests/0083-cb_event.c b/tests/0083-cb_event.c
index ec84ee6e99..628fe0108e 100644
--- a/tests/0083-cb_event.c
+++ b/tests/0083-cb_event.c
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2018-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -101,6 +102,7 @@ int main_0083_cb_event(int argc, char **argv) {
err = test_auto_create_topic_rkt(rk_p, rkt_p, tmout_multip(5000));
TEST_ASSERT(!err, "Topic auto creation failed: %s",
rd_kafka_err2str(err));
+ test_wait_topic_exists(rk_p, topic, 5000);
test_conf_init(&conf, &tconf, 0);
rd_kafka_conf_set_events(conf, RD_KAFKA_EVENT_REBALANCE);
@@ -152,13 +154,16 @@ int main_0083_cb_event(int argc, char **argv) {
"expecting message\n");
if (rd_kafka_event_error(rkev) ==
RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS) {
- rd_kafka_assign(
- rk_c,
+ test_consumer_assign_by_rebalance_protocol(
+ "rebalance event", rk_c,
rd_kafka_event_topic_partition_list(
rkev));
expecting_io = _NOPE;
} else
- rd_kafka_assign(rk_c, NULL);
+ test_consumer_unassign_by_rebalance_protocol(
+ "rebalance event", rk_c,
+ rd_kafka_event_topic_partition_list(
+ rkev));
break;
case RD_KAFKA_EVENT_FETCH:
diff --git a/tests/0084-destroy_flags.c b/tests/0084-destroy_flags.c
index df98a742d7..c2c7a5ad7d 100644
--- a/tests/0084-destroy_flags.c
+++ b/tests/0084-destroy_flags.c
@@ -42,22 +42,7 @@ static void destroy_flags_rebalance_cb(rd_kafka_t *rk,
rd_kafka_topic_partition_list_t *parts,
void *opaque) {
rebalance_cnt++;
-
- TEST_SAY("rebalance_cb: %s with %d partition(s)\n",
- rd_kafka_err2str(err), parts->cnt);
-
- switch (err) {
- case RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS:
- test_consumer_assign("rebalance", rk, parts);
- break;
-
- case RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS:
- test_consumer_unassign("rebalance", rk);
- break;
-
- default:
- TEST_FAIL("rebalance_cb: error: %s", rd_kafka_err2str(err));
- }
+ test_rebalance_cb(rk, err, parts, opaque);
}
struct df_args {
@@ -162,6 +147,8 @@ static void do_test_destroy_flags(const char *topic,
destroy_flags, args->client_type, args->produce_cnt,
args->consumer_subscribe, args->consumer_unsubscribe,
local_mode ? "local" : "broker");
+
+ TEST_LATER_CHECK();
}
@@ -184,8 +171,7 @@ static void destroy_flags(int local_mode) {
/* Create the topic to avoid not-yet-auto-created-topics being
* subscribed to (and thus raising an error). */
if (!local_mode) {
- test_create_topic(NULL, topic, 3, 1);
- test_wait_topic_exists(NULL, topic, 5000);
+ test_create_topic_wait_exists(NULL, topic, 3, 1, 5000);
}
for (i = 0; i < (int)RD_ARRAYSIZE(args); i++) {
@@ -202,11 +188,21 @@ static void destroy_flags(int local_mode) {
int main_0084_destroy_flags_local(int argc, char **argv) {
+ /* FIXME: fix the test with subscribe/unsubscribe PR. */
+ if (!test_consumer_group_protocol_classic()) {
+ TEST_SKIP("FIXME: fix the test with subscribe/unsubscribe PR");
+ return 0;
+ }
destroy_flags(1 /*no brokers*/);
return 0;
}
int main_0084_destroy_flags(int argc, char **argv) {
+ /* FIXME: fix the test with subscribe/unsubscribe PR. */
+ if (!test_consumer_group_protocol_classic()) {
+ TEST_SKIP("FIXME: fix the test with subscribe/unsubscribe PR");
+ return 0;
+ }
destroy_flags(0 /*with brokers*/);
return 0;
}
diff --git a/tests/0085-headers.cpp b/tests/0085-headers.cpp
index aa9c424641..785acb5310 100644
--- a/tests/0085-headers.cpp
+++ b/tests/0085-headers.cpp
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2012-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -357,6 +358,8 @@ int main_0085_headers(int argc, char **argv) {
delete conf;
+ Test::create_topic_wait_exists(p, topic.c_str(), 1, -1, 5000);
+
std::vector parts;
parts.push_back(RdKafka::TopicPartition::create(
topic, 0, RdKafka::Topic::OFFSET_BEGINNING));
@@ -379,6 +382,7 @@ int main_0085_headers(int argc, char **argv) {
test_failed_produce();
test_assignment_op();
+ Test::delete_topic(p, topic.c_str());
c->close();
delete c;
delete p;
diff --git a/tests/0086-purge.c b/tests/0086-purge.c
index 1bf235a313..115713fb8c 100644
--- a/tests/0086-purge.c
+++ b/tests/0086-purge.c
@@ -57,8 +57,13 @@ struct waitmsgs {
};
static mtx_t produce_req_lock;
+static mtx_t produce_sent_req_lock;
static cnd_t produce_req_cnd;
+static cnd_t produce_sent_req_cnd;
+/* Number of completed produce requests, increased on response */
static int produce_req_cnt = 0;
+/* Number of sent produce requests, increased on request */
+static int produce_sent_req_cnt = 0;
#if WITH_SOCKEM
@@ -75,10 +80,13 @@ static rd_kafka_resp_err_t on_request_sent(rd_kafka_t *rk,
size_t size,
void *ic_opaque) {
- /* Save socket fd to limit ProduceRequest */
- if (ApiKey == RD_KAFKAP_ApiVersion) {
+ if (ApiKey == RD_KAFKAP_Produce) {
+ mtx_lock(&produce_sent_req_lock);
+ produce_sent_req_cnt++;
test_sockfd = sockfd;
- return RD_KAFKA_RESP_ERR_NO_ERROR;
+ if (produce_sent_req_cnt > 1)
+ cnd_broadcast(&produce_sent_req_cnd);
+ mtx_unlock(&produce_sent_req_lock);
}
return RD_KAFKA_RESP_ERR_NO_ERROR;
@@ -226,6 +234,8 @@ do_test_purge(const char *what, int remote, int idempotence, int gapless) {
test_conf_set(conf, "enable.gapless.guarantee",
gapless ? "true" : "false");
rd_kafka_conf_set_dr_msg_cb(conf, dr_msg_cb);
+ produce_req_cnt = 0;
+ produce_sent_req_cnt = 0;
if (remote) {
#if WITH_SOCKEM
@@ -238,12 +248,18 @@ do_test_purge(const char *what, int remote, int idempotence, int gapless) {
test_curr->is_fatal_cb = gapless_is_not_fatal_cb;
mtx_init(&produce_req_lock, mtx_plain);
+ mtx_init(&produce_sent_req_lock, mtx_plain);
cnd_init(&produce_req_cnd);
+ cnd_init(&produce_sent_req_cnd);
} else {
test_conf_set(conf, "bootstrap.servers", NULL);
}
rk = test_create_handle(RD_KAFKA_PRODUCER, conf);
+ if (remote)
+ /* Pre-create the topic to avoid unknown topic errors
+ * because of metadata propagation */
+ test_create_topic_wait_exists(NULL, topic, 3, -1, 5000);
TEST_SAY("Producing %d messages to topic %s\n", msgcnt, topic);
@@ -269,16 +285,16 @@ do_test_purge(const char *what, int remote, int idempotence, int gapless) {
rd_kafka_err2str(err));
waitmsgs.exp_err[i] =
- (remote && i < 10
- ? RD_KAFKA_RESP_ERR_NO_ERROR
- : remote && i < 20 ? RD_KAFKA_RESP_ERR__PURGE_INFLIGHT
- : RD_KAFKA_RESP_ERR__PURGE_QUEUE);
+ (remote && i < 10 ? RD_KAFKA_RESP_ERR_NO_ERROR
+ : remote && i < 20 ? RD_KAFKA_RESP_ERR__PURGE_INFLIGHT
+ : RD_KAFKA_RESP_ERR__PURGE_QUEUE);
waitmsgs.cnt++;
}
if (remote) {
+ test_timing_t t_produce;
/* Wait for ProduceRequest to be sent */
mtx_lock(&produce_req_lock);
cnd_timedwait_ms(&produce_req_cnd, &produce_req_lock,
@@ -287,6 +303,23 @@ do_test_purge(const char *what, int remote, int idempotence, int gapless) {
"First Produce request should've been sent by now");
mtx_unlock(&produce_req_lock);
+ /* Wait second ProduceRequest is in-flight */
+ TIMING_START(&t_produce, "PRODUCE");
+ mtx_lock(&produce_sent_req_lock);
+ if (produce_sent_req_cnt < 2) {
+ cnd_timedwait_ms(&produce_sent_req_cnd,
+ &produce_sent_req_lock, 15 * 1000);
+ }
+ TEST_ASSERT(produce_sent_req_cnt == 2,
+ "No more than 2 requests should've "
+ "been sent by now, "
+ "%d sent produce requests",
+ produce_sent_req_cnt);
+ mtx_unlock(&produce_sent_req_lock);
+ /* Even if still not in-flight, it should be sent
+ * almost immediately given there are 19 messages in queue. */
+ TIMING_ASSERT_LATER(&t_produce, 0, 100);
+
purge_and_expect(what, __LINE__, rk, RD_KAFKA_PURGE_F_QUEUE,
&waitmsgs, 10,
"in-flight messages should not be purged");
diff --git a/tests/0088-produce_metadata_timeout.c b/tests/0088-produce_metadata_timeout.c
index 68d02449c1..a34cbfa38b 100644
--- a/tests/0088-produce_metadata_timeout.c
+++ b/tests/0088-produce_metadata_timeout.c
@@ -114,7 +114,7 @@ int main_0088_produce_metadata_timeout(int argc, char **argv) {
rk = test_create_handle(RD_KAFKA_PRODUCER, conf);
/* Create topic with single partition, for simplicity. */
- test_create_topic(rk, topic, 1, 1);
+ test_create_topic_wait_exists(rk, topic, 1, 1, 5000);
rkt = rd_kafka_topic_new(rk, topic, NULL);
diff --git a/tests/0089-max_poll_interval.c b/tests/0089-max_poll_interval.c
index 2089af9907..3678ea0928 100644
--- a/tests/0089-max_poll_interval.c
+++ b/tests/0089-max_poll_interval.c
@@ -61,7 +61,7 @@ static void do_test(void) {
testid = test_id_generate();
- test_create_topic(NULL, topic, 1, 1);
+ test_create_topic_wait_exists(NULL, topic, 1, 1, 5000);
test_produce_msgs_easy(topic, testid, -1, msgcnt);
@@ -212,7 +212,7 @@ static void do_test_with_log_queue(void) {
testid = test_id_generate();
- test_create_topic(NULL, topic, 1, 1);
+ test_create_topic_wait_exists(NULL, topic, 1, 1, 5000);
test_produce_msgs_easy(topic, testid, -1, msgcnt);
@@ -380,7 +380,7 @@ do_test_rejoin_after_interval_expire(rd_bool_t forward_to_another_q,
"%d",
forward_to_another_q, forward_to_consumer_q);
- test_create_topic(NULL, topic, 1, 1);
+ test_create_topic_wait_exists(NULL, topic, 1, 1, 5000);
test_str_id_generate(groupid, sizeof(groupid));
test_conf_init(&conf, NULL, 60);
@@ -413,7 +413,9 @@ do_test_rejoin_after_interval_expire(rd_bool_t forward_to_another_q,
TEST_ASSERT(rd_kafka_event_error(event) ==
RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS,
"Group join should assign partitions");
- rd_kafka_assign(rk, rd_kafka_event_topic_partition_list(event));
+ test_consumer_assign_by_rebalance_protocol(
+ "initial group join", rk,
+ rd_kafka_event_topic_partition_list(event));
rd_kafka_event_destroy(event);
rd_sleep(10 + 1); /* Exceed max.poll.interval.ms. */
@@ -426,7 +428,8 @@ do_test_rejoin_after_interval_expire(rd_bool_t forward_to_another_q,
TEST_ASSERT(rd_kafka_event_error(event) ==
RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS,
"Group leave should revoke partitions");
- rd_kafka_assign(rk, NULL);
+ test_consumer_unassign_by_rebalance_protocol(
+ "group leave", rk, rd_kafka_event_topic_partition_list(event));
rd_kafka_event_destroy(event);
event = test_wait_event(polling_queue, RD_KAFKA_EVENT_REBALANCE,
@@ -435,7 +438,9 @@ do_test_rejoin_after_interval_expire(rd_bool_t forward_to_another_q,
TEST_ASSERT(rd_kafka_event_error(event) ==
RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS,
"Group rejoin should assign partitions");
- rd_kafka_assign(rk, rd_kafka_event_topic_partition_list(event));
+
+ test_consumer_assign_by_rebalance_protocol(
+ "group rejoin", rk, rd_kafka_event_topic_partition_list(event));
rd_kafka_event_destroy(event);
if (forward_to_another_q)
@@ -466,7 +471,7 @@ static void do_test_max_poll_reset_with_consumer_cb(void) {
SUB_TEST();
- test_create_topic(NULL, topic, 1, 1);
+ test_create_topic_wait_exists(NULL, topic, 1, 1, 5000);
uint64_t testid = test_id_generate();
test_produce_msgs_easy(topic, testid, -1, 100);
diff --git a/tests/0090-idempotence.c b/tests/0090-idempotence.c
index c665b5f635..10975a6362 100644
--- a/tests/0090-idempotence.c
+++ b/tests/0090-idempotence.c
@@ -130,11 +130,10 @@ static void do_test_implicit_ack(const char *what,
rk = test_create_handle(RD_KAFKA_PRODUCER, conf);
- test_create_topic(rk, topic, 1, 1);
+ test_create_topic_wait_exists(rk, topic, 1, 1, 5000);
rkt = test_create_producer_topic(rk, topic, NULL);
-
TEST_SAY("Producing %d messages\n", msgcnt);
test_produce_msgs(rk, rkt, testid, -1, 0, msgcnt, NULL, 0);
diff --git a/tests/0091-max_poll_interval_timeout.c b/tests/0091-max_poll_interval_timeout.c
index f736c108a3..e915bb8624 100644
--- a/tests/0091-max_poll_interval_timeout.c
+++ b/tests/0091-max_poll_interval_timeout.c
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2012-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -101,10 +102,7 @@ static void rebalance_cb(rd_kafka_t *rk,
rd_kafka_name(cons->rk), cons->rebalance_cnt,
cons->max_rebalance_cnt);
- if (err == RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS)
- rd_kafka_assign(rk, parts);
- else
- rd_kafka_assign(rk, NULL);
+ test_rebalance_cb(rk, err, parts, opaque);
}
@@ -206,7 +204,7 @@ static void do_test_with_assign(const char *topic) {
test_conf_init(&conf, NULL, 60);
- test_create_topic(NULL, topic, 2, 1);
+ test_create_topic_wait_exists(NULL, topic, 2, 1, 5000);
test_conf_set(conf, "session.timeout.ms", "6000");
test_conf_set(conf, "max.poll.interval.ms", "7000" /*7s*/);
@@ -251,7 +249,7 @@ static void do_test_no_poll(const char *topic) {
test_conf_init(&conf, NULL, 60);
- test_create_topic(NULL, topic, 2, 1);
+ test_create_topic_wait_exists(NULL, topic, 2, 1, 5000);
test_conf_set(conf, "session.timeout.ms", "6000");
test_conf_set(conf, "max.poll.interval.ms", "7000" /*7s*/);
@@ -285,7 +283,7 @@ int main_0091_max_poll_interval_timeout(int argc, char **argv) {
const char *topic =
test_mk_topic_name("0091_max_poll_interval_tmout", 1);
- test_create_topic(NULL, topic, 2, 1);
+ test_create_topic_wait_exists(NULL, topic, 2, 1, 5000);
do_test_with_subscribe(topic);
diff --git a/tests/0092-mixed_msgver.c b/tests/0092-mixed_msgver.c
index 877fc48e07..112239a93c 100644
--- a/tests/0092-mixed_msgver.c
+++ b/tests/0092-mixed_msgver.c
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2018-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -54,6 +55,11 @@ int main_0092_mixed_msgver(int argc, char **argv) {
return 0;
}
+ if (test_broker_version >= TEST_BRKVER(4, 0, 0, 0)) {
+ TEST_SKIP("MsgVersion 1 was removed in Apache Kafka 4.0\n");
+ return 0;
+ }
+
testid = test_id_generate();
rk = test_create_producer();
diff --git a/tests/0093-holb.c b/tests/0093-holb.c
index 8e80b1550e..65fa4083a6 100644
--- a/tests/0093-holb.c
+++ b/tests/0093-holb.c
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2012-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -91,10 +92,7 @@ static void rebalance_cb(rd_kafka_t *rk,
rd_kafka_name(cons->rk), cons->rebalance_cnt,
cons->max_rebalance_cnt);
- if (err == RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS)
- rd_kafka_assign(rk, parts);
- else
- rd_kafka_assign(rk, NULL);
+ test_rebalance_cb(rk, err, parts, opaque);
}
@@ -110,12 +108,17 @@ int main_0093_holb_consumer(int argc, char **argv) {
test_conf_init(&conf, NULL, 60);
- test_create_topic(NULL, topic, 1, 1);
+ test_create_topic_wait_exists(NULL, topic, 1, 1, 5000);
test_produce_msgs_easy(topic, testid, 0, msgcnt);
test_conf_set(conf, "session.timeout.ms", "6000");
test_conf_set(conf, "max.poll.interval.ms", "20000");
+ /* Socket timeout must be greater than
+ * 10s: the interval during which group rebalance is hold
+ * when the first consumer isn't polling below.
+ * 10s corresponds to the duration of the JoinGroup call.
+ * on the second consumer. */
test_conf_set(conf, "socket.timeout.ms", "3000");
test_conf_set(conf, "auto.offset.reset", "earliest");
/* Trigger other requests often */
diff --git a/tests/0094-idempotence_msg_timeout.c b/tests/0094-idempotence_msg_timeout.c
index 4f2b3cbe5f..ca2a365262 100644
--- a/tests/0094-idempotence_msg_timeout.c
+++ b/tests/0094-idempotence_msg_timeout.c
@@ -169,6 +169,7 @@ static void do_test_produce_timeout(const char *topic, const int msgrate) {
rk = test_create_handle(RD_KAFKA_PRODUCER, conf);
rkt = test_create_producer_topic(rk, topic, "message.timeout.ms",
"5000", NULL);
+ test_wait_topic_exists(rk, topic, 5000);
/* Create the topic to make sure connections are up and ready. */
err = test_auto_create_topic_rkt(rk, rkt, tmout_multip(5000));
diff --git a/tests/0097-ssl_verify.cpp b/tests/0097-ssl_verify.cpp
index a5e8885267..1f94423886 100644
--- a/tests/0097-ssl_verify.cpp
+++ b/tests/0097-ssl_verify.cpp
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2019-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -34,6 +35,8 @@
#include "testcpp.h"
#include "tinycthread.h"
+namespace TestSSLVerify {
+
static const std::string envname[RdKafka::CERT__CNT][RdKafka::CERT_ENC__CNT] = {
/* [RdKafka::CERT_PUBLIC_KEY] = */
{
@@ -117,6 +120,51 @@ class TestVerifyCb : public RdKafka::SslCertificateVerifyCb {
}
};
+/**
+ * @name Test event callback.
+ */
+class TestEventCb : public RdKafka::EventCb {
+ public:
+ bool should_succeed;
+
+ TestEventCb(bool should_succeed) : should_succeed(should_succeed) {
+ }
+
+ void event_cb(RdKafka::Event &event) {
+ switch (event.type()) {
+ case RdKafka::Event::EVENT_LOG:
+ Test::Say("Log: " + event.str() + "\n");
+ break;
+ case RdKafka::Event::EVENT_ERROR:
+ if (should_succeed)
+ Test::Fail("Unexpected error event, got: " + event.str());
+ else if (event.err() != RdKafka::ERR__SSL &&
+ event.err() != RdKafka::ERR__ALL_BROKERS_DOWN)
+ Test::Fail(
+ "Expected _SSL or _ALL_BROKERS_DOWN error codes"
+ ", got: " +
+ RdKafka::err2str(event.err()));
+ else if (event.err() == RdKafka::ERR__SSL) {
+ bool expected = false;
+ Test::Say("SSL error: " + event.str() + "\n");
+ if (event.str().find("alert number 42") != std::string::npos)
+ /* Verify that certificate isn't sent if not trusted
+ * by the broker. We should receive 42 (bad_certificate)
+ * instead of 46 (certificate_unknown). */
+ expected = true;
+ else if (event.str().find("broker certificate could not be verified") !=
+ std::string::npos)
+ expected = true;
+
+ if (!expected)
+ Test::Fail("Unexpected SSL error message, got: " + event.str());
+ }
+ break;
+ default:
+ break;
+ }
+ }
+};
/**
* @brief Set SSL PEM cert/key using configuration property.
@@ -126,36 +174,41 @@ class TestVerifyCb : public RdKafka::SslCertificateVerifyCb {
* @param loc_prop ssl.X.location property that will be cleared.
* @param pem_prop ssl.X.pem property that will be set.
* @param cert_type Certificate type.
+ * @param use_conf_value_file Read the file from existing configuration value,
+ * instead of the one in the environment variable.
*/
static void conf_location_to_pem(RdKafka::Conf *conf,
std::string loc_prop,
std::string pem_prop,
- RdKafka::CertificateType cert_type) {
+ RdKafka::CertificateType cert_type,
+ bool use_conf_value_file) {
std::string loc;
std::string errstr;
+ if (use_conf_value_file && conf->get(loc_prop, loc) != RdKafka::Conf::CONF_OK)
+ Test::Fail("Failed to get " + loc_prop);
if (conf->set(loc_prop, "", errstr) != RdKafka::Conf::CONF_OK)
Test::Fail("Failed to reset " + loc_prop + ": " + errstr);
- const char *p;
- p = test_getenv(envname[cert_type][RdKafka::CERT_ENC_PEM].c_str(), NULL);
- if (!p)
- Test::Fail(
- "Invalid test environment: "
- "Missing " +
- envname[cert_type][RdKafka::CERT_ENC_PEM] +
- " env variable: make sure trivup is up to date");
-
- loc = p;
-
+ if (!use_conf_value_file) {
+ const char *p;
+ p = test_getenv(envname[cert_type][RdKafka::CERT_ENC_PEM].c_str(), NULL);
+ if (!p)
+ Test::Fail(
+ "Invalid test environment: "
+ "Missing " +
+ envname[cert_type][RdKafka::CERT_ENC_PEM] +
+ " env variable: make sure trivup is up to date");
+
+ loc = p;
+ }
/* Read file */
std::ifstream ifs(loc.c_str());
std::string pem((std::istreambuf_iterator(ifs)),
std::istreambuf_iterator());
- Test::Say("Read env " + envname[cert_type][RdKafka::CERT_ENC_PEM] + "=" +
- loc + " from disk and changed to in-memory " + pem_prop +
+ Test::Say("Read " + loc + " from disk and changed to in-memory " + pem_prop +
" string\n");
if (conf->set(pem_prop, pem, errstr) != RdKafka::Conf::CONF_OK)
@@ -172,7 +225,8 @@ static void conf_location_to_pem(RdKafka::Conf *conf,
static void conf_location_to_setter(RdKafka::Conf *conf,
std::string loc_prop,
RdKafka::CertificateType cert_type,
- RdKafka::CertificateEncoding encoding) {
+ RdKafka::CertificateEncoding encoding,
+ bool use_conf_value_file) {
std::string loc;
static const std::string encnames[] = {
"PKCS#12",
@@ -182,23 +236,26 @@ static void conf_location_to_setter(RdKafka::Conf *conf,
/* Clear the config property (e.g., ssl.key.location) */
std::string errstr;
+ if (use_conf_value_file && conf->get(loc_prop, loc) != RdKafka::Conf::CONF_OK)
+ Test::Fail("Failed to get " + loc_prop);
if (conf->set(loc_prop, "", errstr) != RdKafka::Conf::CONF_OK)
Test::Fail("Failed to reset " + loc_prop);
- const char *p;
- p = test_getenv(envname[cert_type][encoding].c_str(), NULL);
- if (!p)
- Test::Fail(
- "Invalid test environment: "
- "Missing " +
- envname[cert_type][encoding] +
- " env variable: make sure trivup is up to date");
-
- loc = p;
+ if (!use_conf_value_file) {
+ const char *p;
+ p = test_getenv(envname[cert_type][encoding].c_str(), NULL);
+ if (!p)
+ Test::Fail(
+ "Invalid test environment: "
+ "Missing " +
+ envname[cert_type][encoding] +
+ " env variable: make sure trivup is up to date");
+
+ loc = p;
+ }
- Test::Say(tostr() << "Reading " << loc_prop << " file " << loc << " as "
- << encnames[encoding] << " from env "
- << envname[cert_type][encoding] << "\n");
+ Test::Say(tostr() << "Reading file as " << encnames[encoding] << " from "
+ << loc << "\n");
/* Read file */
std::ifstream ifs(loc.c_str(), std::ios::binary | std::ios::ate);
@@ -231,28 +288,111 @@ static const std::string load_names[] = {
"setter",
};
+static bool is_client_auth_required() {
+ const char *C_client_auth = test_getenv("SSL_client_auth", "required");
+ std::string client_auth(C_client_auth);
+ return client_auth == "required";
+}
+/**
+ * @brief Test SSL certificate verification.
+ *
+ * @param line Test line number.
+ * @param verify_ok Expected verification result.
+ * @param untrusted_client_key Set up an untrusted client key.
+ * @param untrusted_client_key_intermediate_ca The untrusted client key is
+ * signed by an intermediate CA.
+ * @param load_key How to load the client key.
+ * @param key_enc Encoding of the client key.
+ * @param load_pub How to load the client public key.
+ * @param pub_enc Encoding of the client public key.
+ * @param load_ca How to load the CA.
+ * @param ca_enc Encoding of the CA.
+ */
static void do_test_verify(const int line,
bool verify_ok,
+ bool untrusted_client_key,
+ bool untrusted_client_key_intermediate_ca,
cert_load_t load_key,
RdKafka::CertificateEncoding key_enc,
cert_load_t load_pub,
RdKafka::CertificateEncoding pub_enc,
cert_load_t load_ca,
RdKafka::CertificateEncoding ca_enc) {
+#define TEST_FIXTURES_FOLDER "./fixtures"
+#define TEST_FIXTURES_SSL_FOLDER TEST_FIXTURES_FOLDER "/ssl/"
+#define TEST_FIXTURES_KEY_PASSWORD "use_strong_password_keystore_client2"
+
+/* Certificate directly signed by the root CA (untrusted) */
+#define TEST_CERTIFICATE_LOCATION \
+ TEST_FIXTURES_SSL_FOLDER "client2.certificate.pem"
+#define TEST_KEY_LOCATION TEST_FIXTURES_SSL_FOLDER "client2.key"
+
+/* Certificate signed by an intermediate CA (untrusted) */
+#define TEST_CERTIFICATE_INTERMEDIATE_LOCATION \
+ TEST_FIXTURES_SSL_FOLDER "client2.certificate.intermediate.pem"
+#define TEST_KEY_INTERMEDIATE_LOCATION \
+ TEST_FIXTURES_SSL_FOLDER "client2.intermediate.key"
+
+ std::string errstr, existing_key_password;
/*
* Create any type of client
*/
- std::string teststr = tostr() << line << ": "
- << "SSL cert verify: verify_ok=" << verify_ok
- << ", load_key=" << load_names[load_key]
- << ", load_pub=" << load_names[load_pub]
- << ", load_ca=" << load_names[load_ca];
+ std::string teststr =
+ tostr() << line << ": " << "SSL cert verify: verify_ok=" << verify_ok
+ << ", untrusted_client_key=" << untrusted_client_key
+ << ", untrusted_client_key_intermediate_ca="
+ << untrusted_client_key_intermediate_ca
+ << ", load_key=" << load_names[load_key]
+ << ", load_pub=" << load_names[load_pub]
+ << ", load_ca=" << load_names[load_ca];
Test::Say(_C_BLU "[ " + teststr + " ]\n" _C_CLR);
RdKafka::Conf *conf;
+ std::string security_protocol;
Test::conf_init(&conf, NULL, 10);
+ if (conf->get("security.protocol", security_protocol) !=
+ RdKafka::Conf::CONF_OK)
+ Test::Fail("Failed to get security.protocol");
+ /* sasl_ssl endpoints don't require
+ * SSL authentication even when
+ * ssl.client.auth=required */
+ bool should_succeed =
+ verify_ok && (!untrusted_client_key || !is_client_auth_required() ||
+ security_protocol != "ssl");
+ TestEventCb eventCb(should_succeed);
+
+ if (conf->set("event_cb", &eventCb, errstr) != RdKafka::Conf::CONF_OK)
+ Test::Fail("Failed to set event_cb: " + errstr);
+
+ if (untrusted_client_key) {
+ /* Set an untrusted certificate, signed by a root CA or by an
+ * intermediate CA, and verify client authentication fails. */
+
+ const char *untrusted_key_location = untrusted_client_key_intermediate_ca
+ ? TEST_KEY_INTERMEDIATE_LOCATION
+ : TEST_KEY_LOCATION;
+ const char *untrusted_certificate_location =
+ untrusted_client_key_intermediate_ca
+ ? TEST_CERTIFICATE_INTERMEDIATE_LOCATION
+ : TEST_CERTIFICATE_LOCATION;
+
+ if (conf->set("ssl.key.location", untrusted_key_location, errstr) !=
+ RdKafka::Conf::CONF_OK)
+ Test::Fail("Failed to set untrusted ssl.key.location: " + errstr);
+
+ if (conf->get("ssl.key.password", existing_key_password) !=
+ RdKafka::Conf::CONF_OK)
+ Test::Fail("Failed to get existing ssl.key.password: " + errstr);
+ if (conf->set("ssl.key.password", TEST_FIXTURES_KEY_PASSWORD, errstr) !=
+ RdKafka::Conf::CONF_OK)
+ Test::Fail("Failed to set untrusted ssl.key.password: " + errstr);
+
+ if (conf->set("ssl.certificate.location", untrusted_certificate_location,
+ errstr) != RdKafka::Conf::CONF_OK)
+ Test::Fail("Failed to set untrusted ssl.certificate.location: " + errstr);
+ }
std::string val;
if (conf->get("ssl.key.location", val) != RdKafka::Conf::CONF_OK ||
@@ -266,28 +406,35 @@ static void do_test_verify(const int line,
* ssl.key.pem. Same with ssl.certificate.location -> ssl.certificate.pem. */
if (load_key == USE_CONF)
conf_location_to_pem(conf, "ssl.key.location", "ssl.key.pem",
- RdKafka::CERT_PRIVATE_KEY);
+ RdKafka::CERT_PRIVATE_KEY, true);
else if (load_key == USE_SETTER)
conf_location_to_setter(conf, "ssl.key.location", RdKafka::CERT_PRIVATE_KEY,
- key_enc);
+ key_enc, key_enc == RdKafka::CERT_ENC_PEM);
if (load_pub == USE_CONF)
conf_location_to_pem(conf, "ssl.certificate.location",
- "ssl.certificate.pem", RdKafka::CERT_PUBLIC_KEY);
+ "ssl.certificate.pem", RdKafka::CERT_PUBLIC_KEY, true);
else if (load_pub == USE_SETTER)
conf_location_to_setter(conf, "ssl.certificate.location",
- RdKafka::CERT_PUBLIC_KEY, pub_enc);
+ RdKafka::CERT_PUBLIC_KEY, pub_enc,
+ pub_enc == RdKafka::CERT_ENC_PEM);
+
+ if (untrusted_client_key && ca_enc != RdKafka::CERT_ENC_PEM) {
+ /* Original password is needed for reading the
+ * CA certificate in the PKCS12 keystore. */
+ if (conf->set("ssl.key.password", existing_key_password, errstr) !=
+ RdKafka::Conf::CONF_OK)
+ Test::Fail("Failed to revert to existing ssl.key.password: " + errstr);
+ }
if (load_ca == USE_CONF)
conf_location_to_pem(conf, "ssl.ca.location", "ssl.ca.pem",
- RdKafka::CERT_CA);
+ RdKafka::CERT_CA, true);
else if (load_ca == USE_SETTER)
- conf_location_to_setter(conf, "ssl.ca.location", RdKafka::CERT_CA, ca_enc);
+ conf_location_to_setter(conf, "ssl.ca.location", RdKafka::CERT_CA, ca_enc,
+ ca_enc == RdKafka::CERT_ENC_PEM);
-
- std::string errstr;
conf->set("debug", "security", errstr);
-
TestVerifyCb verifyCb(verify_ok);
if (conf->set("ssl_cert_verify_cb", &verifyCb, errstr) !=
RdKafka::Conf::CONF_OK)
@@ -318,9 +465,9 @@ static void do_test_verify(const int line,
* this test. */
std::string cluster = p->clusterid(1000);
- if (verify_ok == cluster.empty())
+ if (should_succeed == cluster.empty())
Test::Fail("Expected connection to " +
- (std::string)(verify_ok ? "succeed" : "fail") +
+ (std::string)(should_succeed ? "succeed" : "fail") +
", but got clusterid '" + cluster + "'");
delete p;
@@ -381,8 +528,27 @@ static void do_test_bad_calls() {
Test::Say("Producer creation failed expectedly: " + errstr + "\n");
}
+} // namespace TestSSLVerify
+
+using namespace TestSSLVerify;
+
extern "C" {
+
+/**
+ * @brief Test SSL certificate verification with various
+ * key types and trusted or untrusted client certificates.
+ *
+ * @remark This tests can be run with a root CA signed certificate
+ * when trivup is started with "--ssl" only,
+ * or with an intermediate CA signed certificate,
+ * when trivup is started with:
+ * --conf='{"ssl_intermediate_ca": true}'
+ * or with "ssl.client.auth=requested" when started with:
+ * --conf='{"ssl_client_auth": "requested"}'
+ * or a combination of both.
+ */
int main_0097_ssl_verify(int argc, char **argv) {
+ int untrusted_client_key, untrusted_client_key_intermediate_ca;
if (!test_check_builtin("ssl")) {
Test::Skip("Test requires SSL support\n");
return 0;
@@ -396,32 +562,58 @@ int main_0097_ssl_verify(int argc, char **argv) {
do_test_bad_calls();
- do_test_verify(__LINE__, true, USE_LOCATION, RdKafka::CERT_ENC_PEM,
- USE_LOCATION, RdKafka::CERT_ENC_PEM, USE_LOCATION,
- RdKafka::CERT_ENC_PEM);
- do_test_verify(__LINE__, false, USE_LOCATION, RdKafka::CERT_ENC_PEM,
- USE_LOCATION, RdKafka::CERT_ENC_PEM, USE_LOCATION,
- RdKafka::CERT_ENC_PEM);
-
- /* Verify various priv and pub key and CA input formats */
- do_test_verify(__LINE__, true, USE_CONF, RdKafka::CERT_ENC_PEM, USE_CONF,
- RdKafka::CERT_ENC_PEM, USE_LOCATION, RdKafka::CERT_ENC_PEM);
- do_test_verify(__LINE__, true, USE_CONF, RdKafka::CERT_ENC_PEM, USE_CONF,
- RdKafka::CERT_ENC_PEM, USE_CONF, RdKafka::CERT_ENC_PEM);
- do_test_verify(__LINE__, true, USE_SETTER, RdKafka::CERT_ENC_PEM, USE_SETTER,
- RdKafka::CERT_ENC_PEM, USE_SETTER, RdKafka::CERT_ENC_PKCS12);
- do_test_verify(__LINE__, true, USE_LOCATION, RdKafka::CERT_ENC_PEM,
- USE_SETTER, RdKafka::CERT_ENC_DER, USE_SETTER,
- RdKafka::CERT_ENC_DER);
- do_test_verify(__LINE__, true, USE_LOCATION, RdKafka::CERT_ENC_PEM,
- USE_SETTER, RdKafka::CERT_ENC_DER, USE_SETTER,
- RdKafka::CERT_ENC_PEM); /* env: SSL_all_cas_pem */
- do_test_verify(__LINE__, true, USE_LOCATION, RdKafka::CERT_ENC_PEM,
- USE_SETTER, RdKafka::CERT_ENC_DER, USE_CONF,
- RdKafka::CERT_ENC_PEM); /* env: SSL_all_cas_pem */
- do_test_verify(__LINE__, true, USE_SETTER, RdKafka::CERT_ENC_PKCS12,
- USE_SETTER, RdKafka::CERT_ENC_PKCS12, USE_SETTER,
- RdKafka::CERT_ENC_PKCS12);
+ for (untrusted_client_key = 0; untrusted_client_key <= 1;
+ untrusted_client_key++) {
+ for (untrusted_client_key_intermediate_ca = 0;
+ untrusted_client_key_intermediate_ca <= untrusted_client_key;
+ untrusted_client_key_intermediate_ca++) {
+ do_test_verify(__LINE__, true /*verify ok*/, untrusted_client_key,
+ untrusted_client_key_intermediate_ca, USE_LOCATION,
+ RdKafka::CERT_ENC_PEM, USE_LOCATION, RdKafka::CERT_ENC_PEM,
+ USE_LOCATION, RdKafka::CERT_ENC_PEM);
+ do_test_verify(__LINE__, false /*verify not ok*/, untrusted_client_key,
+ untrusted_client_key_intermediate_ca, USE_LOCATION,
+ RdKafka::CERT_ENC_PEM, USE_LOCATION, RdKafka::CERT_ENC_PEM,
+ USE_LOCATION, RdKafka::CERT_ENC_PEM);
+
+ /* Verify various priv and pub key and CA input formats */
+ do_test_verify(__LINE__, true /*verify ok*/, untrusted_client_key,
+ untrusted_client_key_intermediate_ca, USE_CONF,
+ RdKafka::CERT_ENC_PEM, USE_CONF, RdKafka::CERT_ENC_PEM,
+ USE_LOCATION, RdKafka::CERT_ENC_PEM);
+ do_test_verify(__LINE__, true /*verify ok*/, untrusted_client_key,
+ untrusted_client_key_intermediate_ca, USE_CONF,
+ RdKafka::CERT_ENC_PEM, USE_CONF, RdKafka::CERT_ENC_PEM,
+ USE_CONF, RdKafka::CERT_ENC_PEM);
+ do_test_verify(__LINE__, true /*verify ok*/, untrusted_client_key,
+ untrusted_client_key_intermediate_ca, USE_SETTER,
+ RdKafka::CERT_ENC_PEM, USE_SETTER, RdKafka::CERT_ENC_PEM,
+ USE_SETTER, RdKafka::CERT_ENC_PKCS12);
+ }
+ }
+
+ if (test_getenv("SSL_intermediate_pub_pem", NULL) == NULL) {
+ Test::Say("Running root CA only tests\n");
+ /* DER format can contain only a single certificate so it's
+ * not suited for sending the complete chain of trust
+ * corresponding to the private key,
+ * that is necessary when using an intermediate CA. */
+ do_test_verify(__LINE__, true /*verify ok*/, false, false, USE_LOCATION,
+ RdKafka::CERT_ENC_PEM, USE_SETTER, RdKafka::CERT_ENC_DER,
+ USE_SETTER, RdKafka::CERT_ENC_DER);
+ do_test_verify(__LINE__, true /*verify ok*/, false, false, USE_LOCATION,
+ RdKafka::CERT_ENC_PEM, USE_SETTER, RdKafka::CERT_ENC_DER,
+ USE_SETTER,
+ RdKafka::CERT_ENC_PEM); /* env: SSL_all_cas_pem */
+ do_test_verify(__LINE__, true /*verify ok*/, false, false, USE_LOCATION,
+ RdKafka::CERT_ENC_PEM, USE_SETTER, RdKafka::CERT_ENC_DER,
+ USE_CONF, RdKafka::CERT_ENC_PEM); /* env: SSL_all_cas_pem */
+ Test::Say("Finished running root CA only tests\n");
+ }
+
+ do_test_verify(__LINE__, true /*verify ok*/, false, false, USE_SETTER,
+ RdKafka::CERT_ENC_PKCS12, USE_SETTER, RdKafka::CERT_ENC_PKCS12,
+ USE_SETTER, RdKafka::CERT_ENC_PKCS12);
return 0;
}
diff --git a/tests/0098-consumer-txn.cpp b/tests/0098-consumer-txn.cpp
index 6045e785a3..6d034a4361 100644
--- a/tests/0098-consumer-txn.cpp
+++ b/tests/0098-consumer-txn.cpp
@@ -500,7 +500,7 @@ static void do_test_consumer_txn_test(bool use_java_producer) {
topic_name = Test::mk_topic_name("0098-consumer_txn-0", 1);
c = create_consumer(topic_name, "READ_COMMITTED");
- Test::create_topic(c, topic_name.c_str(), 1, 3);
+ Test::create_topic_wait_exists(c, topic_name.c_str(), 1, 3, 5000);
run_producer("producer1, -1, 0x0, 5, BeginCommit, DoFlush",
"producer1, -1, 0x10, 5, BeginAbort, DoFlush");
@@ -554,7 +554,7 @@ static void do_test_consumer_txn_test(bool use_java_producer) {
topic_name = Test::mk_topic_name("0098-consumer_txn-0.1", 1);
c = create_consumer(topic_name, "READ_COMMITTED");
- Test::create_topic(c, topic_name.c_str(), 1, 3);
+ Test::create_topic_wait_exists(c, topic_name.c_str(), 1, 3, 5000);
run_producer("producer1, -1, 0x0, 5, BeginCommit, DontFlush",
"producer1, -1, 0x10, 5, BeginAbort, DoFlush");
@@ -598,7 +598,7 @@ static void do_test_consumer_txn_test(bool use_java_producer) {
topic_name = Test::mk_topic_name("0098-consumer_txn-0.2", 1);
c = create_consumer(topic_name, "READ_COMMITTED");
- Test::create_topic(c, topic_name.c_str(), 1, 3);
+ Test::create_topic_wait_exists(c, topic_name.c_str(), 1, 3, 5000);
run_producer("producer1, -1, 0x10, 5, BeginAbort, DoFlush",
"producer1, -1, 0x30, 5, BeginCommit, DoFlush");
@@ -642,7 +642,7 @@ static void do_test_consumer_txn_test(bool use_java_producer) {
topic_name = Test::mk_topic_name("0098-consumer_txn-1", 1);
c = create_consumer(topic_name, "READ_COMMITTED");
- Test::create_topic(c, topic_name.c_str(), 1, 3);
+ Test::create_topic_wait_exists(c, topic_name.c_str(), 1, 3, 5000);
TestEventCb::topic = topic_name;
run_producer("producer3, -1, 0x10, 5, None, DoFlush",
@@ -682,7 +682,7 @@ static void do_test_consumer_txn_test(bool use_java_producer) {
topic_name = Test::mk_topic_name("0098-consumer_txn-1.1", 1);
c = create_consumer(topic_name, "READ_COMMITTED");
- Test::create_topic(c, topic_name.c_str(), 1, 3);
+ Test::create_topic_wait_exists(c, topic_name.c_str(), 1, 3, 5000);
run_producer("producer1, -1, 0x30, 5, BeginAbort, DoFlush",
"producer3, -1, 0x40, 5, None, DoFlush",
@@ -714,7 +714,7 @@ static void do_test_consumer_txn_test(bool use_java_producer) {
topic_name = Test::mk_topic_name("0098-consumer_txn-1.2", 1);
c = create_consumer(topic_name, "READ_COMMITTED");
- Test::create_topic(c, topic_name.c_str(), 1, 3);
+ Test::create_topic_wait_exists(c, topic_name.c_str(), 1, 3, 5000);
run_producer("producer1, -1, 0x10, 5, BeginCommit, DoFlush",
"producer1, -1, 0x20, 5, BeginAbort, DoFlush",
@@ -746,7 +746,7 @@ static void do_test_consumer_txn_test(bool use_java_producer) {
topic_name = Test::mk_topic_name("0098-consumer_txn-2", 1);
c = create_consumer(topic_name, "READ_COMMITTED");
- Test::create_topic(c, topic_name.c_str(), 1, 3);
+ Test::create_topic_wait_exists(c, topic_name.c_str(), 1, 3, 5000);
run_producer("producer1, -1, 0x10, 1, BeginAbort, DontFlush",
"producer1, -1, 0x20, 1, BeginCommit, DontFlush",
@@ -799,7 +799,7 @@ static void do_test_consumer_txn_test(bool use_java_producer) {
topic_name = Test::mk_topic_name("0098-consumer_txn-2.1", 1);
c = create_consumer(topic_name, "READ_COMMITTED");
- Test::create_topic(c, topic_name.c_str(), 1, 3);
+ Test::create_topic_wait_exists(c, topic_name.c_str(), 1, 3, 5000);
run_producer("producer1, -1, 0x10, 1, BeginAbort, DoFlush",
"producer1, -1, 0x20, 1, BeginCommit, DoFlush",
@@ -883,7 +883,7 @@ static void do_test_consumer_txn_test(bool use_java_producer) {
topic_name = Test::mk_topic_name("0098-consumer_txn-3", 1);
c = create_consumer(topic_name, "READ_COMMITTED");
- Test::create_topic(c, topic_name.c_str(), 2, 3);
+ Test::create_topic_wait_exists(c, topic_name.c_str(), 2, 3, 5000);
run_producer("producer1, 0, 0x10, 3, BeginOpen, DoFlush",
"producer1, 1, 0x20, 3, ContinueOpen, DoFlush",
@@ -928,7 +928,7 @@ static void do_test_consumer_txn_test(bool use_java_producer) {
topic_name = Test::mk_topic_name("0098-consumer_txn-3.1", 1);
c = create_consumer(topic_name, "READ_COMMITTED");
- Test::create_topic(c, topic_name.c_str(), 2, 3);
+ Test::create_topic_wait_exists(c, topic_name.c_str(), 2, 3, 5000);
run_producer("producer1, 0, 0x55, 1, BeginCommit, DoFlush",
"producer1, 0, 0x10, 3, BeginOpen, DoFlush",
@@ -969,7 +969,7 @@ static void do_test_consumer_txn_test(bool use_java_producer) {
topic_name = Test::mk_topic_name("0098-consumer_txn-4", 1);
c = create_consumer(topic_name, "READ_COMMITTED");
- Test::create_topic(c, topic_name.c_str(), 1, 3);
+ Test::create_topic_wait_exists(c, topic_name.c_str(), 1, 3, 5000);
run_producer("producer3, 0, 0x10, 1, None, DoFlush",
"producer1, 0, 0x20, 3, BeginOpen, DoFlush",
@@ -1004,7 +1004,7 @@ static void do_test_consumer_txn_test(bool use_java_producer) {
topic_name = Test::mk_topic_name("0098-consumer_txn-4.1", 1);
c = create_consumer(topic_name, "READ_COMMITTED");
- Test::create_topic(c, topic_name.c_str(), 1, 3);
+ Test::create_topic_wait_exists(c, topic_name.c_str(), 1, 3, 5000);
run_producer("producer3, 0, 0x10, 1, None, DoFlush",
"producer1, 0, 0x20, 3, BeginOpen, DoFlush",
@@ -1039,7 +1039,7 @@ static void do_test_consumer_txn_test(bool use_java_producer) {
topic_name = Test::mk_topic_name("0098-consumer_txn-4.2", 1);
c = create_consumer(topic_name, "READ_COMMITTED");
- Test::create_topic(c, topic_name.c_str(), 1, 3);
+ Test::create_topic_wait_exists(c, topic_name.c_str(), 1, 3, 5000);
run_producer("producer3, 0, 0x10, 1, None, DoFlush",
"producer1, 0, 0x20, 3, BeginOpen, DoFlush",
@@ -1074,7 +1074,7 @@ static void do_test_consumer_txn_test(bool use_java_producer) {
topic_name = Test::mk_topic_name("0098-consumer_txn-4.3", 1);
c = create_consumer(topic_name, "READ_COMMITTED");
- Test::create_topic(c, topic_name.c_str(), 1, 3);
+ Test::create_topic_wait_exists(c, topic_name.c_str(), 1, 3, 5000);
run_producer("producer3, 0, 0x10, 1, None, DoFlush",
"producer1, 0, 0x20, 3, BeginOpen, DoFlush",
@@ -1111,7 +1111,7 @@ static void do_test_consumer_txn_test(bool use_java_producer) {
test5:
topic_name = Test::mk_topic_name("0098-consumer_txn-5", 1);
c = create_consumer(topic_name, "READ_COMMITTED");
- Test::create_topic(c, topic_name.c_str(), 1, 3);
+ Test::create_topic_wait_exists(c, topic_name.c_str(), 1, 3, 5000);
run_producer("producer1, 0, 0x10, 2, BeginOpen, DontFlush", "sleep,200",
"producer1, 0, 0x20, 2, ContinueAbort, DontFlush",
@@ -1167,7 +1167,7 @@ static void do_test_consumer_txn_test(bool use_java_producer) {
topic_name = Test::mk_topic_name("0098-consumer_txn-0", 1);
c = create_consumer(topic_name, "READ_COMMITTED");
- Test::create_topic(c, topic_name.c_str(), 1, 3);
+ Test::create_topic_wait_exists(c, topic_name.c_str(), 1, 3, 5000);
TestEventCb::topic = topic_name;
run_producer("producer3, 0, 0x10, 1, None, DoFlush",
diff --git a/tests/0099-commit_metadata.c b/tests/0099-commit_metadata.c
index 9acdb07f55..0ca4a339f2 100644
--- a/tests/0099-commit_metadata.c
+++ b/tests/0099-commit_metadata.c
@@ -158,12 +158,13 @@ int main_0099_commit_metadata(int argc, char **argv) {
rd_kafka_topic_partition_list_t *expected_toppar;
const char *topic = test_mk_topic_name("0099-commit_metadata", 0);
char group_id[16];
+ char *metadata;
test_conf_init(NULL, NULL, 20 /*timeout*/);
test_str_id_generate(group_id, sizeof(group_id));
- test_create_topic(NULL, topic, 1, 1);
+ test_create_topic_wait_exists(NULL, topic, 1, 1, 5000);
origin_toppar = rd_kafka_topic_partition_list_new(1);
@@ -171,10 +172,14 @@ int main_0099_commit_metadata(int argc, char **argv) {
expected_toppar = rd_kafka_topic_partition_list_copy(origin_toppar);
- expected_toppar->elems[0].offset = 42;
- expected_toppar->elems[0].metadata = rd_strdup("Hello world!");
- expected_toppar->elems[0].metadata_size =
- strlen(expected_toppar->elems[0].metadata);
+ metadata = rd_strdup("Hello world!");
+ expected_toppar->elems[0].offset = 42;
+ expected_toppar->elems[0].metadata = metadata;
+ expected_toppar->elems[0].metadata_size = strlen(metadata);
+ /* Make sure it's interpreted as bytes.
+ * To fail before the fix it needs to be configured
+ * with HAVE_STRNDUP */
+ metadata[5] = '\0';
get_committed_metadata(group_id, origin_toppar, origin_toppar);
diff --git a/tests/0101-fetch-from-follower.cpp b/tests/0101-fetch-from-follower.cpp
index db438b2a7e..f4968268ec 100644
--- a/tests/0101-fetch-from-follower.cpp
+++ b/tests/0101-fetch-from-follower.cpp
@@ -290,7 +290,7 @@ static void do_fff_test(void) {
int msgcnt = 1000;
const int msgsize = 100;
std::string topic_str = Test::mk_topic_name("0101-fetch-from-follower", 1);
- test_create_topic(NULL, topic_str.c_str(), 1, 3);
+ test_create_topic_wait_exists(NULL, topic_str.c_str(), 1, 3, 5000);
test_produce_msgs_easy_size(topic_str.c_str(), 0, 0, msgcnt, msgsize);
int leader_id;
diff --git a/tests/0102-static_group_rebalance.c b/tests/0102-static_group_rebalance.c
index ad8bac4dbb..8f6c2a90c9 100644
--- a/tests/0102-static_group_rebalance.c
+++ b/tests/0102-static_group_rebalance.c
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2019-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -28,6 +29,7 @@
#include "test.h"
+#include "../src/rdkafka_proto.h"
/**
* @name KafkaConsumer static membership tests
@@ -160,13 +162,16 @@ static void do_test_static_group_rebalance(void) {
c[0].mv = &mv;
c[1].mv = &mv;
- test_create_topic(NULL, topic, 3, 1);
+ test_create_topic_wait_exists(NULL, topic, 3, 1, 5000);
test_produce_msgs_easy(topic, testid, RD_KAFKA_PARTITION_UA, msgcnt);
test_conf_set(conf, "max.poll.interval.ms", "9000");
test_conf_set(conf, "session.timeout.ms", "6000");
test_conf_set(conf, "auto.offset.reset", "earliest");
- test_conf_set(conf, "topic.metadata.refresh.interval.ms", "500");
+ /* Keep this interval higher than cluster metadata propagation
+ * time to make sure no additional rebalances are triggered
+ * when refreshing the full metadata with a regex subscription. */
+ test_conf_set(conf, "topic.metadata.refresh.interval.ms", "2000");
test_conf_set(conf, "metadata.max.age.ms", "5000");
test_conf_set(conf, "enable.partition.eof", "true");
test_conf_set(conf, "group.instance.id", "consumer1");
@@ -179,7 +184,6 @@ static void do_test_static_group_rebalance(void) {
test_conf_set(conf, "group.instance.id", "consumer2");
c[1].rk = test_create_consumer(topic, rebalance_cb,
rd_kafka_conf_dup(conf), NULL);
- rd_kafka_conf_destroy(conf);
test_wait_topic_exists(c[1].rk, topic, 5000);
@@ -222,15 +226,15 @@ static void do_test_static_group_rebalance(void) {
test_msgver_verify("first.verify", &mv, TEST_MSGVER_ALL, 0, msgcnt);
TEST_SAY("== Testing consumer restart ==\n");
- conf = rd_kafka_conf_dup(rd_kafka_conf(c[1].rk));
/* Only c[1] should exhibit rebalance behavior */
c[1].expected_rb_event = RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS;
TIMING_START(&t_close, "consumer restart");
test_consumer_close(c[1].rk);
rd_kafka_destroy(c[1].rk);
-
- c[1].rk = test_create_handle(RD_KAFKA_CONSUMER, conf);
+ c[1].rk = test_create_consumer(topic, rebalance_cb,
+ rd_kafka_conf_dup(conf), NULL);
+ rd_kafka_conf_destroy(conf);
rd_kafka_poll_set_consumer(c[1].rk);
test_consumer_subscribe(c[1].rk, topics);
@@ -255,7 +259,8 @@ static void do_test_static_group_rebalance(void) {
* New topics matching the subscription pattern should cause
* group rebalance
*/
- test_create_topic(c->rk, tsprintf("%snew", topic), 1, 1);
+ test_create_topic_wait_exists(c->rk, tsprintf("%snew", topic), 1, 1,
+ 5000);
/* Await revocation */
rebalance_start = test_clock();
@@ -328,16 +333,25 @@ static void do_test_static_group_rebalance(void) {
/* max.poll.interval.ms should still be enforced by the consumer */
/*
- * Block long enough for consumer 2 to be evicted from the group
- * `max.poll.interval.ms` + `session.timeout.ms`
+ * Stop polling consumer 2 until we reach
+ * `max.poll.interval.ms` and is evicted from the group.
*/
rebalance_start = test_clock();
c[1].expected_rb_event = RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS;
c[0].expected_rb_event = RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS;
c[0].curr_line = __LINE__;
- test_consumer_poll_no_msgs("wait.max.poll", c[0].rk, testid,
- 6000 + 9000);
- c[1].curr_line = __LINE__;
+ /* consumer 2 will time out and all partitions will be assigned to
+ * consumer 1. */
+ static_member_expect_rebalance(&c[0], rebalance_start, &c[0].revoked_at,
+ -1);
+ c[0].expected_rb_event = RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS;
+ static_member_expect_rebalance(&c[0], rebalance_start,
+ &c[0].assigned_at, -1);
+
+ /* consumer 2 restarts polling and re-joins the group */
+ rebalance_start = test_clock();
+ c[0].expected_rb_event = RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS;
+ c[1].curr_line = __LINE__;
test_consumer_poll_expect_err(c[1].rk, testid, 1000,
RD_KAFKA_RESP_ERR__MAX_POLL_EXCEEDED);
@@ -440,9 +454,9 @@ is_fatal_cb(rd_kafka_t *rk, rd_kafka_resp_err_t err, const char *reason) {
}
/**
- * @brief Test that consumer fencing raises a fatal error
+ * @brief Test that consumer fencing raises a fatal error, classic protocol
*/
-static void do_test_fenced_member(void) {
+static void do_test_fenced_member_classic(void) {
rd_kafka_t *c[3]; /* 0: consumer2b, 1: consumer1, 2: consumer2a */
rd_kafka_conf_t *conf;
const char *topic =
@@ -458,9 +472,11 @@ static void do_test_fenced_member(void) {
test_create_topic(NULL, topic, 3, 1);
test_conf_set(conf, "group.instance.id", "consumer1");
+ test_conf_set(conf, "client.id", "consumer1");
c[1] = test_create_consumer(topic, NULL, rd_kafka_conf_dup(conf), NULL);
test_conf_set(conf, "group.instance.id", "consumer2");
+ test_conf_set(conf, "client.id", "consumer2a");
c[2] = test_create_consumer(topic, NULL, rd_kafka_conf_dup(conf), NULL);
test_wait_topic_exists(c[2], topic, 5000);
@@ -473,6 +489,7 @@ static void do_test_fenced_member(void) {
/* Create conflicting consumer */
TEST_SAY("Creating conflicting consumer2 instance\n");
test_conf_set(conf, "group.instance.id", "consumer2");
+ test_conf_set(conf, "client.id", "consumer2b");
c[0] = test_create_consumer(topic, NULL, rd_kafka_conf_dup(conf), NULL);
rd_kafka_conf_destroy(conf);
@@ -523,13 +540,297 @@ static void do_test_fenced_member(void) {
SUB_TEST_PASS();
}
+/**
+ * @brief Test that consumer fencing raises a fatal error,
+ * consumer protocol (KIP-848).
+ * The difference with the behavior of the classic one is that
+ * the member that is fenced is the one that is joining the group
+ * and not the one that was already in the group.
+ * Also the error is ERR_UNRELEASED_INSTANCE_ID instead of
+ * ERR_FENCED_INSTANCE_ID.
+ */
+static void do_test_fenced_member_consumer(void) {
+ rd_kafka_t *c[3]; /* 0: consumer2b, 1: consumer1, 2: consumer2a */
+ rd_kafka_conf_t *conf;
+ const char *topic =
+ test_mk_topic_name("0102_static_group_rebalance", 1);
+ rd_kafka_message_t *rkm;
+ char errstr[512];
+ rd_kafka_resp_err_t err;
+
+ SUB_TEST();
+ test_conf_init(&conf, NULL, 30);
+
+ test_create_topic(NULL, topic, 3, 1);
+
+ test_conf_set(conf, "group.instance.id", "consumer1");
+ test_conf_set(conf, "client.id", "consumer1");
+ c[1] = test_create_consumer(topic, NULL, rd_kafka_conf_dup(conf), NULL);
+
+ test_conf_set(conf, "group.instance.id", "consumer2");
+ test_conf_set(conf, "client.id", "consumer2a");
+ c[2] = test_create_consumer(topic, NULL, rd_kafka_conf_dup(conf), NULL);
+
+ test_wait_topic_exists(c[2], topic, 5000);
+
+ test_consumer_subscribe(c[1], topic);
+ test_consumer_subscribe(c[2], topic);
+
+ await_assignment_multi("Awaiting initial assignments", &c[1], 2);
+
+ /* Create conflicting consumer */
+ TEST_SAY("Creating conflicting consumer 2 instance\n");
+ test_conf_set(conf, "group.instance.id", "consumer2");
+ test_conf_set(conf, "client.id", "consumer2b");
+ c[0] = test_create_consumer(topic, NULL, rd_kafka_conf_dup(conf), NULL);
+ rd_kafka_conf_destroy(conf);
+
+ test_curr->is_fatal_cb = is_fatal_cb;
+ valid_fatal_rk = c[0]; /* consumer2b is the consumer that should fail */
+
+ test_consumer_subscribe(c[0], topic);
+
+ /* consumer1 should not be affected (other than a rebalance which
+ * we ignore here)... */
+ test_consumer_poll_no_msgs("consumer1", c[1], 0, 5000);
+
+ /* consumer2b should be fenced off on joining */
+ rkm = rd_kafka_consumer_poll(c[0], 5000);
+ TEST_ASSERT(rkm != NULL, "Expected error, not timeout");
+ TEST_ASSERT(rkm->err == RD_KAFKA_RESP_ERR__FATAL,
+ "Expected ERR__FATAL, not %s: %s",
+ rd_kafka_err2str(rkm->err), rd_kafka_message_errstr(rkm));
+ TEST_SAY("Fenced consumer returned expected: %s: %s\n",
+ rd_kafka_err2name(rkm->err), rd_kafka_message_errstr(rkm));
+ rd_kafka_message_destroy(rkm);
+
+
+ /* Read the actual error */
+ err = rd_kafka_fatal_error(c[0], errstr, sizeof(errstr));
+ TEST_SAY("%s fatal error: %s: %s\n", rd_kafka_name(c[0]),
+ rd_kafka_err2name(err), errstr);
+ TEST_ASSERT(
+ err == RD_KAFKA_RESP_ERR_UNRELEASED_INSTANCE_ID,
+ "Expected ERR_UNRELEASED_INSTANCE_ID as fatal error, not %s",
+ rd_kafka_err2name(err));
+
+ TEST_SAY("close\n");
+ /* Close consumer2b, should also return a fatal error */
+ err = rd_kafka_consumer_close(c[0]);
+ TEST_ASSERT(err == RD_KAFKA_RESP_ERR__FATAL,
+ "Expected close on %s to return ERR__FATAL, not %s",
+ rd_kafka_name(c[0]), rd_kafka_err2name(err));
+
+ rd_kafka_destroy(c[0]);
+
+ /* consumer1 and consumer2a should be fine and get their
+ * assignments */
+ await_assignment_multi("Awaiting post-fencing assignment", &c[1], 2);
+
+ rd_kafka_destroy(c[1]);
+ rd_kafka_destroy(c[2]);
+
+ SUB_TEST_PASS();
+}
+/**
+ * @brief Create a new consumer with given \p boostraps
+ * \p group_id and \p group_instance_id .
+ */
+static rd_kafka_t *create_consumer(const char *bootstraps,
+ const char *group_id,
+ const char *group_instance_id) {
+ rd_kafka_conf_t *conf;
+ test_conf_init(&conf, NULL, 0);
+ test_conf_set(conf, "bootstrap.servers", bootstraps);
+ test_conf_set(conf, "group.instance.id", group_instance_id);
+ test_conf_set(conf, "auto.offset.reset", "earliest");
+ test_conf_set(conf, "enable.partition.eof", "true");
+ return test_create_consumer(group_id, NULL, conf, NULL);
+}
+
+/**
+ * @brief Get generation id of consumer \p consumer .
+ */
+static int32_t consumer_generation_id(rd_kafka_t *consumer) {
+ rd_kafka_consumer_group_metadata_t *group_metadata;
+ int32_t generation_id;
+
+ group_metadata = rd_kafka_consumer_group_metadata(consumer);
+ generation_id =
+ rd_kafka_consumer_group_metadata_generation_id(group_metadata);
+ rd_kafka_consumer_group_metadata_destroy(group_metadata);
+ return generation_id;
+}
+
+/**
+ * @brief Check if the API key in \p request is the same as that
+ * pointed by \p opaque .
+ */
+static rd_bool_t is_api_key(rd_kafka_mock_request_t *request, void *opaque) {
+ int32_t api_key = *(int32_t *)opaque;
+ return rd_kafka_mock_request_api_key(request) == api_key;
+}
+
+/**
+ * @enum do_test_static_membership_mock_variation_t
+ * @brief Variations of the static membership mock test.
+ */
+typedef enum do_test_static_membership_mock_variation_t {
+ /** Consumer 1 leaves with unsubscribe and rejoins the group */
+ DO_TEST_STATIC_MEMBERSHIP_MOCK_VARIATION_SAME_INSTANCE = 0,
+ /** Consumer 1 leaves with unsubscribe and a new consumer with same
+ * group.instance.id joins the group */
+ DO_TEST_STATIC_MEMBERSHIP_MOCK_VARIATION_NEW_INSTANCE = 1,
+ DO_TEST_STATIC_MEMBERSHIP_MOCK_VARIATION__CNT
+} do_test_static_membership_mock_variation_t;
+
+/**
+ * @brief Static group membership tests with the mock cluster.
+ * Checks that consumer returns the same assignment
+ * and generation id after re-joining.
+ *
+ * @param variation Test variation to run.
+ *
+ * @sa `do_test_static_membership_mock_variation_t`
+ */
+static void do_test_static_membership_mock(
+ do_test_static_membership_mock_variation_t variation) {
+ const char *bootstraps;
+ rd_kafka_mock_cluster_t *mcluster;
+ int32_t api_key = RD_KAFKAP_ConsumerGroupHeartbeat;
+ const char *topic = test_mk_topic_name(__FUNCTION__, 0);
+ rd_kafka_t *consumer1, *consumer2, *consumer_1_to_destroy = NULL;
+ int32_t prev_generation_id1, next_generation_id1, prev_generation_id2,
+ next_generation_id2;
+ rd_kafka_topic_partition_list_t *prev_assignment1, *prev_assignment2,
+ *next_assignment1, *next_assignment2;
+
+ SUB_TEST_QUICK(
+ "%s",
+ variation == DO_TEST_STATIC_MEMBERSHIP_MOCK_VARIATION_SAME_INSTANCE
+ ? "with same instance"
+ : "with new instance");
+
+ mcluster = test_mock_cluster_new(3, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 2, 3);
+
+ TEST_SAY("Creating consumers\n");
+ consumer1 = create_consumer(bootstraps, topic, "c1");
+ consumer2 = create_consumer(bootstraps, topic, "c2");
+
+ TEST_SAY("Subscribing consumers to topic \"%s\"\n", topic);
+ test_consumer_subscribe(consumer1, topic);
+ test_consumer_subscribe(consumer2, topic);
+
+ TEST_SAY("Waiting one EOF of consumer 1\n");
+ test_consumer_poll_exact("first consumer", consumer1, 0, 1, 0, 0,
+ rd_true, NULL);
+ TEST_SAY("Waiting one EOF of consumer 2\n");
+ test_consumer_poll_exact("second consumer", consumer2, 0, 1, 0, 0,
+ rd_true, NULL);
+
+ prev_generation_id1 = consumer_generation_id(consumer1);
+ prev_generation_id2 = consumer_generation_id(consumer2);
+ TEST_CALL_ERR__(rd_kafka_assignment(consumer1, &prev_assignment1));
+ TEST_CALL_ERR__(rd_kafka_assignment(consumer2, &prev_assignment2));
+ TEST_ASSERT(prev_assignment1 != NULL,
+ "Expected assignment for consumer 1 before the change");
+ TEST_ASSERT(prev_assignment2 != NULL,
+ "Expected assignment for consumer 2 before the change");
+
+ TEST_SAY("Unsubscribing consumer 1\n");
+ rd_kafka_mock_start_request_tracking(mcluster);
+ TEST_CALL_ERR__(rd_kafka_unsubscribe(consumer1));
+ test_mock_wait_matching_requests(mcluster, 1, 1000, is_api_key,
+ &api_key);
+ rd_kafka_mock_stop_request_tracking(mcluster);
+
+ if (variation ==
+ DO_TEST_STATIC_MEMBERSHIP_MOCK_VARIATION_NEW_INSTANCE) {
+ /* Don't destroy it immediately because the
+ * topic partition lists still hold a reference. */
+ consumer_1_to_destroy = consumer1;
+
+ TEST_SAY("Re-creating consumer 1\n");
+ /* Re-create the consumer with same group and instance id. */
+ consumer1 = create_consumer(bootstraps, topic, "c1");
+ }
+
+ TEST_SAY("Subscribing consumer 1 again\n");
+ test_consumer_subscribe(consumer1, topic);
+ test_consumer_wait_assignment(consumer1, rd_false);
+
+ next_generation_id1 = consumer_generation_id(consumer1);
+ next_generation_id2 = consumer_generation_id(consumer2);
+
+ TEST_ASSERT(next_generation_id1 == prev_generation_id1,
+ "Expected same generation id for consumer 1, "
+ "got %d != %d",
+ prev_generation_id1, next_generation_id1);
+ TEST_ASSERT(next_generation_id2 == prev_generation_id2,
+ "Expected same generation id for consumer 2, "
+ "got %d != %d",
+ prev_generation_id2, next_generation_id2);
+
+ TEST_CALL_ERR__(rd_kafka_assignment(consumer1, &next_assignment1));
+ TEST_CALL_ERR__(rd_kafka_assignment(consumer2, &next_assignment2));
+ TEST_ASSERT(next_assignment1 != NULL,
+ "Expected assignment for consumer 1 after the change");
+ TEST_ASSERT(next_assignment2 != NULL,
+ "Expected assignment for consumer 2 after the change");
+ TEST_ASSERT(!test_partition_list_and_offsets_cmp(prev_assignment1,
+ next_assignment1),
+ "Expected same assignment for consumer 1 after the change");
+ TEST_ASSERT(!test_partition_list_and_offsets_cmp(prev_assignment2,
+ next_assignment2),
+ "Expected same assignment for consumer 2 after the change");
+
+ rd_kafka_topic_partition_list_destroy(prev_assignment1);
+ rd_kafka_topic_partition_list_destroy(prev_assignment2);
+ rd_kafka_topic_partition_list_destroy(next_assignment1);
+ rd_kafka_topic_partition_list_destroy(next_assignment2);
+
+ RD_IF_FREE(consumer_1_to_destroy, rd_kafka_destroy);
+ rd_kafka_destroy(consumer1);
+ rd_kafka_destroy(consumer2);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
int main_0102_static_group_rebalance(int argc, char **argv) {
+ /* TODO: check again when regexes
+ * will be supported by KIP-848 */
+ if (test_consumer_group_protocol_classic()) {
+ do_test_static_group_rebalance();
+ }
+
+ if (test_consumer_group_protocol_classic()) {
+ do_test_fenced_member_classic();
+ } else {
+ do_test_fenced_member_consumer();
+ }
+
+ return 0;
+}
- do_test_static_group_rebalance();
+int main_0102_static_group_rebalance_mock(int argc, char **argv) {
+ TEST_SKIP_MOCK_CLUSTER(0);
+ int variation;
- do_test_fenced_member();
+ if (test_consumer_group_protocol_classic()) {
+ TEST_SKIP(
+ "Static membership isn't implemented "
+ "in mock cluster for classic protocol\n");
+ return 0;
+ }
+
+ for (variation = DO_TEST_STATIC_MEMBERSHIP_MOCK_VARIATION_SAME_INSTANCE;
+ variation < DO_TEST_STATIC_MEMBERSHIP_MOCK_VARIATION__CNT;
+ variation++) {
+ do_test_static_membership_mock(variation);
+ }
return 0;
}
diff --git a/tests/0103-transactions.c b/tests/0103-transactions.c
index c2217cd255..0bc1664d83 100644
--- a/tests/0103-transactions.c
+++ b/tests/0103-transactions.c
@@ -143,7 +143,7 @@ static void do_test_basic_producer_txn(rd_bool_t enable_compression) {
// FIXME: add testing were the txn id is reused (and thus fails)
/* Create topic */
- test_create_topic(p, topic, partition_cnt, 3);
+ test_create_topic_wait_exists(p, topic, partition_cnt, 3, 5000);
/* Create consumer */
c_conf = conf;
@@ -348,8 +348,8 @@ void do_test_consumer_producer_txn(void) {
p1 = test_create_handle(RD_KAFKA_PRODUCER, tmpconf);
/* Create input and output topics */
- test_create_topic(p1, input_topic, 4, 3);
- test_create_topic(p1, output_topic, 4, 3);
+ test_create_topic_wait_exists(p1, input_topic, 4, 3, 5000);
+ test_create_topic_wait_exists(p1, output_topic, 4, 3, 5000);
/* Seed input topic with messages */
TEST_CALL_ERROR__(rd_kafka_init_transactions(p1, 30 * 1000));
@@ -879,7 +879,7 @@ static void do_test_fatal_idempo_error_without_kip360(void) {
p = test_create_handle(RD_KAFKA_PRODUCER, conf);
- test_create_topic(p, topic, 1, 3);
+ test_create_topic_wait_exists(p, topic, 1, 3, 5000);
TEST_CALL_ERROR__(rd_kafka_init_transactions(p, 30 * 1000));
@@ -1029,7 +1029,7 @@ static void do_test_empty_txn(rd_bool_t send_offsets, rd_bool_t do_commit) {
rd_kafka_conf_set_dr_msg_cb(conf, test_dr_msg_cb);
p = test_create_handle(RD_KAFKA_PRODUCER, conf);
- test_create_topic(p, topic, 1, 3);
+ test_create_topic_wait_exists(p, topic, 1, 3, 5000);
/* Produce some non-txnn messages for the consumer to read and commit */
test_produce_msgs_easy(topic, testid, 0, msgcnt);
@@ -1130,7 +1130,7 @@ static void do_test_txn_abort_control_message_leader_epoch(void) {
rd_kafka_conf_set_dr_msg_cb(p_conf, test_dr_msg_cb);
p = test_create_handle(RD_KAFKA_PRODUCER, p_conf);
- test_create_topic(p, topic, 1, 3);
+ test_create_topic_wait_exists(p, topic, 1, 3, 5000);
TEST_CALL_ERROR__(rd_kafka_init_transactions(p, 5000));
@@ -1225,7 +1225,7 @@ static void do_test_wmark_isolation_level(void) {
rd_kafka_conf_set_dr_msg_cb(conf, test_dr_msg_cb);
p = test_create_handle(RD_KAFKA_PRODUCER, rd_kafka_conf_dup(conf));
- test_create_topic(p, topic, 1, 3);
+ test_create_topic_wait_exists(p, topic, 1, 3, 5000);
/* Produce some non-txn messages to avoid 0 as the committed hwmark */
test_produce_msgs_easy(topic, testid, 0, 100);
diff --git a/tests/0104-fetch_from_follower_mock.c b/tests/0104-fetch_from_follower_mock.c
index 972ff9c518..b77be219ac 100644
--- a/tests/0104-fetch_from_follower_mock.c
+++ b/tests/0104-fetch_from_follower_mock.c
@@ -29,7 +29,6 @@
#include "test.h"
-
/**
* @name Fetch from follower tests using the mock broker.
*/
@@ -111,6 +110,14 @@ static void do_test_offset_reset(const char *auto_offset_reset) {
else
test_consumer_poll(auto_offset_reset, c, 0, 1, 0, msgcnt, NULL);
+ /* send another batch of messages to ensure the consumer isn't stuck
+ * sending ListOffsets to the replica and receiving
+ * NOT_LEADER_OR_FOLLOWER errors.
+ * See PR #4616 */
+ test_produce_msgs_easy_v(topic, 0, 0, 0, msgcnt, 1000,
+ "bootstrap.servers", bootstraps, NULL);
+ test_consumer_poll("ASSIGN", c, 0, 1, 0, msgcnt, NULL);
+
test_consumer_close(c);
rd_kafka_destroy(c);
@@ -577,6 +584,7 @@ static void do_test_seek_to_offset_with_previous_epoch(void) {
* from offset 'msgcnt'. */
rd_kafka_topic_partition_set_leader_epoch(rktpar, 0);
rd_kafka_seek_partitions(c, rktpars, -1);
+ rd_kafka_topic_partition_list_destroy(rktpars);
test_consumer_poll("third", c, 0, 0, msgcnt, msgcnt, NULL);
diff --git a/tests/0105-transactions_mock.c b/tests/0105-transactions_mock.c
index 04958f7d2a..84545bc60e 100644
--- a/tests/0105-transactions_mock.c
+++ b/tests/0105-transactions_mock.c
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2019-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -44,13 +45,14 @@
static int allowed_error;
+static int allowed_error_2;
/**
* @brief Decide what error_cb's will cause the test to fail.
*/
static int
error_is_fatal_cb(rd_kafka_t *rk, rd_kafka_resp_err_t err, const char *reason) {
- if (err == allowed_error ||
+ if (err == allowed_error || err == allowed_error_2 ||
/* If transport errors are allowed then it is likely
* that we'll also see ALL_BROKERS_DOWN. */
(allowed_error == RD_KAFKA_RESP_ERR__TRANSPORT &&
@@ -1926,8 +1928,8 @@ static void do_test_txn_flush_timeout(void) {
/* The retry should succeed */
test_curr->ignore_dr_err = rd_false;
test_curr->exp_dr_err = is_retry
- ? RD_KAFKA_RESP_ERR_NO_ERROR
- : RD_KAFKA_RESP_ERR__MSG_TIMED_OUT;
+ ? RD_KAFKA_RESP_ERR_NO_ERROR
+ : RD_KAFKA_RESP_ERR__MSG_TIMED_OUT;
rd_kafka_mock_partition_set_leader(mcluster, topic, 0, 1);
rd_kafka_mock_partition_set_leader(mcluster, topic, 1, 1);
@@ -2062,7 +2064,9 @@ static void do_test_txn_coord_req_destroy(void) {
rd_kafka_mock_push_request_errors(
mcluster, RD_KAFKAP_AddOffsetsToTxn,
- 1, /* first request + number of internal retries */
+ 3, /* first request + number of internal retries */
+ RD_KAFKA_RESP_ERR_CONCURRENT_TRANSACTIONS,
+ RD_KAFKA_RESP_ERR_CONCURRENT_TRANSACTIONS,
RD_KAFKA_RESP_ERR_CONCURRENT_TRANSACTIONS);
err = rd_kafka_producev(rk, RD_KAFKA_V_TOPIC("mytopic"),
@@ -2125,6 +2129,7 @@ static void do_test_txn_coord_req_destroy(void) {
TEST_CALL_ERROR__(rd_kafka_abort_transaction(rk, 5000));
}
+ TEST_SAY("Got %d errors\n", errcnt);
TEST_ASSERT(errcnt > 0,
"Expected at least one send_offets_to_transaction() "
"failure");
@@ -2222,12 +2227,12 @@ static void do_test_txn_coord_req_multi_find(void) {
on_response_received_cb = multi_find_on_response_received_cb;
rk = create_txn_producer(&mcluster, txnid, 3,
- /* Need connections to all brokers so we
- * can trigger coord_req_fsm events
- * by toggling connections. */
- "enable.sparse.connections", "false",
- /* Set up on_response_received interceptor */
- "on_response_received", "", NULL);
+ /* Need connections to all brokers so we
+ * can trigger coord_req_fsm events
+ * by toggling connections. */
+ "enable.sparse.connections", "false",
+ /* Set up on_response_received interceptor */
+ "on_response_received", "", NULL);
/* Let broker 1 be both txn and group coordinator
* so that the group coordinator connection is up when it is time
@@ -3042,6 +3047,7 @@ static void do_test_txn_coordinator_null_not_fatal(void) {
/* Broker down is not a test-failing error */
allowed_error = RD_KAFKA_RESP_ERR__TRANSPORT;
+ allowed_error_2 = RD_KAFKA_RESP_ERR__TIMED_OUT;
test_curr->is_fatal_cb = error_is_fatal_cb;
test_curr->exp_dr_err = RD_KAFKA_RESP_ERR__MSG_TIMED_OUT;
@@ -3113,6 +3119,7 @@ static void do_test_txn_coordinator_null_not_fatal(void) {
rd_kafka_destroy(rk);
allowed_error = RD_KAFKA_RESP_ERR_NO_ERROR;
+ allowed_error_2 = RD_KAFKA_RESP_ERR_NO_ERROR;
test_curr->exp_dr_err = RD_KAFKA_RESP_ERR_NO_ERROR;
test_curr->is_fatal_cb = NULL;
@@ -3795,7 +3802,7 @@ do_test_txn_offset_commit_doesnt_retry_too_quickly(rd_bool_t times_out) {
* 2000ms for this call to succeed. */
timeout = times_out ? 500 : 4000;
error = rd_kafka_send_offsets_to_transaction(rk, offsets, cgmetadata,
- timeout);
+ timeout);
rd_kafka_consumer_group_metadata_destroy(cgmetadata);
rd_kafka_topic_partition_list_destroy(offsets);
diff --git a/tests/0106-cgrp_sess_timeout.c b/tests/0106-cgrp_sess_timeout.c
index 6d9f43f160..997f33c3bd 100644
--- a/tests/0106-cgrp_sess_timeout.c
+++ b/tests/0106-cgrp_sess_timeout.c
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2020-2022, Magnus Edenhill
+ * 2025, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -55,7 +56,7 @@ static void rebalance_cb(rd_kafka_t *rk,
rd_kafka_err2name(rebalance_exp_event), rd_kafka_err2name(err));
if (err == RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS) {
- test_consumer_assign("assign", rk, parts);
+ test_consumer_assign_by_rebalance_protocol("assign", rk, parts);
} else {
rd_kafka_resp_err_t commit_err;
@@ -92,8 +93,8 @@ static void rebalance_cb(rd_kafka_t *rk,
rd_kafka_err2name(commit_exp_err),
rd_kafka_err2name(commit_err));
}
-
- test_consumer_unassign("unassign", rk);
+ test_consumer_unassign_by_rebalance_protocol("unassign", rk,
+ parts);
}
/* Make sure only one rebalance callback is served per poll()
@@ -169,21 +170,26 @@ static void do_test_session_timeout(const char *use_commit_type) {
test_conf_set(conf, "auto.offset.reset", "earliest");
test_conf_set(conf, "enable.auto.commit",
!strcmp(commit_type, "auto") ? "true" : "false");
+ rd_kafka_mock_set_group_consumer_session_timeout_ms(mcluster, 1000);
+ rd_kafka_mock_set_group_consumer_heartbeat_interval_ms(mcluster, 100);
c = test_create_consumer(groupid, rebalance_cb, conf, NULL);
test_consumer_subscribe(c, topic);
- /* Let Heartbeats fail after a couple of successful ones */
- rd_kafka_mock_push_request_errors(
- mcluster, RD_KAFKAP_Heartbeat, 9, RD_KAFKA_RESP_ERR_NO_ERROR,
- RD_KAFKA_RESP_ERR_NO_ERROR, RD_KAFKA_RESP_ERR_NOT_COORDINATOR,
- RD_KAFKA_RESP_ERR_NOT_COORDINATOR,
- RD_KAFKA_RESP_ERR_NOT_COORDINATOR,
- RD_KAFKA_RESP_ERR_NOT_COORDINATOR,
- RD_KAFKA_RESP_ERR_NOT_COORDINATOR,
- RD_KAFKA_RESP_ERR_NOT_COORDINATOR,
- RD_KAFKA_RESP_ERR_NOT_COORDINATOR);
+ if (test_consumer_group_protocol_classic()) {
+ /* Let Heartbeats fail after a couple of successful ones */
+ rd_kafka_mock_push_request_errors(
+ mcluster, RD_KAFKAP_Heartbeat, 9,
+ RD_KAFKA_RESP_ERR_NO_ERROR, RD_KAFKA_RESP_ERR_NO_ERROR,
+ RD_KAFKA_RESP_ERR_NOT_COORDINATOR,
+ RD_KAFKA_RESP_ERR_NOT_COORDINATOR,
+ RD_KAFKA_RESP_ERR_NOT_COORDINATOR,
+ RD_KAFKA_RESP_ERR_NOT_COORDINATOR,
+ RD_KAFKA_RESP_ERR_NOT_COORDINATOR,
+ RD_KAFKA_RESP_ERR_NOT_COORDINATOR,
+ RD_KAFKA_RESP_ERR_NOT_COORDINATOR);
+ }
expect_rebalance("initial assignment", c,
RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS, 5 + 2);
@@ -191,6 +197,12 @@ static void do_test_session_timeout(const char *use_commit_type) {
/* Consume a couple of messages so that we have something to commit */
test_consumer_poll("consume", c, 0, -1, 0, 10, NULL);
+ if (!test_consumer_group_protocol_classic()) {
+ /* Increase HB interval so member is fenced from the group */
+ rd_kafka_mock_set_group_consumer_heartbeat_interval_ms(mcluster,
+ 2000);
+ }
+
/* The commit in the rebalance callback should fail when the
* member has timed out from the group. */
commit_exp_err = RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID;
@@ -291,7 +303,16 @@ int main_0106_cgrp_sess_timeout(int argc, char **argv) {
do_test_session_timeout("async");
do_test_session_timeout("auto");
- do_test_commit_on_lost();
+ /* With KIP-848 session timeout is remote only.
+ * Partitions will stay assigned and fetchable
+ * when there's no communication with the
+ * coordinator, but the messages won't be
+ * committed.
+ * TODO: see if session timeout can be received by
+ * the broker and be enforced locally too. */
+ if (test_consumer_group_protocol_classic()) {
+ do_test_commit_on_lost();
+ }
return 0;
}
diff --git a/tests/0107-topic_recreate.c b/tests/0107-topic_recreate.c
index 474ed2f27a..68b9784796 100644
--- a/tests/0107-topic_recreate.c
+++ b/tests/0107-topic_recreate.c
@@ -189,7 +189,7 @@ static void do_test_create_delete_create(int part_cnt_1, int part_cnt_2) {
consumer = test_create_consumer(topic, NULL, NULL, NULL);
/* Create topic */
- test_create_topic(consumer, topic, part_cnt_1, 3);
+ test_create_topic_wait_exists(consumer, topic, part_cnt_1, 3, 5000);
/* Start consumer */
test_consumer_subscribe(consumer, topic);
@@ -216,7 +216,7 @@ static void do_test_create_delete_create(int part_cnt_1, int part_cnt_2) {
rd_sleep(5);
/* Re-create topic */
- test_create_topic(consumer, topic, part_cnt_2, 3);
+ test_create_topic_wait_exists(consumer, topic, part_cnt_2, 3, 5000);
mtx_lock(&value_mtx);
value = "after";
diff --git a/tests/0109-auto_create_topics.cpp b/tests/0109-auto_create_topics.cpp
index b64050fee4..c5582aa072 100644
--- a/tests/0109-auto_create_topics.cpp
+++ b/tests/0109-auto_create_topics.cpp
@@ -39,21 +39,39 @@
* The same test is run with and without allow.auto.create.topics
* and with and without wildcard subscribes.
*
+ * With the KIP 848 consumer group protocol, topics aren't automatically
+ * created when subscribing and the topic authorization error
+ * applies to the whole subscription, not just the unauthorized topics.
*/
static void do_test_consumer(bool allow_auto_create_topics,
- bool with_wildcards) {
+ bool with_wildcards,
+ bool test_unauthorized_topic) {
Test::Say(tostr() << _C_MAG << "[ Test allow.auto.create.topics="
<< (allow_auto_create_topics ? "true" : "false")
<< " with_wildcards=" << (with_wildcards ? "true" : "false")
- << " ]\n");
+ << " test_unauthorized_topic="
+ << (test_unauthorized_topic ? "true" : "false") << " ]\n");
bool has_acl_cli = test_broker_version >= TEST_BRKVER(2, 1, 0, 0) &&
!test_needs_auth(); /* We can't bother passing Java
* security config to kafka-acls.sh */
+ if (test_unauthorized_topic && !has_acl_cli) {
+ Test::Say(
+ "Skipping unauthorized topic test since kafka-acls.sh is not "
+ "available\n");
+ return;
+ }
+ if (!test_consumer_group_protocol_classic() && allow_auto_create_topics) {
+ Test::Say(
+ "Skipping test as it would be duplicate "
+ "with KIP 848 consumer protocol\n");
+ return;
+ }
bool supports_allow = test_broker_version >= TEST_BRKVER(0, 11, 0, 0);
+ const int cgrp_consumer_expected_consecutive_error_cnt = 3;
std::string topic_exists = Test::mk_topic_name("0109-exists", 1);
std::string topic_notexists = Test::mk_topic_name("0109-notexists", 1);
@@ -83,7 +101,7 @@ static void do_test_consumer(bool allow_auto_create_topics,
/* Create topics */
Test::create_topic(c, topic_exists.c_str(), 1, 1);
- if (has_acl_cli) {
+ if (test_unauthorized_topic) {
Test::create_topic(c, topic_unauth.c_str(), 1, 1);
/* Add denying ACL for unauth topic */
@@ -107,34 +125,50 @@ static void do_test_consumer(bool allow_auto_create_topics,
std::map exp_errors;
topics.push_back(topic_notexists);
- if (has_acl_cli)
+
+ if (test_unauthorized_topic)
topics.push_back(topic_unauth);
if (with_wildcards) {
topics.push_back("^" + topic_exists);
topics.push_back("^" + topic_notexists);
- /* If the subscription contains at least one wildcard/regex
- * then no auto topic creation will take place (since the consumer
- * requests all topics in metadata, and not specific ones, thus
- * not triggering topic auto creation).
- * We need to handle the expected error cases accordingly. */
- exp_errors["^" + topic_notexists] = RdKafka::ERR_UNKNOWN_TOPIC_OR_PART;
- exp_errors[topic_notexists] = RdKafka::ERR_UNKNOWN_TOPIC_OR_PART;
-
- if (has_acl_cli) {
- /* Unauthorized topics are not included in list-all-topics Metadata,
- * which we use for wildcards, so in this case the error code for
- * unauthorixed topics show up as unknown topic. */
- exp_errors[topic_unauth] = RdKafka::ERR_UNKNOWN_TOPIC_OR_PART;
- }
} else {
topics.push_back(topic_exists);
+ }
- if (has_acl_cli)
+ /* `classic` protocol case: if the subscription contains at least one
+ * wildcard/regex then no auto topic creation will take place (since the
+ * consumer requests all topics in metadata, and not specific ones, thus not
+ * triggering topic auto creation). We need to handle the expected error cases
+ * accordingly.
+ *
+ * `consumer` protocol case: there's no automatic topic creation. */
+ if (test_consumer_group_protocol_classic()) {
+ if (with_wildcards) {
+ exp_errors["^" + topic_notexists] = RdKafka::ERR_UNKNOWN_TOPIC_OR_PART;
+ exp_errors[topic_notexists] = RdKafka::ERR_UNKNOWN_TOPIC_OR_PART;
+ if (test_unauthorized_topic) {
+ /* Unauthorized topics are not included in list-all-topics Metadata,
+ * which we use for wildcards, so in this case the error code for
+ * unauthorixed topics show up as unknown topic. */
+ exp_errors[topic_unauth] = RdKafka::ERR_UNKNOWN_TOPIC_OR_PART;
+ }
+ } else if (test_unauthorized_topic) {
exp_errors[topic_unauth] = RdKafka::ERR_TOPIC_AUTHORIZATION_FAILED;
+ }
+ } else if (test_unauthorized_topic) {
+ /* Authorization errors happen if even a single topic
+ * is unauthorized and an error is returned for the whole subscription
+ * without reference to the topic. */
+ exp_errors[""] = RdKafka::ERR_TOPIC_AUTHORIZATION_FAILED;
}
- if (supports_allow && !allow_auto_create_topics)
+ /* `classic` protocol case: expect an error only if the broker supports the
+ * property and the test disallowed it.
+ *
+ * `consumer` protocol case: there's no automatic topic creation. */
+ if (supports_allow && !allow_auto_create_topics &&
+ test_consumer_group_protocol_classic())
exp_errors[topic_notexists] = RdKafka::ERR_UNKNOWN_TOPIC_OR_PART;
RdKafka::ErrorCode err;
@@ -143,7 +177,8 @@ static void do_test_consumer(bool allow_auto_create_topics,
/* Start consuming until EOF is reached, which indicates that we have an
* assignment and any errors should have been reported. */
- bool run = true;
+ bool run = true;
+ int consecutive_error_cnt = 0;
while (run) {
RdKafka::Message *msg = c->consume(tmout_multip(1000));
switch (msg->err()) {
@@ -155,6 +190,17 @@ static void do_test_consumer(bool allow_auto_create_topics,
run = false;
break;
+ case RdKafka::ERR_TOPIC_AUTHORIZATION_FAILED:
+ if (test_consumer_group_protocol_classic()) {
+ run = true;
+ } else {
+ /* `consumer` rebalance protocol:
+ * wait for `unauthorized_error_cnt` consecutive errors. */
+ run = (++consecutive_error_cnt) <
+ cgrp_consumer_expected_consecutive_error_cnt;
+ }
+ /* FALLTHRU */
+
default:
Test::Say("Consume error on " + msg->topic_name() + ": " + msg->errstr() +
"\n");
@@ -186,6 +232,13 @@ static void do_test_consumer(bool allow_auto_create_topics,
RdKafka::err2str(msg->err()));
} else {
exp_errors.erase(msg->topic_name());
+ if (!test_consumer_group_protocol_classic() &&
+ test_unauthorized_topic &&
+ consecutive_error_cnt <
+ cgrp_consumer_expected_consecutive_error_cnt) {
+ /* Expect same error on next HB */
+ exp_errors[""] = RdKafka::ERR_TOPIC_AUTHORIZATION_FAILED;
+ }
}
break;
@@ -207,11 +260,18 @@ static void do_test_consumer(bool allow_auto_create_topics,
extern "C" {
int main_0109_auto_create_topics(int argc, char **argv) {
/* Parameters:
- * allow auto create, with wildcards */
- do_test_consumer(true, true);
- do_test_consumer(true, false);
- do_test_consumer(false, true);
- do_test_consumer(false, false);
+ * allow auto create, with wildcards, test unauthorized topic */
+ do_test_consumer(true, false, false);
+ do_test_consumer(false, false, false);
+
+ do_test_consumer(true, true, false);
+ do_test_consumer(false, true, false);
+
+ do_test_consumer(true, false, true);
+ do_test_consumer(false, false, true);
+
+ do_test_consumer(true, true, true);
+ do_test_consumer(false, true, true);
return 0;
}
diff --git a/tests/0110-batch_size.cpp b/tests/0110-batch_size.cpp
index 5b216c2804..2d89e7162a 100644
--- a/tests/0110-batch_size.cpp
+++ b/tests/0110-batch_size.cpp
@@ -154,8 +154,7 @@ static void do_test_batch_size() {
while (event_cb.avg_batchsize == 0)
p->poll(1000);
- Test::Say(tostr() << "Batchsize: "
- << "configured " << batchsize << ", min "
+ Test::Say(tostr() << "Batchsize: " << "configured " << batchsize << ", min "
<< event_cb.min_batchsize << ", max "
<< event_cb.max_batchsize << ", avg "
<< event_cb.avg_batchsize << "\n");
diff --git a/tests/0112-assign_unknown_part.c b/tests/0112-assign_unknown_part.c
index a32d8f39ad..d5549c99e7 100644
--- a/tests/0112-assign_unknown_part.c
+++ b/tests/0112-assign_unknown_part.c
@@ -43,7 +43,6 @@ int main_0112_assign_unknown_part(int argc, char **argv) {
uint64_t testid = test_id_generate();
rd_kafka_t *c;
rd_kafka_topic_partition_list_t *tpl;
- int r;
test_conf_init(NULL, NULL, 60);
@@ -51,8 +50,7 @@ int main_0112_assign_unknown_part(int argc, char **argv) {
c = test_create_consumer(topic, NULL, NULL, NULL);
TEST_SAY("Creating topic %s with 1 partition\n", topic);
- test_create_topic(c, topic, 1, 1);
- test_wait_topic_exists(c, topic, 10 * 1000);
+ test_create_topic_wait_exists(c, topic, 1, 1, 10 * 1000);
TEST_SAY("Producing message to partition 0\n");
test_produce_msgs_easy(topic, testid, 0, 1);
@@ -68,17 +66,8 @@ int main_0112_assign_unknown_part(int argc, char **argv) {
TEST_SAY("Changing partition count for topic %s\n", topic);
test_create_partitions(NULL, topic, 2);
- /* FIXME: The new partition might not have propagated through the
- * cluster by the time the producer tries to produce to it
- * which causes the produce to fail.
- * Loop until the partition count is correct. */
- while ((r = test_get_partition_count(c, topic, 5000)) != 2) {
- TEST_SAY(
- "Waiting for %s partition count to reach 2, "
- "currently %d\n",
- topic, r);
- rd_sleep(1);
- }
+ /* Allow the partition to propagate */
+ rd_sleep(1);
TEST_SAY("Producing message to partition 1\n");
test_produce_msgs_easy(topic, testid, 1, 1);
diff --git a/tests/0113-cooperative_rebalance.cpp b/tests/0113-cooperative_rebalance.cpp
index e94b1b7853..c9b068cfd6 100644
--- a/tests/0113-cooperative_rebalance.cpp
+++ b/tests/0113-cooperative_rebalance.cpp
@@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2020-2022, Magnus Edenhill
+ * 2023, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -26,10 +27,6 @@
* POSSIBILITY OF SUCH DAMAGE.
*/
-extern "C" {
-#include "../src/rdkafka_protocol.h"
-#include "test.h"
-}
#include
#include