Merge branch 'trunk' into KAFKA-19434

Conflicts:
	streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java
	streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java
This commit is contained in:
Nikita 2025-10-03 10:19:07 -07:00
commit bfda2319c6
No known key found for this signature in database
349 changed files with 10574 additions and 3285 deletions

View File

@ -37,7 +37,7 @@ runs:
using: "composite"
steps:
- name: Setup Java
uses: actions/setup-java@v4
uses: actions/setup-java@v5
with:
distribution: temurin
java-version: ${{ inputs.java-version }}

View File

@ -22,7 +22,7 @@ runs:
using: "composite"
steps:
- name: Setup Python
uses: actions/setup-python@v5
uses: actions/setup-python@v6
with:
python-version: 3.12
- name: Pip install

View File

@ -12,6 +12,8 @@
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
# Note: Ensure the 'requests' version here matches the version in tests/setup.py
PyYAML~=6.0
pytz==2024.2
requests==2.32.3
requests==2.32.4

View File

@ -66,12 +66,12 @@ jobs:
name: Load Test Catalog
steps:
- name: Checkout main
uses: actions/checkout@v4
uses: actions/checkout@v5
with:
persist-credentials: false
- name: Checkout test-catalog
uses: actions/checkout@v4
uses: actions/checkout@v5
with:
ref: 'test-catalog'
persist-credentials: false
@ -118,7 +118,7 @@ jobs:
env:
GITHUB_CONTEXT: ${{ toJson(github) }}
- name: Checkout code
uses: actions/checkout@v4
uses: actions/checkout@v5
with:
persist-credentials: false
ref: ${{ github.sha }} # this is the default, just being explicit.
@ -192,7 +192,7 @@ jobs:
name: JUnit tests Java ${{ matrix.java }}${{ matrix.run-flaky == true && ' (flaky)' || '' }}${{ matrix.run-new == true && ' (new)' || '' }}
steps:
- name: Checkout code
uses: actions/checkout@v4
uses: actions/checkout@v5
with:
persist-credentials: false
ref: ${{ needs.configure.outputs.sha }}
@ -210,7 +210,7 @@ jobs:
# the overall workflow, so we'll continue here without a test catalog.
- name: Load Test Catalog
id: load-test-catalog
uses: actions/download-artifact@v4
uses: actions/download-artifact@v5
continue-on-error: true
with:
name: combined-test-catalog
@ -282,11 +282,11 @@ jobs:
uploaded-test-catalog: ${{ steps.archive-test-catalog.outcome == 'success' }}
steps:
- name: Checkout code
uses: actions/checkout@v4
uses: actions/checkout@v5
with:
persist-credentials: false
- name: Download Thread Dumps
uses: actions/download-artifact@v4
uses: actions/download-artifact@v5
with:
pattern: junit-thread-dumps-24-*
path: thread-dumps
@ -300,7 +300,7 @@ jobs:
exit 1;
fi
- name: Download JUnit XMLs
uses: actions/download-artifact@v4
uses: actions/download-artifact@v5
with:
pattern: junit-xml-24-* # Only look at JDK 24 tests for the test catalog
path: junit-xml
@ -334,7 +334,7 @@ jobs:
contents: write
steps:
- name: Checkout Test Catalog
uses: actions/checkout@v4
uses: actions/checkout@v5
with:
persist-credentials: true # Needed to commit and push later
ref: test-catalog
@ -342,7 +342,7 @@ jobs:
run: |
rm -rf test-catalog
- name: Download Test Catalog
uses: actions/download-artifact@v4
uses: actions/download-artifact@v5
with:
name: test-catalog
path: test-catalog

View File

@ -61,7 +61,7 @@ jobs:
env:
GITHUB_CONTEXT: ${{ toJson(github) }}
- name: Checkout code
uses: actions/checkout@v4
uses: actions/checkout@v5
with:
persist-credentials:
false
@ -72,7 +72,7 @@ jobs:
develocity-access-key: ${{ secrets.DEVELOCITY_ACCESS_KEY }}
- name: Download build scan archive
id: download-build-scan
uses: actions/download-artifact@v4
uses: actions/download-artifact@v5
continue-on-error: true # Don't want this step to fail the overall workflow
with:
github-token: ${{ github.token }}

View File

@ -42,7 +42,7 @@ jobs:
name: Deflake JUnit tests
steps:
- name: Checkout code
uses: actions/checkout@v4
uses: actions/checkout@v5
with:
persist-credentials: false

View File

@ -32,9 +32,9 @@ jobs:
build:
runs-on: ubuntu-latest
steps:
- uses: actions/checkout@v4
- uses: actions/checkout@v5
- name: Set up Python 3.10
uses: actions/setup-python@v5
uses: actions/setup-python@v6
with:
python-version: "3.10"
- name: Setup Docker Compose

View File

@ -31,9 +31,9 @@ jobs:
build:
runs-on: ubuntu-latest
steps:
- uses: actions/checkout@v4
- uses: actions/checkout@v5
- name: Set up Python 3.10
uses: actions/setup-python@v5
uses: actions/setup-python@v6
with:
python-version: "3.10"
- name: Setup Docker Compose

View File

@ -37,9 +37,9 @@ jobs:
runs-on: ubuntu-latest
steps:
- name: Checkout
uses: actions/checkout@v4
uses: actions/checkout@v5
- name: Set up Python 3.10
uses: actions/setup-python@v5
uses: actions/setup-python@v6
with:
python-version: "3.10"
- name: Install dependencies

View File

@ -32,7 +32,7 @@ jobs:
env:
GITHUB_CONTEXT: ${{ toJson(github) }}
- name: Checkout code
uses: actions/checkout@v4
uses: actions/checkout@v5
- name: Setup Python
uses: ./.github/actions/setup-python
- name: Run Report

View File

@ -35,7 +35,7 @@ jobs:
env:
GITHUB_CONTEXT: ${{ toJson(github) }}
- name: Checkout code
uses: actions/checkout@v4
uses: actions/checkout@v5
with:
persist-credentials:
false

View File

@ -31,12 +31,12 @@ jobs:
env:
GITHUB_CONTEXT: ${{ toJson(github) }}
- name: Checkout code
uses: actions/checkout@v4
uses: actions/checkout@v5
with:
persist-credentials: false
- name: Load PR Number
id: load-pr-number
uses: actions/download-artifact@v4
uses: actions/download-artifact@v5
with:
github-token: ${{ github.token }}
run-id: ${{ github.event.workflow_run.id }}

View File

@ -37,8 +37,8 @@ jobs:
runs-on: ubuntu-latest
steps:
- name: Checkout code
uses: actions/checkout@v4
- uses: actions/labeler@v5
uses: actions/checkout@v5
- uses: actions/labeler@v6
with:
configuration-path: .github/configs/labeler.yml
- name: check small label

View File

@ -31,9 +31,9 @@ jobs:
build:
runs-on: ubuntu-latest
steps:
- uses: actions/checkout@v4
- uses: actions/checkout@v5
- name: Set up Python 3.10
uses: actions/setup-python@v5
uses: actions/setup-python@v6
with:
python-version: "3.10"
- name: Install dependencies

View File

@ -38,7 +38,7 @@ jobs:
env:
GITHUB_CONTEXT: ${{ toJson(github) }}
- name: Checkout code
uses: actions/checkout@v4
uses: actions/checkout@v5
with:
persist-credentials:
false

View File

@ -211,7 +211,7 @@ License Version 2.0:
- commons-digester-2.1
- commons-lang3-3.18.0
- commons-logging-1.3.5
- commons-validator-1.9.0
- commons-validator-1.10.0
- hash4j-0.22.0
- jackson-annotations-2.19.0
- jackson-core-2.19.0

View File

@ -282,7 +282,7 @@ fi
# JVM performance options
# MaxInlineLevel=15 is the default since JDK 14 and can be removed once older JDKs are no longer supported
if [ -z "$KAFKA_JVM_PERFORMANCE_OPTS" ]; then
KAFKA_JVM_PERFORMANCE_OPTS="-server -XX:+UseG1GC -XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35 -XX:+ExplicitGCInvokesConcurrent -XX:MaxInlineLevel=15 -Djava.awt.headless=true"
KAFKA_JVM_PERFORMANCE_OPTS="-server -XX:+UseG1GC -XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35 -XX:+ExplicitGCInvokesConcurrent -XX:MaxInlineLevel=15"
fi
while [ $# -gt 0 ]; do

View File

@ -177,7 +177,7 @@ IF ["%KAFKA_HEAP_OPTS%"] EQU [""] (
rem JVM performance options
IF ["%KAFKA_JVM_PERFORMANCE_OPTS%"] EQU [""] (
set KAFKA_JVM_PERFORMANCE_OPTS=-server -XX:+UseG1GC -XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35 -XX:+ExplicitGCInvokesConcurrent -Djava.awt.headless=true
set KAFKA_JVM_PERFORMANCE_OPTS=-server -XX:+UseG1GC -XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35 -XX:+ExplicitGCInvokesConcurrent
)
IF not defined CLASSPATH (

View File

@ -157,7 +157,7 @@ ext {
libs.log4j2Api,
libs.log4j2Core
]
}
allprojects {
@ -193,8 +193,6 @@ allprojects {
// ensure we have a single version in the classpath despite transitive dependencies
libs.scalaLibrary,
libs.scalaReflect,
// Workaround before `commons-validator` has new release. See KAFKA-19359.
libs.commonsBeanutils,
libs.jacksonAnnotations,
libs.commonsLang
)
@ -484,11 +482,36 @@ subprojects {
}
}
// Workaround for Mockito Java Agent restrictions in Java 21+
// Starting with Java 21, the JDK restricts libraries from attaching a Java agent
// to their own JVM. As a result, Mockitos inline mock maker (mockito-core)
// fails without explicit instrumentation, and the JVM consistently emits warnings.
// See also: https://javadoc.io/doc/org.mockito/mockito-core/latest/org.mockito/org/mockito/Mockito.html#mockito-instrumentation
afterEvaluate { subproject ->
def hasMockitoCore = subproject.configurations.findAll {
it.canBeResolved
}.any { config ->
config.incoming.dependencies.any { dependency ->
"$dependency" == libs.mockitoCore
}
}
if (hasMockitoCore) {
subproject.configurations {
mockitoAgent {
transitive = false
}
}
subproject.dependencies {
mockitoAgent libs.mockitoCore
}
}
}
// The suites are for running sets of tests in IDEs.
// Gradle will run each test class, so we exclude the suites to avoid redundantly running the tests twice.
def testsToExclude = ['**/*Suite.class']
// This task will copy JUnit XML files out of the sub-project's build directory and into
// a top-level build/junit-xml directory. This is necessary to avoid reporting on tests which
// were not run, but instead were restored via FROM-CACHE. See KAFKA-17479 for more details.
@ -518,6 +541,14 @@ subprojects {
}
test {
doFirst {
def mockitoAgentConfig = configurations.findByName('mockitoAgent')
if (mockitoAgentConfig) {
jvmArgs("-javaagent:${mockitoAgentConfig.asPath}")
}
}
maxParallelForks = maxTestForks
ignoreFailures = userIgnoreFailures
@ -551,7 +582,7 @@ subprojects {
maxFailures = userMaxTestRetryFailures
}
}
finalizedBy("copyTestXml")
}
@ -3316,6 +3347,22 @@ project(':streams:upgrade-system-tests-40') {
}
}
project(':streams:upgrade-system-tests-41') {
base {
archivesName = "kafka-streams-upgrade-system-tests-41"
}
dependencies {
testImplementation libs.kafkaStreams_41
testRuntimeOnly libs.junitJupiter
testRuntimeOnly runtimeTestLibs
}
systemTestLibs {
dependsOn testJar
}
}
project(':jmh-benchmarks') {
apply plugin: 'com.gradleup.shadow'
@ -3334,6 +3381,7 @@ project(':jmh-benchmarks') {
implementation project(':raft')
implementation project(':clients')
implementation project(':coordinator-common')
implementation project(':coordinator-common').sourceSets.test.output
implementation project(':group-coordinator')
implementation project(':group-coordinator:group-coordinator-api')
implementation project(':metadata')

View File

@ -52,9 +52,7 @@
<allow pkg="org.apache.kafka.server"/>
<allow pkg="org.apache.kafka.storage"/>
<allow pkg="org.apache.kafka.clients"/>
<allow class="org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage"/>
<allow class="org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataImage"/>
<allow class="org.apache.kafka.coordinator.common.runtime.HdrHistogram"/>
<allow pkg="org.apache.kafka.coordinator.common.runtime"/>
<allow pkg="org.apache.kafka.coordinator.group"/>
<allow pkg="org.apache.kafka.image"/>
<allow pkg="org.apache.kafka.metadata"/>

View File

@ -107,6 +107,7 @@
<allow pkg="org.apache.kafka.common.config" />
<allow pkg="org.apache.kafka.common.message" />
<allow pkg="org.apache.kafka.common.metadata" />
<allow pkg="org.apache.kafka.common.network" />
<allow pkg="org.apache.kafka.common.protocol" />
<allow pkg="org.apache.kafka.common.quota" />
<allow pkg="org.apache.kafka.common.record" />
@ -161,6 +162,7 @@
<allow pkg="org.apache.kafka.metadata" />
<allow pkg="org.apache.kafka.queue" />
<allow pkg="org.apache.kafka.raft" />
<allow pkg="org.apache.kafka.security" />
<allow pkg="org.apache.kafka.server.authorizer" />
<allow pkg="org.apache.kafka.server.common" />
<allow pkg="org.apache.kafka.server.fault" />

View File

@ -33,6 +33,7 @@
<allow pkg="javax.net.ssl" />
<allow pkg="javax.security" />
<allow pkg="net.jqwik.api" />
<allow pkg="javax.crypto" />
<!-- no one depends on the server -->
<disallow pkg="kafka" />
@ -49,6 +50,9 @@
<!-- persistent collection factories/non-library-specific wrappers -->
<allow pkg="org.apache.kafka.server.immutable" exact-match="true" />
<!-- allow config classes for server package -->
<allow pkg="org.apache.kafka.server.config" />
<subpackage name="queue">
<allow pkg="org.apache.kafka.test" />
</subpackage>
@ -60,6 +64,7 @@
</subpackage>
<subpackage name="security">
<allow pkg="org.apache.kafka.clients.admin" />
<allow pkg="org.apache.kafka.common.config" />
<allow pkg="org.apache.kafka.common.config.types" />
<allow pkg="org.apache.kafka.server.util" />

View File

@ -40,6 +40,7 @@
<allow pkg="org.apache.kafka.common.test.api" />
<allow pkg="org.apache.kafka.test" />
<allow pkg="org.slf4j" />
<allow pkg="kafka.coordinator.transaction" />
<subpackage name="generated">
<allow pkg="org.apache.kafka.common.protocol" />
<allow pkg="org.apache.kafka.common.errors" />

View File

@ -497,6 +497,7 @@
<allow pkg="org.apache.kafka.server.common.serialization" />
<allow pkg="org.apache.kafka.server.config" />
<allow pkg="org.apache.kafka.server.fault"/>
<allow pkg="org.apache.kafka.server.metrics" />
<allow pkg="org.apache.kafka.server.util" />
<allow pkg="org.apache.kafka.test"/>
<allow pkg="com.fasterxml.jackson" />

View File

@ -0,0 +1,132 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.clients.admin;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.errors.InconsistentClusterIdException;
import org.apache.kafka.common.test.KafkaClusterTestKit;
import org.apache.kafka.common.test.TestKitNodes;
import org.apache.kafka.test.TestUtils;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import java.util.stream.Collectors;
import static org.junit.jupiter.api.Assertions.assertEquals;
@Tag("integration")
public class ReconfigurableQuorumIntegrationTest {
static Map<Integer, Uuid> descVoterDirs(Admin admin) throws ExecutionException, InterruptedException {
var quorumInfo = admin.describeMetadataQuorum().quorumInfo().get();
return quorumInfo.voters().stream().collect(Collectors.toMap(QuorumInfo.ReplicaState::replicaId, QuorumInfo.ReplicaState::replicaDirectoryId));
}
@Test
public void testRemoveAndAddVoterWithValidClusterId() throws Exception {
final var nodes = new TestKitNodes.Builder()
.setClusterId("test-cluster")
.setNumBrokerNodes(1)
.setNumControllerNodes(3)
.build();
final Map<Integer, Uuid> initialVoters = new HashMap<>();
for (final var controllerNode : nodes.controllerNodes().values()) {
initialVoters.put(
controllerNode.id(),
controllerNode.metadataDirectoryId()
);
}
try (var cluster = new KafkaClusterTestKit.Builder(nodes).setInitialVoterSet(initialVoters).build()) {
cluster.format();
cluster.startup();
try (Admin admin = Admin.create(cluster.clientProperties())) {
TestUtils.waitForCondition(() -> {
Map<Integer, Uuid> voters = descVoterDirs(admin);
assertEquals(Set.of(3000, 3001, 3002), voters.keySet());
return voters.values().stream().noneMatch(directory -> directory.equals(Uuid.ZERO_UUID));
}, "Initial quorum voters should be {3000, 3001, 3002} and all should have non-zero directory IDs");
Uuid dirId = cluster.nodes().controllerNodes().get(3000).metadataDirectoryId();
admin.removeRaftVoter(
3000,
dirId,
new RemoveRaftVoterOptions().setClusterId(Optional.of("test-cluster"))
).all().get();
TestUtils.waitForCondition(() -> {
Map<Integer, Uuid> voters = descVoterDirs(admin);
assertEquals(Set.of(3001, 3002), voters.keySet());
return voters.values().stream().noneMatch(directory -> directory.equals(Uuid.ZERO_UUID));
}, "After removing voter 3000, remaining voters should be {3001, 3002} with non-zero directory IDs");
admin.addRaftVoter(
3000,
dirId,
Set.of(new RaftVoterEndpoint("CONTROLLER", "example.com", 8080)),
new AddRaftVoterOptions().setClusterId(Optional.of("test-cluster"))
).all().get();
}
}
}
@Test
public void testRemoveAndAddVoterWithInconsistentClusterId() throws Exception {
final var nodes = new TestKitNodes.Builder()
.setClusterId("test-cluster")
.setNumBrokerNodes(1)
.setNumControllerNodes(3)
.build();
final Map<Integer, Uuid> initialVoters = new HashMap<>();
for (final var controllerNode : nodes.controllerNodes().values()) {
initialVoters.put(
controllerNode.id(),
controllerNode.metadataDirectoryId()
);
}
try (var cluster = new KafkaClusterTestKit.Builder(nodes).setInitialVoterSet(initialVoters).build()) {
cluster.format();
cluster.startup();
try (Admin admin = Admin.create(cluster.clientProperties())) {
Uuid dirId = cluster.nodes().controllerNodes().get(3000).metadataDirectoryId();
var removeFuture = admin.removeRaftVoter(
3000,
dirId,
new RemoveRaftVoterOptions().setClusterId(Optional.of("inconsistent"))
).all();
TestUtils.assertFutureThrows(InconsistentClusterIdException.class, removeFuture);
var addFuture = admin.addRaftVoter(
3000,
dirId,
Set.of(new RaftVoterEndpoint("CONTROLLER", "example.com", 8080)),
new AddRaftVoterOptions().setClusterId(Optional.of("inconsistent"))
).all();
TestUtils.assertFutureThrows(InconsistentClusterIdException.class, addFuture);
}
}
}
}

View File

@ -27,7 +27,9 @@ import java.util.Collection;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.BiConsumer;
import static org.apache.kafka.clients.ClientsTestUtils.consumeAndVerifyRecords;
@ -166,6 +168,80 @@ public class PlaintextConsumerCallbackTest {
testGetPositionOfNewlyAssignedPartitionOnPartitionsAssignedCallback(CLASSIC);
}
@ClusterTest
public void testOnPartitionsAssignedCalledWithNewPartitionsOnlyForClassicCooperative() throws InterruptedException {
try (var consumer = createClassicConsumerCooperativeProtocol()) {
testOnPartitionsAssignedCalledWithExpectedPartitions(consumer, true);
}
}
@ClusterTest
public void testOnPartitionsAssignedCalledWithNewPartitionsOnlyForAsyncConsumer() throws InterruptedException {
try (var consumer = createConsumer(CONSUMER)) {
testOnPartitionsAssignedCalledWithExpectedPartitions(consumer, true);
}
}
@ClusterTest
public void testOnPartitionsAssignedCalledWithNewPartitionsOnlyForClassicEager() throws InterruptedException {
try (var consumer = createConsumer(CLASSIC)) {
testOnPartitionsAssignedCalledWithExpectedPartitions(consumer, false);
}
}
private void testOnPartitionsAssignedCalledWithExpectedPartitions(
Consumer<byte[], byte[]> consumer,
boolean expectNewPartitionsOnlyInCallback) throws InterruptedException {
subscribeAndExpectOnPartitionsAssigned(consumer, List.of(topic), List.of(tp));
assertEquals(Set.of(tp), consumer.assignment());
// Add a new partition assignment while keeping the previous one
String newTopic = "newTopic";
TopicPartition addedPartition = new TopicPartition(newTopic, 0);
List<TopicPartition> expectedPartitionsInCallback;
if (expectNewPartitionsOnlyInCallback) {
expectedPartitionsInCallback = List.of(addedPartition);
} else {
expectedPartitionsInCallback = List.of(tp, addedPartition);
}
// Change subscription to keep the previous one and add a new topic. Assignment should be updated
// to contain partitions from both topics, but the onPartitionsAssigned parameters may containing
// the full new assignment or just the newly added partitions depending on the case.
subscribeAndExpectOnPartitionsAssigned(
consumer,
List.of(topic, newTopic),
expectedPartitionsInCallback);
assertEquals(Set.of(tp, addedPartition), consumer.assignment());
}
private void subscribeAndExpectOnPartitionsAssigned(Consumer<byte[], byte[]> consumer, List<String> topics, Collection<TopicPartition> expectedPartitionsInCallback) throws InterruptedException {
var partitionsAssigned = new AtomicBoolean(false);
AtomicReference<Collection<TopicPartition>> partitionsFromCallback = new AtomicReference<>();
consumer.subscribe(topics, new ConsumerRebalanceListener() {
@Override
public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
if (partitions.containsAll(expectedPartitionsInCallback)) {
partitionsFromCallback.set(partitions);
partitionsAssigned.set(true);
}
}
@Override
public void onPartitionsRevoked(Collection<TopicPartition> partitions) {
// noop
}
});
ClientsTestUtils.pollUntilTrue(
consumer,
partitionsAssigned::get,
"Timed out before expected rebalance completed"
);
// These are different types, so comparing values instead
assertTrue(expectedPartitionsInCallback.containsAll(partitionsFromCallback.get()) && partitionsFromCallback.get().containsAll(expectedPartitionsInCallback),
"Expected partitions " + expectedPartitionsInCallback + " as parameter for onPartitionsAssigned, but got " + partitionsFromCallback.get());
}
@ClusterTest
public void testAsyncConsumerGetPositionOfNewlyAssignedPartitionOnPartitionsAssignedCallback() throws InterruptedException {
testGetPositionOfNewlyAssignedPartitionOnPartitionsAssignedCallback(CONSUMER);
@ -284,4 +360,12 @@ public class PlaintextConsumerCallbackTest {
ENABLE_AUTO_COMMIT_CONFIG, "false"
));
}
private Consumer<byte[], byte[]> createClassicConsumerCooperativeProtocol() {
return cluster.consumer(Map.of(
GROUP_PROTOCOL_CONFIG, CLASSIC.name.toLowerCase(Locale.ROOT),
ENABLE_AUTO_COMMIT_CONFIG, "false",
ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "org.apache.kafka.clients.consumer.CooperativeStickyAssignor"
));
}
}

View File

@ -28,6 +28,7 @@ import org.apache.kafka.common.test.api.ClusterTest;
import org.apache.kafka.common.test.api.ClusterTestDefaults;
import org.apache.kafka.common.test.api.Type;
import org.apache.kafka.test.MockConsumerInterceptor;
import org.apache.kafka.test.TestUtils;
import org.junit.jupiter.api.BeforeEach;
@ -452,6 +453,40 @@ public class PlaintextConsumerCommitTest {
}
}
/**
* This is testing when closing the consumer but commit request has already been sent.
* During the closing, the consumer won't find the coordinator anymore.
*/
@ClusterTest
public void testCommitAsyncFailsWhenCoordinatorUnavailableDuringClose() throws InterruptedException {
try (Producer<byte[], byte[]> producer = cluster.producer();
var consumer = createConsumer(GroupProtocol.CONSUMER, false)
) {
sendRecords(producer, tp, 3, System.currentTimeMillis());
consumer.assign(List.of(tp));
var callback = new CountConsumerCommitCallback();
// Close the coordinator before committing because otherwise the commit will fail to find the coordinator.
cluster.brokerIds().forEach(cluster::shutdownBroker);
TestUtils.waitForCondition(() -> cluster.aliveBrokers().isEmpty(), "All brokers should be shut down");
consumer.poll(Duration.ofMillis(500));
consumer.commitAsync(Map.of(tp, new OffsetAndMetadata(1L)), callback);
long startTime = System.currentTimeMillis();
consumer.close(CloseOptions.timeout(Duration.ofMillis(500)));
long closeDuration = System.currentTimeMillis() - startTime;
assertTrue(closeDuration < 1000, "The closing process for the consumer was too long: " + closeDuration + " ms");
assertTrue(callback.lastError.isPresent());
assertEquals(CommitFailedException.class, callback.lastError.get().getClass());
assertEquals("Failed to commit offsets: Coordinator unknown and consumer is closing", callback.lastError.get().getMessage());
assertEquals(1, callback.exceptionCount);
}
}
// TODO: This only works in the new consumer, but should be fixed for the old consumer as well
@ClusterTest
public void testCommitAsyncCompletedBeforeConsumerCloses() throws InterruptedException {
@ -575,6 +610,7 @@ public class PlaintextConsumerCommitTest {
private static class CountConsumerCommitCallback implements OffsetCommitCallback {
private int successCount = 0;
private int exceptionCount = 0;
private Optional<Exception> lastError = Optional.empty();
@Override
@ -582,6 +618,7 @@ public class PlaintextConsumerCommitTest {
if (exception == null) {
successCount += 1;
} else {
exceptionCount += 1;
lastError = Optional.of(exception);
}
}

View File

@ -203,7 +203,10 @@ public class PlaintextConsumerTest {
) {
var record = new ProducerRecord<>(TP.topic(), TP.partition(), null, "key".getBytes(), "value".getBytes());
record.headers().add("headerKey", "headerValue".getBytes());
record.headers().add("headerKey2", "headerValue2".getBytes());
record.headers().add("headerKey3", "headerValue3".getBytes());
producer.send(record);
producer.flush();
assertEquals(0, consumer.assignment().size());
consumer.assign(List.of(TP));
@ -212,8 +215,15 @@ public class PlaintextConsumerTest {
consumer.seek(TP, 0);
var records = consumeRecords(consumer, numRecords);
assertEquals(numRecords, records.size());
var header = records.get(0).headers().lastHeader("headerKey");
assertEquals("headerValue", header == null ? null : new String(header.value()));
// Test the order of headers in a record is preserved when producing and consuming
Header[] headers = records.get(0).headers().toArray();
assertEquals("headerKey", headers[0].key());
assertEquals("headerKey2", headers[1].key());
assertEquals("headerKey3", headers[2].key());
}
}

View File

@ -467,6 +467,8 @@ public class ShareConsumerTest {
int numRecords = 1;
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
record.headers().add("headerKey", "headerValue".getBytes());
record.headers().add("headerKey2", "headerValue2".getBytes());
record.headers().add("headerKey3", "headerValue3".getBytes());
producer.send(record);
producer.flush();
@ -475,11 +477,15 @@ public class ShareConsumerTest {
List<ConsumerRecord<byte[], byte[]>> records = consumeRecords(shareConsumer, numRecords);
assertEquals(numRecords, records.size());
for (ConsumerRecord<byte[], byte[]> consumerRecord : records) {
Header header = consumerRecord.headers().lastHeader("headerKey");
if (header != null)
assertEquals("headerValue", new String(header.value()));
}
Header header = records.get(0).headers().lastHeader("headerKey");
assertEquals("headerValue", new String(header.value()));
// Test the order of headers in a record is preserved when producing and consuming
Header[] headers = records.get(0).headers().toArray();
assertEquals("headerKey", headers[0].key());
assertEquals("headerKey2", headers[1].key());
assertEquals("headerKey3", headers[2].key());
verifyShareGroupStateTopicRecordsProduced();
}
}

View File

@ -204,10 +204,10 @@ public class ProducerIdExpirationTest {
// Update the producer ID expiration ms to a very high value.
admin.incrementalAlterConfigs(producerIdExpirationConfig("100000"));
cluster.brokers().values().forEach(broker -> {
cluster.brokers().values().forEach(broker ->
TestUtils.waitUntilTrue(() -> broker.logManager().producerStateManagerConfig().producerIdExpirationMs() == 100000,
() -> "Configuration was not updated.", DEFAULT_MAX_WAIT_MS, 100);
});
() -> "Configuration was not updated.", DEFAULT_MAX_WAIT_MS, 100)
);
// Send more records to send producer ID back to brokers.
producer.send(new ProducerRecord<>(topic1, 0, null, "key".getBytes(), "value".getBytes()));
producer.flush();
@ -226,10 +226,10 @@ public class ProducerIdExpirationTest {
kafkaBroker.awaitShutdown();
kafkaBroker.startup();
cluster.waitForReadyBrokers();
cluster.brokers().values().forEach(broker -> {
cluster.brokers().values().forEach(broker ->
TestUtils.waitUntilTrue(() -> broker.logManager().producerStateManagerConfig().producerIdExpirationMs() == 100,
() -> "Configuration was not updated.", DEFAULT_MAX_WAIT_MS, 100);
});
() -> "Configuration was not updated.", DEFAULT_MAX_WAIT_MS, 100)
);
// Ensure producer ID expires quickly again.
waitProducerIdExpire(admin);

View File

@ -184,9 +184,8 @@ public class ProducerSendWhileDeletionTest {
try (var producer = createProducer()) {
for (int i = 1; i <= numRecords; i++) {
producer.send(new ProducerRecord<>(topic, null, ("value" + i).getBytes()),
(metadata, exception) -> {
numAcks.incrementAndGet();
});
(metadata, exception) -> numAcks.incrementAndGet()
);
}
producer.flush();
}

View File

@ -17,11 +17,20 @@
package org.apache.kafka.clients.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.protocol.Errors;
import java.util.Optional;
/**
* Options for {@link Admin#addRaftVoter}.
*
* <p>
* The clusterId is optional.
* <p>
* If provided, the request will only succeed if the cluster id matches the id of the current cluster.
* If the cluster id does not match, the request will fail with {@link Errors#INCONSISTENT_CLUSTER_ID}.
* <p>
* If not provided, the cluster id check is skipped.
*/
@InterfaceStability.Stable
public class AddRaftVoterOptions extends AbstractOptions<AddRaftVoterOptions> {

View File

@ -32,6 +32,7 @@ import org.apache.kafka.common.acl.AclBindingFilter;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.config.ConfigResource;
import org.apache.kafka.common.errors.FeatureUpdateFailedException;
import org.apache.kafka.common.errors.InconsistentClusterIdException;
import org.apache.kafka.common.errors.InterruptException;
import org.apache.kafka.common.metrics.KafkaMetric;
import org.apache.kafka.common.quota.ClientQuotaAlteration;
@ -1866,10 +1867,17 @@ public interface Admin extends AutoCloseable {
/**
* Add a new voter node to the KRaft metadata quorum.
*
* <p>
* The clusterId in {@link AddRaftVoterOptions} is optional.
* If provided, the operation will only succeed if the cluster id matches the id
* of the current cluster. If the cluster id does not match, the operation
* will fail with {@link InconsistentClusterIdException}.
* If not provided, the cluster id check is skipped.
*
* @param voterId The node ID of the voter.
* @param voterDirectoryId The directory ID of the voter.
* @param endpoints The endpoints that the new voter has.
* @param options The options to use when adding the new voter node.
* @param options Additional options for the operation, including optional cluster ID.
*/
AddRaftVoterResult addRaftVoter(
int voterId,
@ -1894,9 +1902,16 @@ public interface Admin extends AutoCloseable {
/**
* Remove a voter node from the KRaft metadata quorum.
*
* <p>
* The clusterId in {@link AddRaftVoterOptions} is optional.
* If provided, the operation will only succeed if the cluster id matches the id
* of the current cluster. If the cluster id does not match, the operation
* will fail with {@link InconsistentClusterIdException}.
* If not provided, the cluster id check is skipped.
*
* @param voterId The node ID of the voter.
* @param voterDirectoryId The directory ID of the voter.
* @param options The options to use when removing the voter node.
* @param options Additional options for the operation, including optional cluster ID.
*/
RemoveRaftVoterResult removeRaftVoter(
int voterId,

View File

@ -22,7 +22,7 @@ import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection;
/**
* Options for the {@link Admin#deleteShareGroups(Collection <String>, DeleteShareGroupsOptions)} call.
* Options for the {@link Admin#deleteShareGroups(Collection, DeleteShareGroupsOptions)} call.
* <p>
* The API of this class is evolving, see {@link Admin} for details.
*/

View File

@ -25,7 +25,7 @@ import java.util.HashMap;
import java.util.Map;
/**
* The result of the {@link Admin#deleteShareGroups(Collection <String>, DeleteShareGroupsOptions)} call.
* The result of the {@link Admin#deleteShareGroups(Collection, DeleteShareGroupsOptions)} call.
* <p></p>
* The API of this class is evolving, see {@link Admin} for details.
*/

View File

@ -21,7 +21,7 @@ import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection;
/**
* Options for the {@link Admin#deleteStreamsGroups(Collection<String>, DeleteStreamsGroupsOptions)} call.
* Options for the {@link Admin#deleteStreamsGroups(Collection, DeleteStreamsGroupsOptions)} call.
* <p>
* The API of this class is evolving, see {@link Admin} for details.
*/

View File

@ -2494,8 +2494,7 @@ public class KafkaAdminClient extends AdminClient {
DescribeClusterResponse response = (DescribeClusterResponse) abstractResponse;
Errors error = Errors.forCode(response.data().errorCode());
if (error != Errors.NONE) {
ApiError apiError = new ApiError(error, response.data().errorMessage());
handleFailure(apiError.exception());
handleFailure(error.exception(response.data().errorMessage()));
return;
}
@ -2691,10 +2690,9 @@ public class KafkaAdminClient extends AdminClient {
} else {
List<FilterResult> filterResults = new ArrayList<>();
for (DeleteAclsMatchingAcl matchingAcl : filterResult.matchingAcls()) {
ApiError aclError = new ApiError(Errors.forCode(matchingAcl.errorCode()),
matchingAcl.errorMessage());
Errors aclError = Errors.forCode(matchingAcl.errorCode());
AclBinding aclBinding = DeleteAclsResponse.aclBinding(matchingAcl);
filterResults.add(new FilterResult(aclBinding, aclError.exception()));
filterResults.add(new FilterResult(aclBinding, aclError.exception(matchingAcl.errorMessage())));
}
future.complete(new FilterResults(filterResults));
}
@ -3995,7 +3993,7 @@ public class KafkaAdminClient extends AdminClient {
for (ReassignablePartitionResponse partition : topicResponse.partitions()) {
errors.put(
new TopicPartition(topicName, partition.partitionIndex()),
new ApiError(topLevelError, response.data().errorMessage()).exception()
topLevelError.exception(response.data().errorMessage())
);
receivedResponsesCount += 1;
}
@ -4035,7 +4033,7 @@ public class KafkaAdminClient extends AdminClient {
if (partitionError == Errors.NONE) {
errors.put(tp, null);
} else {
errors.put(tp, new ApiError(partitionError, partResponse.errorMessage()).exception());
errors.put(tp, partitionError.exception(partResponse.errorMessage()));
}
receivedResponsesCount += 1;
}
@ -4111,7 +4109,7 @@ public class KafkaAdminClient extends AdminClient {
handleNotControllerError(error);
break;
default:
partitionReassignmentsFuture.completeExceptionally(new ApiError(error, response.data().errorMessage()).exception());
partitionReassignmentsFuture.completeExceptionally(error.exception(response.data().errorMessage()));
break;
}
Map<TopicPartition, PartitionReassignment> reassignmentMap = new HashMap<>();
@ -4993,14 +4991,11 @@ public class KafkaAdminClient extends AdminClient {
void handleResponse(AbstractResponse response) {
handleNotControllerError(response);
AddRaftVoterResponse addResponse = (AddRaftVoterResponse) response;
if (addResponse.data().errorCode() != Errors.NONE.code()) {
ApiError error = new ApiError(
addResponse.data().errorCode(),
addResponse.data().errorMessage());
future.completeExceptionally(error.exception());
} else {
Errors error = Errors.forCode(addResponse.data().errorCode());
if (error != Errors.NONE)
future.completeExceptionally(error.exception(addResponse.data().errorMessage()));
else
future.complete(null);
}
}
@Override
@ -5038,14 +5033,11 @@ public class KafkaAdminClient extends AdminClient {
void handleResponse(AbstractResponse response) {
handleNotControllerError(response);
RemoveRaftVoterResponse addResponse = (RemoveRaftVoterResponse) response;
if (addResponse.data().errorCode() != Errors.NONE.code()) {
ApiError error = new ApiError(
addResponse.data().errorCode(),
addResponse.data().errorMessage());
future.completeExceptionally(error.exception());
} else {
Errors error = Errors.forCode(addResponse.data().errorCode());
if (error != Errors.NONE)
future.completeExceptionally(error.exception(addResponse.data().errorMessage()));
else
future.complete(null);
}
}
@Override

View File

@ -46,7 +46,7 @@ public class ListShareGroupOffsetsResult {
/**
* Return the future when the requests for all groups succeed.
*
* @return Future which yields all {@code Map<String, Map<TopicPartition, Long>>} objects, if requests for all the groups succeed.
* @return Future which yields all {@code Map<String, Map<TopicPartition, OffsetAndMetadata>>} objects, if requests for all the groups succeed.
*/
public KafkaFuture<Map<String, Map<TopicPartition, OffsetAndMetadata>>> all() {
return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture<?>[0])).thenApply(

View File

@ -21,6 +21,7 @@ import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection;
import java.util.Map;
/**
* Specification of streams group offsets to list using {@link Admin#listStreamsGroupOffsets(Map, ListStreamsGroupOffsetsOptions)}.

View File

@ -33,14 +33,16 @@ public class RecordsToDelete {
/**
* Delete all the records before the given {@code offset}
*
* @param offset the offset before which all records will be deleted
* @param offset The offset before which all records will be deleted.
* Use {@code -1} to truncate to the high watermark.
*/
public static RecordsToDelete beforeOffset(long offset) {
return new RecordsToDelete(offset);
}
/**
* The offset before which all records will be deleted
* The offset before which all records will be deleted.
* Use {@code -1} to truncate to the high watermark.
*/
public long beforeOffset() {
return offset;

View File

@ -17,11 +17,20 @@
package org.apache.kafka.clients.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.protocol.Errors;
import java.util.Optional;
/**
* Options for {@link Admin#removeRaftVoter}.
*
* <p>
* The clusterId is optional.
* <p>
* If provided, the request will only succeed if the cluster id matches the id of the current cluster.
* If the cluster id does not match, the request will fail with {@link Errors#INCONSISTENT_CLUSTER_ID}.
* <p>
* If not provided, the cluster id check is skipped.
*/
@InterfaceStability.Stable
public class RemoveRaftVoterOptions extends AbstractOptions<RemoveRaftVoterOptions> {

View File

@ -122,8 +122,8 @@ public class DeleteShareGroupOffsetsHandler extends AdminApiHandler.Batched<Coor
if (topic.errorCode() != Errors.NONE.code()) {
final Errors topicError = Errors.forCode(topic.errorCode());
final String topicErrorMessage = topic.errorMessage();
log.debug("DeleteShareGroupOffsets request for group id {} and topic {} failed and returned error {}." + topicErrorMessage,
groupId.idValue, topic.topicName(), topicError);
log.debug("DeleteShareGroupOffsets request for group id {} and topic {} failed and returned error {}. {}",
groupId.idValue, topic.topicName(), topicError, topicErrorMessage);
}
topicResults.put(
topic.topicName(),
@ -147,14 +147,14 @@ public class DeleteShareGroupOffsetsHandler extends AdminApiHandler.Batched<Coor
case REBALANCE_IN_PROGRESS:
// If the coordinator is in the middle of loading, then we just need to retry
log.debug("DeleteShareGroupOffsets request for group id {} failed because the coordinator" +
" is still in the process of loading state. Will retry. " + errorMessage, groupId.idValue);
" is still in the process of loading state. Will retry. {}", groupId.idValue, errorMessage);
break;
case COORDINATOR_NOT_AVAILABLE:
case NOT_COORDINATOR:
// If the coordinator is unavailable or there was a coordinator change, then we unmap
// the key so that we retry the `FindCoordinator` request
log.debug("DeleteShareGroupOffsets request for group id {} returned error {}. Will rediscover the coordinator and retry. " + errorMessage,
groupId.idValue, error);
log.debug("DeleteShareGroupOffsets request for group id {} returned error {}. Will rediscover the coordinator and retry. {}",
groupId.idValue, error, errorMessage);
groupsToUnmap.add(groupId);
break;
case INVALID_GROUP_ID:
@ -164,11 +164,11 @@ public class DeleteShareGroupOffsetsHandler extends AdminApiHandler.Batched<Coor
case UNKNOWN_SERVER_ERROR:
case KAFKA_STORAGE_ERROR:
case GROUP_AUTHORIZATION_FAILED:
log.debug("DeleteShareGroupOffsets request for group id {} failed due to error {}. " + errorMessage, groupId.idValue, error);
log.debug("DeleteShareGroupOffsets request for group id {} failed due to error {}. {}", groupId.idValue, error, errorMessage);
failed.put(groupId, error.exception(errorMessage));
break;
default:
log.error("DeleteShareGroupOffsets request for group id {} failed due to unexpected error {}. " + errorMessage, groupId.idValue, error);
log.error("DeleteShareGroupOffsets request for group id {} failed due to unexpected error {}. {}", groupId.idValue, error, errorMessage);
failed.put(groupId, error.exception(errorMessage));
}
}

View File

@ -202,7 +202,7 @@ public final class ListOffsetsHandler extends Batched<TopicPartition, ListOffset
public Map<TopicPartition, Throwable> handleUnsupportedVersionException(
int brokerId, UnsupportedVersionException exception, Set<TopicPartition> keys
) {
log.warn("Broker " + brokerId + " does not support MAX_TIMESTAMP offset specs");
log.warn("Broker {} does not support MAX_TIMESTAMP offset specs", brokerId);
Map<TopicPartition, Throwable> maxTimestampPartitions = new HashMap<>();
for (TopicPartition topicPartition : keys) {
Long offsetTimestamp = offsetTimestampsByPartition.get(topicPartition);

View File

@ -50,7 +50,7 @@ import java.util.Collection;
* Under normal conditions, if a partition is reassigned from one consumer to another, then the old consumer will
* always invoke {@link #onPartitionsRevoked(Collection) onPartitionsRevoked} for that partition prior to the new consumer
* invoking {@link #onPartitionsAssigned(Collection) onPartitionsAssigned} for the same partition. So if offsets or other state is saved in the
* {@link #onPartitionsRevoked(Collection) onPartitionsRevoked} call by one consumer member, it will be always accessible by the time the
* {@link #onPartitionsRevoked(Collection) onPartitionsRevoked} call by one consumer member, it will always be accessible by the time the
* other consumer member taking over that partition and triggering its {@link #onPartitionsAssigned(Collection) onPartitionsAssigned} callback to load the state.
* <p>
* You can think of revocation as a graceful way to give up ownership of a partition. In some cases, the consumer may not have an opportunity to do so.
@ -120,13 +120,31 @@ public interface ConsumerRebalanceListener {
/**
* A callback method the user can implement to provide handling of offset commits to a customized store.
* This method will be called during a rebalance operation when the consumer has to give up some partitions.
* It can also be called when consumer is being closed ({@link KafkaConsumer#close(CloseOptions option)})
* or is unsubscribing ({@link KafkaConsumer#unsubscribe()}).
* The consumer may need to give up some partitions (thus this callback executed) under the following scenarios:
* <ul>
* <li>If the consumer assignment changes</li>
* <li>If the consumer is being closed ({@link KafkaConsumer#close(CloseOptions option)})</li>
* <li>If the consumer is unsubscribing ({@link KafkaConsumer#unsubscribe()})</li>
* </ul>
* It is recommended that offsets should be committed in this callback to either Kafka or a
* custom offset store to prevent duplicate data.
* <p>
* In eager rebalancing, it will always be called at the start of a rebalance and after the consumer stops fetching data.
* In cooperative rebalancing, it will be called at the end of a rebalance on the set of partitions being revoked iff the set is non-empty.
* This callback is always called before re-assigning the partitions.
* If the consumer is using the {@link GroupProtocol#CLASSIC} rebalance protocol:
* <ul>
* <li>
* In eager rebalancing, onPartitionsRevoked will be called with the full set of assigned partitions as a parameter (all partitions are revoked).
* It will be called even if there are no partitions to revoke.
* </li>
* <li>
* In cooperative rebalancing, onPartitionsRevoked will be called with the set of partitions to revoke,
* iff the set is non-empty.
* </li>
* </ul>
* If the consumer is using the {@link GroupProtocol#CONSUMER} rebalance protocol, this callback will be called
* with the set of partitions to revoke iff the set is non-empty
* (same behavior as the {@link GroupProtocol#CLASSIC} rebalance protocol with Cooperative mode).
* <p>
* For examples on usage of this API, see Usage Examples section of {@link KafkaConsumer KafkaConsumer}.
* <p>
* It is common for the revocation callback to use the consumer instance in order to commit offsets. It is possible
@ -135,8 +153,9 @@ public interface ConsumerRebalanceListener {
* invocation of {@link KafkaConsumer#poll(java.time.Duration)} in which this callback is being executed. This means it is not
* necessary to catch these exceptions and re-attempt to wakeup or interrupt the consumer thread.
*
* @param partitions The list of partitions that were assigned to the consumer and now need to be revoked (may not
* include all currently assigned partitions, i.e. there may still be some partitions left)
* @param partitions The list of partitions that were assigned to the consumer and now need to be revoked. This will
* include the full assignment under the Classic/Eager protocol, given that it revokes all partitions.
* It will only include the subset to revoke under the Classic/Cooperative and Consumer protocols.
* @throws org.apache.kafka.common.errors.WakeupException If raised from a nested call to {@link KafkaConsumer}
* @throws org.apache.kafka.common.errors.InterruptException If raised from a nested call to {@link KafkaConsumer}
*/
@ -144,12 +163,13 @@ public interface ConsumerRebalanceListener {
/**
* A callback method the user can implement to provide handling of customized offsets on completion of a successful
* partition re-assignment. This method will be called after the partition re-assignment completes and before the
* consumer starts fetching data, and only as the result of a {@link Consumer#poll(java.time.Duration) poll(long)} call.
* partition re-assignment. This method will be called after the partition re-assignment completes (even if no new
* partitions were assigned to the consumer), and before the consumer starts fetching data,
* and only as the result of a {@link Consumer#poll(java.time.Duration) poll(long)} call.
* <p>
* It is guaranteed that under normal conditions all the processes in a consumer group will execute their
* {@link #onPartitionsRevoked(Collection)} callback before any instance executes its
* {@link #onPartitionsAssigned(Collection)} callback. During exceptional scenarios, partitions may be migrated
* {@link #onPartitionsRevoked(Collection)} callback before any instance executes this onPartitionsAssigned callback.
* During exceptional scenarios, partitions may be migrated
* without the old owner being notified (i.e. their {@link #onPartitionsRevoked(Collection)} callback not triggered),
* and later when the old owner consumer realized this event, the {@link #onPartitionsLost(Collection)} callback
* will be triggered by the consumer then.
@ -160,9 +180,11 @@ public interface ConsumerRebalanceListener {
* invocation of {@link KafkaConsumer#poll(java.time.Duration)} in which this callback is being executed. This means it is not
* necessary to catch these exceptions and re-attempt to wakeup or interrupt the consumer thread.
*
* @param partitions The list of partitions that are now assigned to the consumer (previously owned partitions will
* NOT be included, i.e. this list will only include newly added partitions)
* @throws org.apache.kafka.common.errors.WakeupException If raised from a nested call to {@link KafkaConsumer}
* @param partitions Partitions that have been added to the assignment as a result of the rebalance.
* Note that partitions that were already owned by this consumer and remain assigned are not
* included in this list under the Classic/Cooperative or Consumer protocols. THe full assignment
* will be received under the Classic/Eager protocol.
* @throws org.apache.kafka.common.errors.WakeupException If raised from a nested call to {@link KafkaConsumer}
* @throws org.apache.kafka.common.errors.InterruptException If raised from a nested call to {@link KafkaConsumer}
*/
void onPartitionsAssigned(Collection<TopicPartition> partitions);
@ -187,10 +209,9 @@ public interface ConsumerRebalanceListener {
* necessary to catch these exceptions and re-attempt to wakeup or interrupt the consumer thread.
*
* @param partitions The list of partitions that were assigned to the consumer and now have been reassigned
* to other consumers. With the current protocol this will always include all of the consumer's
* previously assigned partitions, but this may change in future protocols (ie there would still
* be some partitions left)
* @throws org.apache.kafka.common.errors.WakeupException If raised from a nested call to {@link KafkaConsumer}
* to other consumers. With both, the Classic and Consumer protocols, this will always include
* all partitions that were previously assigned to the consumer.
* @throws org.apache.kafka.common.errors.WakeupException If raised from a nested call to {@link KafkaConsumer}
* @throws org.apache.kafka.common.errors.InterruptException If raised from a nested call to {@link KafkaConsumer}
*/
default void onPartitionsLost(Collection<TopicPartition> partitions) {

View File

@ -661,7 +661,7 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
* If the given list of topics is empty, it is treated the same as {@link #unsubscribe()}.
*
* <p>
* As part of group management, the consumer will keep track of the list of consumers that belong to a particular
* As part of group management, the group coordinator will keep track of the list of consumers that belong to a particular
* group and will trigger a rebalance operation if any one of the following events are triggered:
* <ul>
* <li>Number of partitions change for any of the subscribed topics
@ -670,8 +670,11 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
* <li>A new member is added to the consumer group
* </ul>
* <p>
* When any of these events are triggered, the provided listener will be invoked first to indicate that
* the consumer's assignment has been revoked, and then again when the new assignment has been received.
* When any of these events are triggered, the provided listener will be invoked in this way:
* <ul>
* <li>{@link ConsumerRebalanceListener#onPartitionsRevoked(Collection)} will be invoked with the partitions to revoke, before re-assigning those partitions to another consumer.</li>
* <li>{@link ConsumerRebalanceListener#onPartitionsAssigned(Collection)} will be invoked when the rebalance completes (even if no new partitions are assigned to the consumer)</li>
* </ul>
* Note that rebalances will only occur during an active call to {@link #poll(Duration)}, so callbacks will
* also only be invoked during that time.
*

View File

@ -988,7 +988,7 @@ public abstract class AbstractMembershipManager<R extends AbstractResponse> impl
String reason = rejoinedWhileReconciliationInProgress ?
"the member has re-joined the group" :
"the member already transitioned out of the reconciling state into " + state;
log.info("Interrupting reconciliation that is not relevant anymore because " + reason);
log.info("Interrupting reconciliation that is not relevant anymore because {}", reason);
markReconciliationCompleted();
}
return shouldAbort;

View File

@ -965,8 +965,8 @@ public abstract class AbstractStickyAssignor extends AbstractPartitionAssignor {
super(partitionsPerTopic, rackInfo, currentAssignment);
this.subscriptions = subscriptions;
topic2AllPotentialConsumers = new HashMap<>(partitionsPerTopic.keySet().size());
consumer2AllPotentialTopics = new HashMap<>(subscriptions.keySet().size());
topic2AllPotentialConsumers = new HashMap<>(partitionsPerTopic.size());
consumer2AllPotentialTopics = new HashMap<>(subscriptions.size());
// initialize topic2AllPotentialConsumers and consumer2AllPotentialTopics
partitionsPerTopic.keySet().forEach(

View File

@ -187,25 +187,6 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
*/
private class BackgroundEventProcessor implements EventProcessor<BackgroundEvent> {
private Optional<StreamsRebalanceListener> streamsRebalanceListener = Optional.empty();
private final Optional<StreamsRebalanceData> streamsRebalanceData;
public BackgroundEventProcessor() {
this.streamsRebalanceData = Optional.empty();
}
public BackgroundEventProcessor(final Optional<StreamsRebalanceData> streamsRebalanceData) {
this.streamsRebalanceData = streamsRebalanceData;
}
private void setStreamsRebalanceListener(final StreamsRebalanceListener streamsRebalanceListener) {
if (streamsRebalanceData.isEmpty()) {
throw new IllegalStateException("Background event processor was not created to be used with Streams " +
"rebalance protocol events");
}
this.streamsRebalanceListener = Optional.of(streamsRebalanceListener);
}
@Override
public void process(final BackgroundEvent event) {
switch (event.type()) {
@ -278,44 +259,26 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
private StreamsOnTasksRevokedCallbackCompletedEvent invokeOnTasksRevokedCallback(final Set<StreamsRebalanceData.TaskId> activeTasksToRevoke,
final CompletableFuture<Void> future) {
final Optional<Exception> exceptionFromCallback = streamsRebalanceListener().onTasksRevoked(activeTasksToRevoke);
final Optional<Exception> exceptionFromCallback = Optional.ofNullable(streamsRebalanceListenerInvoker().invokeTasksRevoked(activeTasksToRevoke));
final Optional<KafkaException> error = exceptionFromCallback.map(e -> ConsumerUtils.maybeWrapAsKafkaException(e, "Task revocation callback throws an error"));
return new StreamsOnTasksRevokedCallbackCompletedEvent(future, error);
}
private StreamsOnTasksAssignedCallbackCompletedEvent invokeOnTasksAssignedCallback(final StreamsRebalanceData.Assignment assignment,
final CompletableFuture<Void> future) {
final Optional<KafkaException> error;
final Optional<Exception> exceptionFromCallback = streamsRebalanceListener().onTasksAssigned(assignment);
if (exceptionFromCallback.isPresent()) {
error = Optional.of(ConsumerUtils.maybeWrapAsKafkaException(exceptionFromCallback.get(), "Task assignment callback throws an error"));
} else {
error = Optional.empty();
streamsRebalanceData().setReconciledAssignment(assignment);
}
final Optional<Exception> exceptionFromCallback = Optional.ofNullable(streamsRebalanceListenerInvoker().invokeTasksAssigned(assignment));
final Optional<KafkaException> error = exceptionFromCallback.map(e -> ConsumerUtils.maybeWrapAsKafkaException(e, "Task assignment callback throws an error"));
return new StreamsOnTasksAssignedCallbackCompletedEvent(future, error);
}
private StreamsOnAllTasksLostCallbackCompletedEvent invokeOnAllTasksLostCallback(final CompletableFuture<Void> future) {
final Optional<KafkaException> error;
final Optional<Exception> exceptionFromCallback = streamsRebalanceListener().onAllTasksLost();
if (exceptionFromCallback.isPresent()) {
error = Optional.of(ConsumerUtils.maybeWrapAsKafkaException(exceptionFromCallback.get(), "All tasks lost callback throws an error"));
} else {
error = Optional.empty();
streamsRebalanceData().setReconciledAssignment(StreamsRebalanceData.Assignment.EMPTY);
}
final Optional<Exception> exceptionFromCallback = Optional.ofNullable(streamsRebalanceListenerInvoker().invokeAllTasksLost());
final Optional<KafkaException> error = exceptionFromCallback.map(e -> ConsumerUtils.maybeWrapAsKafkaException(e, "All tasks lost callback throws an error"));
return new StreamsOnAllTasksLostCallbackCompletedEvent(future, error);
}
private StreamsRebalanceData streamsRebalanceData() {
return streamsRebalanceData.orElseThrow(
() -> new IllegalStateException("Background event processor was not created to be used with Streams " +
"rebalance protocol events"));
}
private StreamsRebalanceListener streamsRebalanceListener() {
return streamsRebalanceListener.orElseThrow(
private StreamsRebalanceListenerInvoker streamsRebalanceListenerInvoker() {
return streamsRebalanceListenerInvoker.orElseThrow(
() -> new IllegalStateException("Background event processor was not created to be used with Streams " +
"rebalance protocol events"));
}
@ -367,6 +330,7 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
private final WakeupTrigger wakeupTrigger = new WakeupTrigger();
private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker;
private final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker;
private final Optional<StreamsRebalanceListenerInvoker> streamsRebalanceListenerInvoker;
// Last triggered async commit future. Used to wait until all previous async commits are completed.
// We only need to keep track of the last one, since they are guaranteed to complete in order.
private CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> lastPendingAsyncCommit = null;
@ -517,7 +481,9 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
time,
new RebalanceCallbackMetricsManager(metrics)
);
this.backgroundEventProcessor = new BackgroundEventProcessor(streamsRebalanceData);
this.streamsRebalanceListenerInvoker = streamsRebalanceData.map(s ->
new StreamsRebalanceListenerInvoker(logContext, s));
this.backgroundEventProcessor = new BackgroundEventProcessor();
this.backgroundEventReaper = backgroundEventReaperFactory.build(logContext);
// The FetchCollector is only used on the application thread.
@ -577,6 +543,7 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
this.time = time;
this.backgroundEventQueue = backgroundEventQueue;
this.rebalanceListenerInvoker = rebalanceListenerInvoker;
this.streamsRebalanceListenerInvoker = Optional.empty();
this.backgroundEventProcessor = new BackgroundEventProcessor();
this.backgroundEventReaper = backgroundEventReaper;
this.metrics = metrics;
@ -699,6 +666,7 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
networkClientDelegateSupplier,
requestManagersSupplier,
asyncConsumerMetrics);
this.streamsRebalanceListenerInvoker = Optional.empty();
this.backgroundEventProcessor = new BackgroundEventProcessor();
this.backgroundEventReaper = new CompletableEventReaper(logContext);
}
@ -1477,7 +1445,7 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
() -> autoCommitOnClose(closeTimer), firstException);
swallow(log, Level.ERROR, "Failed to stop finding coordinator",
this::stopFindCoordinatorOnClose, firstException);
swallow(log, Level.ERROR, "Failed to release group assignment",
swallow(log, Level.ERROR, "Failed to run rebalance callbacks",
this::runRebalanceCallbacksOnClose, firstException);
swallow(log, Level.ERROR, "Failed to leave group while closing consumer",
() -> leaveGroupOnClose(closeTimer, membershipOperation), firstException);
@ -1527,26 +1495,39 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
}
private void runRebalanceCallbacksOnClose() {
if (groupMetadata.get().isEmpty() || rebalanceListenerInvoker == null)
if (groupMetadata.get().isEmpty())
return;
int memberEpoch = groupMetadata.get().get().generationId();
Set<TopicPartition> assignedPartitions = groupAssignmentSnapshot.get();
Exception error = null;
if (assignedPartitions.isEmpty())
// Nothing to revoke.
return;
if (streamsRebalanceListenerInvoker != null && streamsRebalanceListenerInvoker.isPresent()) {
SortedSet<TopicPartition> droppedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR);
droppedPartitions.addAll(assignedPartitions);
if (memberEpoch > 0) {
error = streamsRebalanceListenerInvoker.get().invokeAllTasksRevoked();
} else {
error = streamsRebalanceListenerInvoker.get().invokeAllTasksLost();
}
final Exception error;
} else if (rebalanceListenerInvoker != null) {
if (memberEpoch > 0)
error = rebalanceListenerInvoker.invokePartitionsRevoked(droppedPartitions);
else
error = rebalanceListenerInvoker.invokePartitionsLost(droppedPartitions);
Set<TopicPartition> assignedPartitions = groupAssignmentSnapshot.get();
if (assignedPartitions.isEmpty())
// Nothing to revoke.
return;
SortedSet<TopicPartition> droppedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR);
droppedPartitions.addAll(assignedPartitions);
if (memberEpoch > 0) {
error = rebalanceListenerInvoker.invokePartitionsRevoked(droppedPartitions);
} else {
error = rebalanceListenerInvoker.invokePartitionsLost(droppedPartitions);
}
}
if (error != null)
throw ConsumerUtils.maybeWrapAsKafkaException(error);
@ -1963,8 +1944,12 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
}
public void subscribe(Collection<String> topics, StreamsRebalanceListener streamsRebalanceListener) {
streamsRebalanceListenerInvoker
.orElseThrow(() -> new IllegalStateException("Consumer was not created to be used with Streams rebalance protocol events"))
.setRebalanceListener(streamsRebalanceListener);
subscribeInternal(topics, Optional.empty());
backgroundEventProcessor.setStreamsRebalanceListener(streamsRebalanceListener);
}
@Override

View File

@ -181,6 +181,14 @@ public class CommitRequestManager implements RequestManager, MemberStateListener
// poll when the coordinator node is known and fatal error is not present
if (coordinatorRequestManager.coordinator().isEmpty()) {
pendingRequests.maybeFailOnCoordinatorFatalError();
if (closing && pendingRequests.hasUnsentRequests()) {
CommitFailedException exception = new CommitFailedException(
"Failed to commit offsets: Coordinator unknown and consumer is closing");
pendingRequests.drainPendingCommits()
.forEach(request -> request.future().completeExceptionally(exception));
}
return EMPTY;
}
@ -541,7 +549,7 @@ public class CommitRequestManager implements RequestManager, MemberStateListener
boolean inflightRemoved = pendingRequests.inflightOffsetFetches.remove(fetchRequest);
if (!inflightRemoved) {
log.warn("A duplicated, inflight, request was identified, but unable to find it in the " +
"outbound buffer:" + fetchRequest);
"outbound buffer: {}", fetchRequest);
}
if (error == null) {
maybeUpdateLastSeenEpochIfNewer(res);

View File

@ -163,7 +163,7 @@ public class ConsumerMembershipManager extends AbstractMembershipManager<Consume
logContext,
backgroundEventHandler,
time,
new ConsumerRebalanceMetricsManager(metrics),
new ConsumerRebalanceMetricsManager(metrics, subscriptions),
autoCommitEnabled);
}

View File

@ -287,7 +287,7 @@ public class StreamsGroupHeartbeatRequestManager implements RequestManager {
private final HeartbeatMetricsManager metricsManager;
private StreamsRebalanceData streamsRebalanceData;
private final StreamsRebalanceData streamsRebalanceData;
/**
* Timer for tracking the time since the last consumer poll. If the timer expires, the consumer will stop

View File

@ -296,7 +296,7 @@ public class StreamsMembershipManager implements RequestManager {
this.backgroundEventHandler = backgroundEventHandler;
this.streamsRebalanceData = streamsRebalanceData;
this.subscriptionState = subscriptionState;
metricsManager = new ConsumerRebalanceMetricsManager(metrics);
metricsManager = new ConsumerRebalanceMetricsManager(metrics, subscriptionState);
this.time = time;
}
@ -1010,8 +1010,8 @@ public class StreamsMembershipManager implements RequestManager {
return;
}
if (reconciliationInProgress) {
log.trace("Ignoring reconciliation attempt. Another reconciliation is already in progress. Assignment " +
targetAssignment + " will be handled in the next reconciliation loop.");
log.trace("Ignoring reconciliation attempt. Another reconciliation is already in progress. Assignment {}" +
" will be handled in the next reconciliation loop.", targetAssignment);
return;
}
@ -1131,12 +1131,12 @@ public class StreamsMembershipManager implements RequestManager {
);
final SortedSet<TopicPartition> partitionsToAssign = topicPartitionsForActiveTasks(activeTasksToAssign);
final SortedSet<TopicPartition> partitionsToAssigneNotPreviouslyOwned =
final SortedSet<TopicPartition> partitionsToAssignNotPreviouslyOwned =
partitionsToAssignNotPreviouslyOwned(partitionsToAssign, topicPartitionsForActiveTasks(ownedActiveTasks));
subscriptionState.assignFromSubscribedAwaitingCallback(
partitionsToAssign,
partitionsToAssigneNotPreviouslyOwned
partitionsToAssignNotPreviouslyOwned
);
notifyAssignmentChange(partitionsToAssign);
@ -1152,10 +1152,10 @@ public class StreamsMembershipManager implements RequestManager {
if (callbackError == null) {
subscriptionState.enablePartitionsAwaitingCallback(partitionsToAssign);
} else {
if (!partitionsToAssigneNotPreviouslyOwned.isEmpty()) {
if (!partitionsToAssignNotPreviouslyOwned.isEmpty()) {
log.warn("Leaving newly assigned partitions {} marked as non-fetchable and not " +
"requiring initializing positions after onTasksAssigned callback failed.",
partitionsToAssigneNotPreviouslyOwned, callbackError);
partitionsToAssignNotPreviouslyOwned, callbackError);
}
}
});
@ -1205,9 +1205,9 @@ public class StreamsMembershipManager implements RequestManager {
Stream.concat(
streamsRebalanceData.subtopologies().get(task.subtopologyId()).sourceTopics().stream(),
streamsRebalanceData.subtopologies().get(task.subtopologyId()).repartitionSourceTopics().keySet().stream()
).forEach(topic -> {
topicPartitions.add(new TopicPartition(topic, task.partitionId()));
})
).forEach(topic ->
topicPartitions.add(new TopicPartition(topic, task.partitionId()))
)
);
return topicPartitions;
}
@ -1223,7 +1223,7 @@ public class StreamsMembershipManager implements RequestManager {
String reason = rejoinedWhileReconciliationInProgress ?
"the member has re-joined the group" :
"the member already transitioned out of the reconciling state into " + state;
log.info("Interrupting reconciliation that is not relevant anymore because " + reason);
log.info("Interrupting reconciliation that is not relevant anymore because {}", reason);
markReconciliationCompleted();
}
return shouldAbort;

View File

@ -16,7 +16,6 @@
*/
package org.apache.kafka.clients.consumer.internals;
import java.util.Optional;
import java.util.Set;
/**
@ -28,22 +27,18 @@ public interface StreamsRebalanceListener {
* Called when tasks are revoked from a stream thread.
*
* @param tasks The tasks to be revoked.
* @return The exception thrown during the callback, if any.
*/
Optional<Exception> onTasksRevoked(final Set<StreamsRebalanceData.TaskId> tasks);
void onTasksRevoked(final Set<StreamsRebalanceData.TaskId> tasks);
/**
* Called when tasks are assigned from a stream thread.
*
* @param assignment The tasks assigned.
* @return The exception thrown during the callback, if any.
*/
Optional<Exception> onTasksAssigned(final StreamsRebalanceData.Assignment assignment);
void onTasksAssigned(final StreamsRebalanceData.Assignment assignment);
/**
* Called when a stream thread loses all assigned tasks.
*
* @return The exception thrown during the callback, if any.
* Called when a stream thread loses all assigned tasks
*/
Optional<Exception> onAllTasksLost();
void onAllTasksLost();
}

View File

@ -0,0 +1,117 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.clients.consumer.internals;
import org.apache.kafka.common.errors.InterruptException;
import org.apache.kafka.common.errors.WakeupException;
import org.apache.kafka.common.utils.LogContext;
import org.slf4j.Logger;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
/**
* This class encapsulates the invocation of the callback methods defined in the {@link StreamsRebalanceListener}
* interface. When streams group task assignment changes, these methods are invoked. This class wraps those
* callback calls with some logging and error handling.
*/
public class StreamsRebalanceListenerInvoker {
private final Logger log;
private final StreamsRebalanceData streamsRebalanceData;
private Optional<StreamsRebalanceListener> listener;
StreamsRebalanceListenerInvoker(LogContext logContext, StreamsRebalanceData streamsRebalanceData) {
this.log = logContext.logger(getClass());
this.listener = Optional.empty();
this.streamsRebalanceData = streamsRebalanceData;
}
public void setRebalanceListener(StreamsRebalanceListener streamsRebalanceListener) {
Objects.requireNonNull(streamsRebalanceListener, "StreamsRebalanceListener cannot be null");
this.listener = Optional.of(streamsRebalanceListener);
}
public Exception invokeAllTasksRevoked() {
if (listener.isEmpty()) {
return null;
}
return invokeTasksRevoked(streamsRebalanceData.reconciledAssignment().activeTasks());
}
public Exception invokeTasksAssigned(final StreamsRebalanceData.Assignment assignment) {
if (listener.isEmpty()) {
return null;
}
log.info("Invoking tasks assigned callback for new assignment: {}", assignment);
try {
listener.get().onTasksAssigned(assignment);
} catch (WakeupException | InterruptException e) {
throw e;
} catch (Exception e) {
log.error(
"Streams rebalance listener failed on invocation of onTasksAssigned for tasks {}",
assignment,
e
);
return e;
}
return null;
}
public Exception invokeTasksRevoked(final Set<StreamsRebalanceData.TaskId> tasks) {
if (listener.isEmpty()) {
return null;
}
log.info("Invoking task revoked callback for revoked active tasks {}", tasks);
try {
listener.get().onTasksRevoked(tasks);
} catch (WakeupException | InterruptException e) {
throw e;
} catch (Exception e) {
log.error(
"Streams rebalance listener failed on invocation of onTasksRevoked for tasks {}",
tasks,
e
);
return e;
}
return null;
}
public Exception invokeAllTasksLost() {
if (listener.isEmpty()) {
return null;
}
log.info("Invoking tasks lost callback for all tasks");
try {
listener.get().onAllTasksLost();
} catch (WakeupException | InterruptException e) {
throw e;
} catch (Exception e) {
log.error(
"Streams rebalance listener failed on invocation of onTasksLost.",
e
);
return e;
}
return null;
}
}

View File

@ -477,7 +477,7 @@ public class SubscriptionState {
* Provides the number of assigned partitions in a thread safe manner.
* @return the number of assigned partitions.
*/
synchronized int numAssignedPartitions() {
public synchronized int numAssignedPartitions() {
return this.assignment.size();
}

View File

@ -16,6 +16,8 @@
*/
package org.apache.kafka.clients.consumer.internals.metrics;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.consumer.internals.SubscriptionState;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.metrics.Measurable;
import org.apache.kafka.common.metrics.Metrics;
@ -27,7 +29,9 @@ import org.apache.kafka.common.metrics.stats.Max;
import org.apache.kafka.common.metrics.stats.Rate;
import org.apache.kafka.common.metrics.stats.WindowedCount;
import java.util.Collection;
import java.util.concurrent.TimeUnit;
import java.util.regex.Pattern;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP_PREFIX;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.COORDINATOR_METRICS_SUFFIX;
@ -44,11 +48,14 @@ public final class ConsumerRebalanceMetricsManager extends RebalanceMetricsManag
public final MetricName lastRebalanceSecondsAgo;
public final MetricName failedRebalanceTotal;
public final MetricName failedRebalanceRate;
public final MetricName assignedPartitionsCount;
private long lastRebalanceEndMs = -1L;
private long lastRebalanceStartMs = -1L;
private final Metrics metrics;
public ConsumerRebalanceMetricsManager(Metrics metrics) {
public ConsumerRebalanceMetricsManager(Metrics metrics, SubscriptionState subscriptions) {
super(CONSUMER_METRIC_GROUP_PREFIX + COORDINATOR_METRICS_SUFFIX);
this.metrics = metrics;
rebalanceLatencyAvg = createMetric(metrics, "rebalance-latency-avg",
"The average time in ms taken for a group to complete a rebalance");
@ -64,6 +71,9 @@ public final class ConsumerRebalanceMetricsManager extends RebalanceMetricsManag
"The total number of failed rebalance events");
failedRebalanceRate = createMetric(metrics, "failed-rebalance-rate-per-hour",
"The number of failed rebalance events per hour");
assignedPartitionsCount = createMetric(metrics, "assigned-partitions",
"The number of partitions currently assigned to this consumer");
registerAssignedPartitionCount(subscriptions);
successfulRebalanceSensor = metrics.sensor("rebalance-latency");
successfulRebalanceSensor.add(rebalanceLatencyAvg, new Avg());
@ -106,4 +116,15 @@ public final class ConsumerRebalanceMetricsManager extends RebalanceMetricsManag
public boolean rebalanceStarted() {
return lastRebalanceStartMs > lastRebalanceEndMs;
}
/**
* Register metric to track the number of assigned partitions.
* It will consider partitions assigned to the consumer
* regardless of whether they were assigned via {@link KafkaConsumer#subscribe(Pattern)} or
* {@link KafkaConsumer#assign(Collection)}
*/
private void registerAssignedPartitionCount(SubscriptionState subscriptions) {
Measurable numParts = (config, now) -> subscriptions.numAssignedPartitions();
metrics.addMetric(assignedPartitionsCount, numParts);
}
}

View File

@ -1027,15 +1027,15 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
* expensive callbacks it is recommended to use your own {@link java.util.concurrent.Executor} in the callback body
* to parallelize processing.
*
* @param record The record to send
* @param record The record to send
* @param callback A user-supplied callback to execute when the record has been acknowledged by the server (null
* indicates no callback)
*
* @throws IllegalStateException if a transactional.id has been configured and no transaction has been started, or
* when send is invoked after producer has been closed.
* @throws InterruptException If the thread is interrupted while blocked
* indicates no callback)
* @throws IllegalStateException if a transactional.id has been configured and no transaction has been started, or
* when send is invoked after producer has been closed.
* @throws TimeoutException if the topic or the partition specified in the record cannot be found in metadata within {@code max.block.ms}
* @throws InterruptException If the thread is interrupted while blocked
* @throws SerializationException If the key or value are not valid objects given the configured serializers
* @throws KafkaException If a Kafka related error occurs that does not belong to the public API exceptions.
* @throws KafkaException If a Kafka related error occurs that does not belong to the public API exceptions.
*/
@Override
public Future<RecordMetadata> send(ProducerRecord<K, V> record, Callback callback) {
@ -1335,11 +1335,14 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
/**
* Get the partition metadata for the given topic. This can be used for custom partitioning.
* <p/>
* This will attempt to refresh metadata until it finds the topic in it, or the configured {@link ProducerConfig#MAX_BLOCK_MS_CONFIG} expires.
*
* @throws AuthenticationException if authentication fails. See the exception for more details
* @throws AuthorizationException if not authorized to the specified topic. See the exception for more details
* @throws InterruptException if the thread is interrupted while blocked
* @throws TimeoutException if metadata could not be refreshed within {@code max.block.ms}
* @throws KafkaException for all Kafka-related exceptions, including the case where this method is called after producer close
* @throws AuthorizationException if not authorized to the specified topic. See the exception for more details
* @throws InterruptException if the thread is interrupted while blocked
* @throws TimeoutException if the topic cannot be found in metadata within {@code max.block.ms}
* @throws KafkaException for all Kafka-related exceptions, including the case where this method is called after producer close
*/
@Override
public List<PartitionInfo> partitionsFor(String topic) {

View File

@ -16,10 +16,23 @@
*/
package org.apache.kafka.common.header;
/**
* A header is a key-value pair.
*/
public interface Header {
/**
* Returns the key of the header.
*
* @return the header's key; must not be null.
*/
String key();
/**
* Returns the value of the header.
*
* @return the header's value; may be null.
*/
byte[] value();
}

View File

@ -16,12 +16,18 @@
*/
package org.apache.kafka.common.header;
/**
* A mutable ordered collection of {@link Header} objects. Note that multiple headers may have the same {@link Header#key() key}.
* <p>
* The order of headers is preserved in the order they were added.
*/
public interface Headers extends Iterable<Header> {
/**
* Adds a header (key inside), to the end, returning if the operation succeeded.
*
* @param header the Header to be added
* @param header the Header to be added.
* @return this instance of the Headers, once the header is added.
* @throws IllegalStateException is thrown if headers are in a read-only state.
*/
@ -30,17 +36,18 @@ public interface Headers extends Iterable<Header> {
/**
* Creates and adds a header, to the end, returning if the operation succeeded.
*
* @param key of the header to be added.
* @param value of the header to be added.
* @param key of the header to be added; must not be null.
* @param value of the header to be added; may be null.
* @return this instance of the Headers, once the header is added.
* @throws IllegalStateException is thrown if headers are in a read-only state.
*/
Headers add(String key, byte[] value) throws IllegalStateException;
/**
* Removes all headers for the given key returning if the operation succeeded.
* Removes all headers for the given key returning if the operation succeeded,
* while preserving the insertion order of the remaining headers.
*
* @param key to remove all headers for.
* @param key to remove all headers for; must not be null.
* @return this instance of the Headers, once the header is removed.
* @throws IllegalStateException is thrown if headers are in a read-only state.
*/
@ -49,16 +56,17 @@ public interface Headers extends Iterable<Header> {
/**
* Returns just one (the very last) header for the given key, if present.
*
* @param key to get the last header for.
* @param key to get the last header for; must not be null.
* @return this last header matching the given key, returns null if not present.
*/
Header lastHeader(String key);
/**
* Returns all headers for the given key, in the order they were added in, if present.
* The iterator does not support {@link java.util.Iterator#remove()}.
*
* @param key to return the headers for.
* @return all headers for the given key, in the order they were added in, if NO headers are present an empty iterable is returned.
* @param key to return the headers for; must not be null.
* @return all headers for the given key, in the order they were added in, if NO headers are present an empty iterable is returned.
*/
Iterable<Header> headers(String key);

View File

@ -553,7 +553,7 @@ public final class Metrics implements Closeable {
try {
reporter.metricRemoval(metric);
} catch (Exception e) {
log.error("Error when removing metric from " + reporter.getClass().getName(), e);
log.error("Error when removing metric from {}", reporter.getClass().getName(), e);
}
}
log.trace("Removed metric named {}", metricName);
@ -596,7 +596,7 @@ public final class Metrics implements Closeable {
try {
reporter.metricChange(metric);
} catch (Exception e) {
log.error("Error when registering metric on " + reporter.getClass().getName(), e);
log.error("Error when registering metric on {}", reporter.getClass().getName(), e);
}
}
log.trace("Registered metric named {}", metricName);
@ -688,7 +688,7 @@ public final class Metrics implements Closeable {
log.info("Closing reporter {}", reporter.getClass().getName());
reporter.close();
} catch (Exception e) {
log.error("Error when closing " + reporter.getClass().getName(), e);
log.error("Error when closing {}", reporter.getClass().getName(), e);
}
}
log.info("Metrics reporters closed");

View File

@ -125,7 +125,7 @@ public class ChannelBuilders {
switch (securityProtocol) {
case SSL:
requireNonNullMode(connectionMode, securityProtocol);
channelBuilder = new SslChannelBuilder(connectionMode, listenerName, isInterBrokerListener, logContext);
channelBuilder = new SslChannelBuilder(connectionMode, listenerName, isInterBrokerListener);
break;
case SASL_SSL:
case SASL_PLAINTEXT:

View File

@ -26,7 +26,6 @@ import org.apache.kafka.common.security.auth.KafkaPrincipalSerde;
import org.apache.kafka.common.security.auth.SslAuthenticationContext;
import org.apache.kafka.common.security.ssl.SslFactory;
import org.apache.kafka.common.security.ssl.SslPrincipalMapper;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Utils;
import java.io.Closeable;
@ -53,8 +52,7 @@ public class SslChannelBuilder implements ChannelBuilder, ListenerReconfigurable
*/
public SslChannelBuilder(ConnectionMode connectionMode,
ListenerName listenerName,
boolean isInterBrokerListener,
LogContext logContext) {
boolean isInterBrokerListener) {
this.connectionMode = connectionMode;
this.listenerName = listenerName;
this.isInterBrokerListener = isInterBrokerListener;

View File

@ -19,6 +19,7 @@ package org.apache.kafka.common.protocol;
import org.apache.kafka.common.message.RequestHeaderData;
import org.apache.kafka.common.message.ResponseHeaderData;
import org.apache.kafka.common.protocol.types.BoundField;
import org.apache.kafka.common.protocol.types.Field;
import org.apache.kafka.common.protocol.types.Schema;
import org.apache.kafka.common.protocol.types.TaggedFields;
import org.apache.kafka.common.protocol.types.Type;
@ -27,6 +28,7 @@ import java.util.LinkedHashMap;
import java.util.LinkedHashSet;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
public class Protocol {
@ -49,7 +51,23 @@ public class Protocol {
subTypes.put(field.def.name, type.arrayElementType().get());
}
} else if (type instanceof TaggedFields) {
b.append("_tagged_fields ");
Map<Integer, Field> taggedFields = new TreeMap<>(((TaggedFields) type).fields());
taggedFields.forEach((tag, taggedField) -> {
if (taggedField.type.isArray()) {
b.append("[");
b.append(taggedField.name);
b.append("]");
if (!subTypes.containsKey(taggedField.name))
subTypes.put(taggedField.name + "&lt;tag: " + tag.toString() + "&gt;", taggedField.type.arrayElementType().get());
} else {
b.append(taggedField.name);
if (!subTypes.containsKey(taggedField.name))
subTypes.put(taggedField.name + "&lt;tag: " + tag.toString() + "&gt;", taggedField.type);
}
b.append("&lt;tag: ");
b.append(tag);
b.append("&gt; ");
});
} else {
b.append(field.def.name);
b.append(" ");
@ -90,6 +108,12 @@ public class Protocol {
}
}
private static void appendFieldNameToTable(String name, StringBuilder b) {
b.append("<td>");
b.append(name);
b.append("</td>");
}
private static void schemaToFieldTableHtml(Schema schema, StringBuilder b) {
Set<BoundField> fields = new LinkedHashSet<>();
populateSchemaFields(schema, fields);
@ -101,28 +125,12 @@ public class Protocol {
b.append("</tr>");
for (BoundField field : fields) {
b.append("<tr>\n");
b.append("<td>");
b.append(field.def.name);
b.append("</td>");
b.append("<td>");
if (field.def.type instanceof TaggedFields) {
TaggedFields taggedFields = (TaggedFields) field.def.type;
// Only include the field in the table if there are actually tags defined
if (taggedFields.numFields() > 0) {
b.append("<table class=\"data-table\"><tbody>\n");
b.append("<tr>");
b.append("<th>Tag</th>\n");
b.append("<th>Tagged field</th>\n");
b.append("<th>Description</th>\n");
b.append("</tr>");
taggedFields.fields().forEach((tag, taggedField) -> {
b.append("<tr>\n");
b.append("<td>");
b.append(tag);
b.append("</td>");
b.append("<td>");
b.append(taggedField.name);
b.append("</td>");
appendFieldNameToTable(taggedField.name + "&lt;tag: " + tag.toString() + "&gt;", b);
b.append("<td>");
b.append(taggedField.docString);
if (taggedField.type.isArray()) {
@ -136,11 +144,10 @@ public class Protocol {
b.append("</td>");
b.append("</tr>\n");
});
b.append("</tbody></table>\n");
} else {
b.append(field.def.docString);
}
} else {
appendFieldNameToTable(field.def.name, b);
b.append("<td>");
b.append(field.def.docString);
}
b.append("</td>");

View File

@ -25,6 +25,7 @@ import org.slf4j.LoggerFactory;
import java.io.InputStream;
import java.lang.management.ManagementFactory;
import java.util.Map;
import java.util.Properties;
import javax.management.JMException;
@ -68,7 +69,7 @@ public class AppInfoParser {
AppInfo mBean = new AppInfo(nowMs);
server.registerMBean(mBean, name);
registerMetrics(metrics, mBean); // prefix will be added later by JmxReporter
registerMetrics(metrics, mBean, id); // prefix will be added later by JmxReporter
} catch (JMException e) {
log.warn("Error registering AppInfo mbean", e);
}
@ -81,7 +82,7 @@ public class AppInfoParser {
if (server.isRegistered(name))
server.unregisterMBean(name);
unregisterMetrics(metrics);
unregisterMetrics(metrics, id);
} catch (JMException e) {
log.warn("Error unregistering AppInfo mbean", e);
} finally {
@ -89,23 +90,36 @@ public class AppInfoParser {
}
}
private static MetricName metricName(Metrics metrics, String name) {
return metrics.metricName(name, "app-info", "Metric indicating " + name);
private static MetricName metricName(Metrics metrics, String name, Map<String, String> tags) {
return metrics.metricName(name, "app-info", "Metric indicating " + name, tags);
}
private static void registerMetrics(Metrics metrics, AppInfo appInfo) {
if (metrics != null) {
metrics.addMetric(metricName(metrics, "version"), (Gauge<String>) (config, now) -> appInfo.getVersion());
metrics.addMetric(metricName(metrics, "commit-id"), (Gauge<String>) (config, now) -> appInfo.getCommitId());
metrics.addMetric(metricName(metrics, "start-time-ms"), (Gauge<Long>) (config, now) -> appInfo.getStartTimeMs());
private static void registerMetrics(Metrics metrics, AppInfo appInfo, String clientId) {
if (metrics == null) return;
// Most Kafka clients (producer/consumer/admin) set the client-id tag in the metrics config.
// Although we dont explicitly parse client-id here, these metrics are automatically tagged with client-id.
metrics.addMetric(metricName(metrics, "version", Map.of()), (Gauge<String>) (config, now) -> appInfo.getVersion());
metrics.addMetric(metricName(metrics, "commit-id", Map.of()), (Gauge<String>) (config, now) -> appInfo.getCommitId());
metrics.addMetric(metricName(metrics, "start-time-ms", Map.of()), (Gauge<Long>) (config, now) -> appInfo.getStartTimeMs());
// MirrorMaker/Worker doesn't set client-id tag into the metrics config, so we need to set it here.
if (!metrics.config().tags().containsKey("client-id") && clientId != null) {
metrics.addMetric(metricName(metrics, "version", Map.of("client-id", clientId)), (Gauge<String>) (config, now) -> appInfo.getVersion());
metrics.addMetric(metricName(metrics, "commit-id", Map.of("client-id", clientId)), (Gauge<String>) (config, now) -> appInfo.getCommitId());
metrics.addMetric(metricName(metrics, "start-time-ms", Map.of("client-id", clientId)), (Gauge<Long>) (config, now) -> appInfo.getStartTimeMs());
}
}
private static void unregisterMetrics(Metrics metrics) {
if (metrics != null) {
metrics.removeMetric(metricName(metrics, "version"));
metrics.removeMetric(metricName(metrics, "commit-id"));
metrics.removeMetric(metricName(metrics, "start-time-ms"));
private static void unregisterMetrics(Metrics metrics, String clientId) {
if (metrics == null) return;
metrics.removeMetric(metricName(metrics, "version", Map.of()));
metrics.removeMetric(metricName(metrics, "commit-id", Map.of()));
metrics.removeMetric(metricName(metrics, "start-time-ms", Map.of()));
if (!metrics.config().tags().containsKey("client-id") && clientId != null) {
metrics.removeMetric(metricName(metrics, "version", Map.of("client-id", clientId)));
metrics.removeMetric(metricName(metrics, "commit-id", Map.of("client-id", clientId)));
metrics.removeMetric(metricName(metrics, "start-time-ms", Map.of("client-id", clientId)));
}
}

View File

@ -40,7 +40,7 @@ public final class ByteBufferUnmapper {
private static final RuntimeException UNMAP_NOT_SUPPORTED_EXCEPTION;
static {
Object unmap = null;
MethodHandle unmap = null;
RuntimeException exception = null;
try {
unmap = lookupUnmapMethodHandle();
@ -48,7 +48,7 @@ public final class ByteBufferUnmapper {
exception = e;
}
if (unmap != null) {
UNMAP = (MethodHandle) unmap;
UNMAP = unmap;
UNMAP_NOT_SUPPORTED_EXCEPTION = null;
} else {
UNMAP = null;

View File

@ -75,7 +75,8 @@ public class ConfigUtils {
} else if (value instanceof String) {
return Boolean.parseBoolean((String) value);
} else {
log.error("Invalid value (" + value + ") on configuration '" + key + "'. The default value '" + defaultValue + "' will be used instead. Please specify a true/false value.");
log.error("Invalid value ({}) on configuration '{}'. The default value '{}' will be used instead. Please specify a true/false value.",
value, key, defaultValue);
return defaultValue;
}
}

View File

@ -69,7 +69,7 @@ public class LoggingSignalHandler {
for (String signal : SIGNALS) {
register(signal, jvmSignalHandlers);
}
log.info("Registered signal handlers for " + String.join(", ", SIGNALS));
log.info("Registered signal handlers for {}", String.join(", ", SIGNALS));
}
private Object createSignalHandler(final Map<String, Object> jvmSignalHandlers) {

View File

@ -33,7 +33,7 @@ import java.util.Map;
public class SecurityUtils {
private static final Logger LOGGER = LoggerFactory.getLogger(SecurityConfig.class);
private static final Logger LOGGER = LoggerFactory.getLogger(SecurityUtils.class);
private static final Map<String, ResourceType> NAME_TO_RESOURCE_TYPES;
private static final Map<String, AclOperation> NAME_TO_OPERATIONS;

View File

@ -857,7 +857,7 @@ public final class Utils {
public static void delete(final File rootFile) throws IOException {
if (rootFile == null)
return;
Files.walkFileTree(rootFile.toPath(), new SimpleFileVisitor<Path>() {
Files.walkFileTree(rootFile.toPath(), new SimpleFileVisitor<>() {
@Override
public FileVisitResult visitFileFailed(Path path, IOException exc) throws IOException {
if (exc instanceof NoSuchFileException) {
@ -1403,7 +1403,7 @@ public final class Utils {
* @return new Collector<Map.Entry<K, V>, M, M>
*/
public static <K, V, M extends Map<K, V>> Collector<Map.Entry<K, V>, M, M> entriesToMap(final Supplier<M> mapSupplier) {
return new Collector<Map.Entry<K, V>, M, M>() {
return new Collector<>() {
@Override
public Supplier<M> supplier() {
return mapSupplier;

View File

@ -30,7 +30,7 @@
{ "name": "ResourceTypeFilter", "type": "int8", "versions": "0+",
"about": "The resource type." },
{ "name": "ResourceNameFilter", "type": "string", "versions": "0+", "nullableVersions": "0+",
"about": "The resource name." },
"about": "The resource name, or null to match any resource name." },
{ "name": "PatternTypeFilter", "type": "int8", "versions": "1+", "default": "3", "ignorable": false,
"about": "The pattern type." },
{ "name": "PrincipalFilter", "type": "string", "versions": "0+", "nullableVersions": "0+",

View File

@ -33,7 +33,7 @@
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
"about": "The partition index." },
{ "name": "Offset", "type": "int64", "versions": "0+",
"about": "The deletion offset." }
"about": "The deletion offset. -1 means that records should be truncated to the high watermark." }
]}
]},
{ "name": "TimeoutMs", "type": "int32", "versions": "0+",

View File

@ -186,7 +186,7 @@ public class ClusterConnectionStatesTest {
connectionStates.authenticationFailed(nodeId1, time.milliseconds(), new AuthenticationException("No path to CA for certificate!"));
time.sleep(1000);
assertEquals(connectionStates.connectionState(nodeId1), ConnectionState.AUTHENTICATION_FAILED);
assertEquals(ConnectionState.AUTHENTICATION_FAILED, connectionStates.connectionState(nodeId1));
assertNotNull(connectionStates.authenticationException(nodeId1));
assertFalse(connectionStates.hasReadyNodes(time.milliseconds()));
assertFalse(connectionStates.canConnect(nodeId1, time.milliseconds()));
@ -210,7 +210,7 @@ public class ClusterConnectionStatesTest {
connectionStates.remove(nodeId1);
assertTrue(connectionStates.canConnect(nodeId1, time.milliseconds()));
assertFalse(connectionStates.isBlackedOut(nodeId1, time.milliseconds()));
assertEquals(connectionStates.connectionDelay(nodeId1, time.milliseconds()), 0L);
assertEquals(0L, connectionStates.connectionDelay(nodeId1, time.milliseconds()));
}
@Test

View File

@ -361,28 +361,28 @@ public class MetadataTest {
// Metadata with newer epoch is handled
metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), Collections.singletonMap("topic-1", 1), _tp -> 10);
metadata.updateWithCurrentRequestVersion(metadataResponse, false, 1L);
assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(leaderAndEpoch.intValue(), 10));
assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(10, leaderAndEpoch.intValue()));
// Don't update to an older one
assertFalse(metadata.updateLastSeenEpochIfNewer(tp, 1));
assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(leaderAndEpoch.intValue(), 10));
assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(10, leaderAndEpoch.intValue()));
// Don't cause update if it's the same one
assertFalse(metadata.updateLastSeenEpochIfNewer(tp, 10));
assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(leaderAndEpoch.intValue(), 10));
assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(10, leaderAndEpoch.intValue()));
// Update if we see newer epoch
assertTrue(metadata.updateLastSeenEpochIfNewer(tp, 12));
assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(leaderAndEpoch.intValue(), 12));
assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(12, leaderAndEpoch.intValue()));
metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), Collections.singletonMap("topic-1", 1), _tp -> 12);
metadata.updateWithCurrentRequestVersion(metadataResponse, false, 2L);
assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(leaderAndEpoch.intValue(), 12));
assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(12, leaderAndEpoch.intValue()));
// Don't overwrite metadata with older epoch
metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), Collections.singletonMap("topic-1", 1), _tp -> 11);
metadata.updateWithCurrentRequestVersion(metadataResponse, false, 3L);
assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(leaderAndEpoch.intValue(), 12));
assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(12, leaderAndEpoch.intValue()));
}
@Test
@ -465,7 +465,7 @@ public class MetadataTest {
metadata.updateWithCurrentRequestVersion(metadataResponse, false, 10L);
assertNotNull(metadata.fetch().partition(tp));
assertTrue(metadata.lastSeenLeaderEpoch(tp).isPresent());
assertEquals(metadata.lastSeenLeaderEpoch(tp).get().longValue(), 100);
assertEquals(100, metadata.lastSeenLeaderEpoch(tp).get().longValue());
}
// Fake an empty ISR, but with an older epoch, should reject it
@ -475,8 +475,8 @@ public class MetadataTest {
new MetadataResponse.PartitionMetadata(error, partition, leader,
leaderEpoch, replicas, Collections.emptyList(), offlineReplicas), ApiKeys.METADATA.latestVersion(), Collections.emptyMap());
metadata.updateWithCurrentRequestVersion(metadataResponse, false, 20L);
assertEquals(metadata.fetch().partition(tp).inSyncReplicas().length, 1);
assertEquals(metadata.lastSeenLeaderEpoch(tp).get().longValue(), 100);
assertEquals(1, metadata.fetch().partition(tp).inSyncReplicas().length);
assertEquals(100, metadata.lastSeenLeaderEpoch(tp).get().longValue());
}
// Fake an empty ISR, with same epoch, accept it
@ -486,8 +486,8 @@ public class MetadataTest {
new MetadataResponse.PartitionMetadata(error, partition, leader,
leaderEpoch, replicas, Collections.emptyList(), offlineReplicas), ApiKeys.METADATA.latestVersion(), Collections.emptyMap());
metadata.updateWithCurrentRequestVersion(metadataResponse, false, 20L);
assertEquals(metadata.fetch().partition(tp).inSyncReplicas().length, 0);
assertEquals(metadata.lastSeenLeaderEpoch(tp).get().longValue(), 100);
assertEquals(0, metadata.fetch().partition(tp).inSyncReplicas().length);
assertEquals(100, metadata.lastSeenLeaderEpoch(tp).get().longValue());
}
// Empty metadata response, should not keep old partition but should keep the last-seen epoch
@ -495,7 +495,7 @@ public class MetadataTest {
MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), Collections.emptyMap());
metadata.updateWithCurrentRequestVersion(metadataResponse, false, 20L);
assertNull(metadata.fetch().partition(tp));
assertEquals(metadata.lastSeenLeaderEpoch(tp).get().longValue(), 100);
assertEquals(100, metadata.lastSeenLeaderEpoch(tp).get().longValue());
}
// Back in the metadata, with old epoch, should not get added
@ -503,7 +503,7 @@ public class MetadataTest {
MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), partitionCounts, _tp -> 99);
metadata.updateWithCurrentRequestVersion(metadataResponse, false, 10L);
assertNull(metadata.fetch().partition(tp));
assertEquals(metadata.lastSeenLeaderEpoch(tp).get().longValue(), 100);
assertEquals(100, metadata.lastSeenLeaderEpoch(tp).get().longValue());
}
}
@ -522,31 +522,31 @@ public class MetadataTest {
metadata.updateWithCurrentRequestVersion(metadataResponse, false, 10L);
assertNotNull(metadata.fetch().partition(tp));
assertTrue(metadata.lastSeenLeaderEpoch(tp).isPresent());
assertEquals(metadata.lastSeenLeaderEpoch(tp).get().longValue(), 100);
assertEquals(100, metadata.lastSeenLeaderEpoch(tp).get().longValue());
// Simulate a leader epoch from another response, like a fetch response or list offsets
assertTrue(metadata.updateLastSeenEpochIfNewer(tp, 101));
// Cache of partition stays, but current partition info is not available since it's stale
assertNotNull(metadata.fetch().partition(tp));
assertEquals(Objects.requireNonNull(metadata.fetch().partitionCountForTopic("topic-1")).longValue(), 5);
assertEquals(5, Objects.requireNonNull(metadata.fetch().partitionCountForTopic("topic-1")).longValue());
assertFalse(metadata.partitionMetadataIfCurrent(tp).isPresent());
assertEquals(metadata.lastSeenLeaderEpoch(tp).get().longValue(), 101);
assertEquals(101, metadata.lastSeenLeaderEpoch(tp).get().longValue());
// Metadata with older epoch is rejected, metadata state is unchanged
metadata.updateWithCurrentRequestVersion(metadataResponse, false, 20L);
assertNotNull(metadata.fetch().partition(tp));
assertEquals(Objects.requireNonNull(metadata.fetch().partitionCountForTopic("topic-1")).longValue(), 5);
assertEquals(5, Objects.requireNonNull(metadata.fetch().partitionCountForTopic("topic-1")).longValue());
assertFalse(metadata.partitionMetadataIfCurrent(tp).isPresent());
assertEquals(metadata.lastSeenLeaderEpoch(tp).get().longValue(), 101);
assertEquals(101, metadata.lastSeenLeaderEpoch(tp).get().longValue());
// Metadata with equal or newer epoch is accepted
metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), partitionCounts, _tp -> 101);
metadata.updateWithCurrentRequestVersion(metadataResponse, false, 30L);
assertNotNull(metadata.fetch().partition(tp));
assertEquals(Objects.requireNonNull(metadata.fetch().partitionCountForTopic("topic-1")).longValue(), 5);
assertEquals(5, Objects.requireNonNull(metadata.fetch().partitionCountForTopic("topic-1")).longValue());
assertTrue(metadata.partitionMetadataIfCurrent(tp).isPresent());
assertEquals(metadata.lastSeenLeaderEpoch(tp).get().longValue(), 101);
assertEquals(101, metadata.lastSeenLeaderEpoch(tp).get().longValue());
}
@Test
@ -585,18 +585,18 @@ public class MetadataTest {
metadata.updateWithCurrentRequestVersion(metadataResponse, false, 0L);
Cluster cluster = metadata.fetch();
assertEquals(cluster.clusterResource().clusterId(), "dummy");
assertEquals(cluster.nodes().size(), 4);
assertEquals("dummy", cluster.clusterResource().clusterId());
assertEquals(4, cluster.nodes().size());
// topic counts
assertEquals(cluster.invalidTopics(), Collections.singleton("topic3"));
assertEquals(cluster.unauthorizedTopics(), Collections.singleton("topic4"));
assertEquals(cluster.topics().size(), 3);
assertEquals(3, cluster.topics().size());
assertEquals(cluster.internalTopics(), Collections.singleton(Topic.GROUP_METADATA_TOPIC_NAME));
// partition counts
assertEquals(cluster.partitionsForTopic("topic1").size(), 2);
assertEquals(cluster.partitionsForTopic("topic2").size(), 3);
assertEquals(2, cluster.partitionsForTopic("topic1").size());
assertEquals(3, cluster.partitionsForTopic("topic2").size());
// Sentinel instances
InetSocketAddress address = InetSocketAddress.createUnresolved("localhost", 0);
@ -798,10 +798,10 @@ public class MetadataTest {
TopicPartition tp = new TopicPartition("topic-1", 0);
assertOptional(metadata.fetch().nodeIfOnline(tp, 0), node -> assertEquals(node.id(), 0));
assertOptional(metadata.fetch().nodeIfOnline(tp, 0), node -> assertEquals(0, node.id()));
assertFalse(metadata.fetch().nodeIfOnline(tp, 1).isPresent());
assertEquals(metadata.fetch().nodeById(0).id(), 0);
assertEquals(metadata.fetch().nodeById(1).id(), 1);
assertEquals(0, metadata.fetch().nodeById(0).id());
assertEquals(1, metadata.fetch().nodeById(1).id());
}
@Test
@ -831,7 +831,7 @@ public class MetadataTest {
TopicPartition tp = new TopicPartition("topic-1", 0);
assertEquals(metadata.fetch().nodeById(0).id(), 0);
assertEquals(0, metadata.fetch().nodeById(0).id());
assertNull(metadata.fetch().partition(tp));
assertEquals(metadata.fetch().nodeIfOnline(tp, 0), Optional.empty());
}
@ -955,13 +955,13 @@ public class MetadataTest {
// Update the metadata to add a new topic variant, "new", which will be retained with "keep". Note this
// means that all of the "old" topics should be dropped.
Cluster cluster = metadata.fetch();
assertEquals(cluster.clusterResource().clusterId(), oldClusterId);
assertEquals(cluster.nodes().size(), oldNodes);
assertEquals(oldClusterId, cluster.clusterResource().clusterId());
assertEquals(oldNodes, cluster.nodes().size());
assertEquals(cluster.invalidTopics(), Set.of("oldInvalidTopic", "keepInvalidTopic"));
assertEquals(cluster.unauthorizedTopics(), Set.of("oldUnauthorizedTopic", "keepUnauthorizedTopic"));
assertEquals(cluster.topics(), Set.of("oldValidTopic", "keepValidTopic"));
assertEquals(cluster.partitionsForTopic("oldValidTopic").size(), 2);
assertEquals(cluster.partitionsForTopic("keepValidTopic").size(), 3);
assertEquals(2, cluster.partitionsForTopic("oldValidTopic").size());
assertEquals(3, cluster.partitionsForTopic("keepValidTopic").size());
assertEquals(new HashSet<>(cluster.topicIds()), new HashSet<>(topicIds.values()));
String newClusterId = "newClusterId";
@ -990,13 +990,13 @@ public class MetadataTest {
assertNull(metadataTopicIds2.get("oldValidTopic"));
cluster = metadata.fetch();
assertEquals(cluster.clusterResource().clusterId(), newClusterId);
assertEquals(newClusterId, cluster.clusterResource().clusterId());
assertEquals(cluster.nodes().size(), newNodes);
assertEquals(cluster.invalidTopics(), Set.of("keepInvalidTopic", "newInvalidTopic"));
assertEquals(cluster.unauthorizedTopics(), Set.of("keepUnauthorizedTopic", "newUnauthorizedTopic"));
assertEquals(cluster.topics(), Set.of("keepValidTopic", "newValidTopic"));
assertEquals(cluster.partitionsForTopic("keepValidTopic").size(), 2);
assertEquals(cluster.partitionsForTopic("newValidTopic").size(), 4);
assertEquals(2, cluster.partitionsForTopic("keepValidTopic").size());
assertEquals(4, cluster.partitionsForTopic("newValidTopic").size());
assertEquals(new HashSet<>(cluster.topicIds()), new HashSet<>(topicIds.values()));
// Perform another metadata update, but this time all topic metadata should be cleared.
@ -1008,7 +1008,7 @@ public class MetadataTest {
topicIds.forEach((topicName, topicId) -> assertNull(metadataTopicIds3.get(topicName)));
cluster = metadata.fetch();
assertEquals(cluster.clusterResource().clusterId(), newClusterId);
assertEquals(newClusterId, cluster.clusterResource().clusterId());
assertEquals(cluster.nodes().size(), newNodes);
assertEquals(cluster.invalidTopics(), Collections.emptySet());
assertEquals(cluster.unauthorizedTopics(), Collections.emptySet());

View File

@ -64,7 +64,7 @@ public class ConfigTest {
assertEquals(config, config);
assertEquals(config, new Config(config.entries()));
assertNotEquals(new Config(Collections.singletonList(E1)), config);
assertNotEquals(config, "this");
assertNotEquals("this", config);
}
@Test

View File

@ -81,7 +81,7 @@ public class CooperativeStickyAssignorTest extends AbstractStickyAssignorTest {
Optional<Integer> encodedGeneration = ((CooperativeStickyAssignor) assignor).memberData(subscription).generation;
assertTrue(encodedGeneration.isPresent());
assertEquals(encodedGeneration.get(), DEFAULT_GENERATION);
assertEquals(DEFAULT_GENERATION, encodedGeneration.get());
int generation = 10;
assignor.onAssignment(null, new ConsumerGroupMetadata("dummy-group-id", generation, "dummy-member-id", Optional.empty()));
@ -90,7 +90,7 @@ public class CooperativeStickyAssignorTest extends AbstractStickyAssignorTest {
encodedGeneration = ((CooperativeStickyAssignor) assignor).memberData(subscription).generation;
assertTrue(encodedGeneration.isPresent());
assertEquals(encodedGeneration.get(), generation);
assertEquals(generation, encodedGeneration.get());
}
@Test

View File

@ -155,6 +155,8 @@ import javax.management.ObjectName;
import static java.util.Collections.singletonList;
import static org.apache.kafka.clients.consumer.internals.ClassicKafkaConsumer.DEFAULT_REASON;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP_PREFIX;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.COORDINATOR_METRICS_SUFFIX;
import static org.apache.kafka.common.requests.FetchMetadata.INVALID_SESSION_ID;
import static org.apache.kafka.common.utils.Utils.propsToMap;
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
@ -271,6 +273,35 @@ public class KafkaConsumerTest {
}
}
@ParameterizedTest
@EnumSource(GroupProtocol.class)
public void testAssignedPartitionsMetrics(GroupProtocol groupProtocol) throws InterruptedException {
consumer = newConsumer(groupProtocol, time, mock(KafkaClient.class), subscription,
mock(ConsumerMetadata.class), assignor, false, groupInstanceId);
Metrics metrics = consumer.metricsRegistry();
// This metric is added in the background thread for the AsyncConsumer, so waiting on it to avoid flakiness.
TestUtils.waitForCondition(() -> getMetric(metrics, "assigned-partitions") != null,
"Consumer should register the assigned-partitions metric");
assertNotNull(getMetric(metrics, "assigned-partitions"));
assertEquals(0.0d, getMetric(metrics, "assigned-partitions").metricValue());
subscription.assignFromUser(Set.of(tp0));
assertEquals(1.0d, getMetric(metrics, "assigned-partitions").metricValue());
subscription.assignFromUser(Set.of(tp0, tp1));
assertEquals(2.0d, getMetric(metrics, "assigned-partitions").metricValue());
subscription.unsubscribe();
subscription.subscribe(Set.of(topic), Optional.empty());
subscription.assignFromSubscribed(Set.of(tp0));
assertEquals(1.0d, getMetric(metrics, "assigned-partitions").metricValue());
}
private KafkaMetric getMetric(Metrics metrics, String name) {
return metrics.metrics().get(metrics.metricName(name, CONSUMER_METRIC_GROUP_PREFIX + COORDINATOR_METRICS_SUFFIX));
}
@ParameterizedTest
@EnumSource(GroupProtocol.class)
public void testUnsubscribingCustomMetricsWithSameNameDoesntAffectConsumerMetrics(GroupProtocol groupProtocol) {

View File

@ -209,9 +209,7 @@ public class MockConsumerTest {
consumer.assign(Collections.singleton(partition));
consumer.updateBeginningOffsets(Collections.singletonMap(partition, 0L));
IntStream.range(0, 10).forEach(offset -> {
consumer.addRecord(new ConsumerRecord<>("test", 0, offset, null, null));
});
IntStream.range(0, 10).forEach(offset -> consumer.addRecord(new ConsumerRecord<>("test", 0, offset, null, null)));
consumer.setMaxPollRecords(2L);

View File

@ -1025,7 +1025,7 @@ public abstract class AbstractStickyAssignorTest {
Map<String, List<TopicPartition>> assignment = assignor.assignPartitions(partitionsPerTopic, subscriptions);
assertTrue(assignor.partitionsTransferringOwnership.isEmpty());
assertEquals(assignment.values().stream().mapToInt(List::size).sum(), 1 + 100);
assertEquals(1 + 100, assignment.values().stream().mapToInt(List::size).sum());
assertEquals(Collections.singleton(consumerId), assignment.keySet());
assertTrue(isFullyBalanced(assignment));
}
@ -1043,7 +1043,7 @@ public abstract class AbstractStickyAssignorTest {
assignment = assignor.assign(Collections.emptyMap(), subscriptions);
assertTrue(assignor.partitionsTransferringOwnership.isEmpty());
assertEquals(assignment.size(), 1);
assertEquals(1, assignment.size());
assertTrue(assignment.get(consumerId).isEmpty());
}

View File

@ -2210,6 +2210,71 @@ public class AsyncKafkaConsumerTest {
}).when(applicationEventHandler).add(ArgumentMatchers.isA(CommitEvent.class));
}
@Test
public void testCloseInvokesStreamsRebalanceListenerOnTasksRevokedWhenMemberEpochPositive() {
final String groupId = "streamsGroup";
final StreamsRebalanceData streamsRebalanceData = new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of());
try (final MockedStatic<RequestManagers> requestManagers = mockStatic(RequestManagers.class)) {
consumer = newConsumerWithStreamRebalanceData(requiredConsumerConfigAndGroupId(groupId), streamsRebalanceData);
StreamsRebalanceListener mockStreamsListener = mock(StreamsRebalanceListener.class);
consumer.subscribe(singletonList("topic"), mockStreamsListener);
final MemberStateListener groupMetadataUpdateListener = captureGroupMetadataUpdateListener(requestManagers);
final int memberEpoch = 42;
final String memberId = "memberId";
groupMetadataUpdateListener.onMemberEpochUpdated(Optional.of(memberEpoch), memberId);
consumer.close(CloseOptions.timeout(Duration.ZERO));
verify(mockStreamsListener).onTasksRevoked(any());
}
}
@Test
public void testCloseInvokesStreamsRebalanceListenerOnAllTasksLostWhenMemberEpochZeroOrNegative() {
final String groupId = "streamsGroup";
final StreamsRebalanceData streamsRebalanceData = new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of());
try (final MockedStatic<RequestManagers> requestManagers = mockStatic(RequestManagers.class)) {
consumer = newConsumerWithStreamRebalanceData(requiredConsumerConfigAndGroupId(groupId), streamsRebalanceData);
StreamsRebalanceListener mockStreamsListener = mock(StreamsRebalanceListener.class);
consumer.subscribe(singletonList("topic"), mockStreamsListener);
final MemberStateListener groupMetadataUpdateListener = captureGroupMetadataUpdateListener(requestManagers);
final int memberEpoch = 0;
final String memberId = "memberId";
groupMetadataUpdateListener.onMemberEpochUpdated(Optional.of(memberEpoch), memberId);
consumer.close(CloseOptions.timeout(Duration.ZERO));
verify(mockStreamsListener).onAllTasksLost();
}
}
@Test
public void testCloseWrapsStreamsRebalanceListenerException() {
final String groupId = "streamsGroup";
final StreamsRebalanceData streamsRebalanceData = new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of());
try (final MockedStatic<RequestManagers> requestManagers = mockStatic(RequestManagers.class)) {
consumer = newConsumerWithStreamRebalanceData(requiredConsumerConfigAndGroupId(groupId), streamsRebalanceData);
StreamsRebalanceListener mockStreamsListener = mock(StreamsRebalanceListener.class);
RuntimeException testException = new RuntimeException("Test streams listener exception");
doThrow(testException).when(mockStreamsListener).onTasksRevoked(any());
consumer.subscribe(singletonList("topic"), mockStreamsListener);
final MemberStateListener groupMetadataUpdateListener = captureGroupMetadataUpdateListener(requestManagers);
final int memberEpoch = 1;
final String memberId = "memberId";
groupMetadataUpdateListener.onMemberEpochUpdated(Optional.of(memberEpoch), memberId);
KafkaException thrownException = assertThrows(KafkaException.class,
() -> consumer.close(CloseOptions.timeout(Duration.ZERO)));
assertInstanceOf(RuntimeException.class, thrownException.getCause());
assertTrue(thrownException.getCause().getMessage().contains("Test streams listener exception"));
verify(mockStreamsListener).onTasksRevoked(any());
}
}
private void markReconcileAndAutoCommitCompleteForPollEvent() {
doAnswer(invocation -> {
PollEvent event = invocation.getArgument(0);

View File

@ -1494,6 +1494,47 @@ public class CommitRequestManagerTest {
assertTrue(commitRequestManager.pendingRequests.unsentOffsetCommits.isEmpty());
}
@Test
public void testPollWithFatalErrorDuringCoordinatorIsEmptyAndClosing() {
CommitRequestManager commitRequestManager = create(true, 100);
Map<TopicPartition, OffsetAndMetadata> offsets = Map.of(new TopicPartition("topic", 1),
new OffsetAndMetadata(0));
var commitFuture = commitRequestManager.commitAsync(offsets);
commitRequestManager.signalClose();
when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty());
when(coordinatorRequestManager.fatalError())
.thenReturn(Optional.of(new GroupAuthorizationException("Fatal error")));
assertEquals(NetworkClientDelegate.PollResult.EMPTY, commitRequestManager.poll(time.milliseconds()));
assertTrue(commitFuture.isCompletedExceptionally());
TestUtils.assertFutureThrows(GroupAuthorizationException.class, commitFuture, "Fatal error");
}
@Test
public void testPollWithClosingAndPendingRequests() {
CommitRequestManager commitRequestManager = create(true, 100);
Map<TopicPartition, OffsetAndMetadata> offsets = Map.of(new TopicPartition("topic", 1),
new OffsetAndMetadata(0));
var commitFuture = commitRequestManager.commitAsync(offsets);
commitRequestManager.signalClose();
when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty());
assertEquals(NetworkClientDelegate.PollResult.EMPTY, commitRequestManager.poll(time.milliseconds()));
assertTrue(commitFuture.isCompletedExceptionally());
TestUtils.assertFutureThrows(CommitFailedException.class, commitFuture,
"Failed to commit offsets: Coordinator unknown and consumer is closing");
}
// Supplies (error, isRetriable)
private static Stream<Arguments> partitionDataErrorSupplier() {
return Stream.of(

View File

@ -568,13 +568,13 @@ public abstract class ConsumerCoordinatorTest {
assertFalse(client.hasInFlightRequests());
// should try to find coordinator since we are commit async
coordinator.commitOffsetsAsync(singletonMap(t1p, new OffsetAndMetadata(100L)), (offsets, exception) -> {
fail("Commit should not get responses, but got offsets:" + offsets + ", and exception:" + exception);
});
coordinator.commitOffsetsAsync(singletonMap(t1p, new OffsetAndMetadata(100L)), (offsets, exception) ->
fail("Commit should not get responses, but got offsets:" + offsets + ", and exception:" + exception)
);
coordinator.poll(time.timer(0));
assertTrue(coordinator.coordinatorUnknown());
assertTrue(client.hasInFlightRequests());
assertEquals(coordinator.inFlightAsyncCommits.get(), 0);
assertEquals(0, coordinator.inFlightAsyncCommits.get());
client.respond(groupCoordinatorResponse(node, Errors.NONE));
coordinator.poll(time.timer(0));
@ -582,7 +582,7 @@ public abstract class ConsumerCoordinatorTest {
// after we've discovered the coordinator we should send
// out the commit request immediately
assertTrue(client.hasInFlightRequests());
assertEquals(coordinator.inFlightAsyncCommits.get(), 1);
assertEquals(1, coordinator.inFlightAsyncCommits.get());
}
@Test
@ -619,13 +619,13 @@ public abstract class ConsumerCoordinatorTest {
assertFalse(coordinator.commitOffsetsSync(Collections.emptyMap(), time.timer(100L)), "expected sync commit to fail");
assertFalse(committed.get());
assertEquals(coordinator.inFlightAsyncCommits.get(), 1);
assertEquals(1, coordinator.inFlightAsyncCommits.get());
prepareOffsetCommitRequest(singletonMap(tp, 123L), Errors.NONE);
assertTrue(coordinator.commitOffsetsSync(Collections.emptyMap(), time.timer(Long.MAX_VALUE)), "expected sync commit to succeed");
assertTrue(committed.get(), "expected commit callback to be invoked");
assertEquals(coordinator.inFlightAsyncCommits.get(), 0);
assertEquals(0, coordinator.inFlightAsyncCommits.get());
}
@Test
@ -646,13 +646,13 @@ public abstract class ConsumerCoordinatorTest {
"Unexpected exception cause type: " + (cause == null ? null : cause.getClass()));
});
}
assertEquals(coordinator.inFlightAsyncCommits.get(), numRequests);
assertEquals(numRequests, coordinator.inFlightAsyncCommits.get());
coordinator.markCoordinatorUnknown("test cause");
consumerClient.pollNoWakeup();
coordinator.invokeCompletedOffsetCommitCallbacks();
assertEquals(numRequests, responses.get());
assertEquals(coordinator.inFlightAsyncCommits.get(), 0);
assertEquals(0, coordinator.inFlightAsyncCommits.get());
}
@Test
@ -697,7 +697,7 @@ public abstract class ConsumerCoordinatorTest {
coordinator.markCoordinatorUnknown("test cause");
consumerClient.pollNoWakeup();
assertTrue(asyncCallbackInvoked.get());
assertEquals(coordinator.inFlightAsyncCommits.get(), 0);
assertEquals(0, coordinator.inFlightAsyncCommits.get());
}
@Test
@ -2350,7 +2350,7 @@ public abstract class ConsumerCoordinatorTest {
MockCommitCallback secondCommitCallback = new MockCommitCallback();
coordinator.commitOffsetsAsync(singletonMap(t1p, new OffsetAndMetadata(100L)), firstCommitCallback);
coordinator.commitOffsetsAsync(singletonMap(t1p, new OffsetAndMetadata(100L)), secondCommitCallback);
assertEquals(coordinator.inFlightAsyncCommits.get(), 2);
assertEquals(2, coordinator.inFlightAsyncCommits.get());
respondToOffsetCommitRequest(singletonMap(t1p, 100L), error);
consumerClient.pollNoWakeup();
@ -2360,7 +2360,7 @@ public abstract class ConsumerCoordinatorTest {
assertTrue(coordinator.coordinatorUnknown());
assertInstanceOf(RetriableCommitFailedException.class, firstCommitCallback.exception);
assertInstanceOf(RetriableCommitFailedException.class, secondCommitCallback.exception);
assertEquals(coordinator.inFlightAsyncCommits.get(), 0);
assertEquals(0, coordinator.inFlightAsyncCommits.get());
}
@Test
@ -2549,7 +2549,7 @@ public abstract class ConsumerCoordinatorTest {
coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.NONE);
coordinator.commitOffsetsAsync(singletonMap(t1p, new OffsetAndMetadata(100L)), mockOffsetCommitCallback);
assertEquals(coordinator.inFlightAsyncCommits.get(), 0);
assertEquals(0, coordinator.inFlightAsyncCommits.get());
coordinator.invokeCompletedOffsetCommitCallbacks();
assertEquals(invokedBeforeTest + 1, mockOffsetCommitCallback.invoked);
assertNull(mockOffsetCommitCallback.exception);
@ -2580,7 +2580,7 @@ public abstract class ConsumerCoordinatorTest {
coordinator.commitOffsetsAsync(singletonMap(t1p, new OffsetAndMetadata(100L)), callback(success));
coordinator.invokeCompletedOffsetCommitCallbacks();
assertTrue(success.get());
assertEquals(coordinator.inFlightAsyncCommits.get(), 0);
assertEquals(0, coordinator.inFlightAsyncCommits.get());
}
@Test
@ -2590,7 +2590,7 @@ public abstract class ConsumerCoordinatorTest {
coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.COORDINATOR_NOT_AVAILABLE);
coordinator.commitOffsetsAsync(singletonMap(t1p, new OffsetAndMetadata(100L)), mockOffsetCommitCallback);
assertEquals(coordinator.inFlightAsyncCommits.get(), 0);
assertEquals(0, coordinator.inFlightAsyncCommits.get());
coordinator.invokeCompletedOffsetCommitCallbacks();
assertEquals(invokedBeforeTest + 1, mockOffsetCommitCallback.invoked);
assertInstanceOf(RetriableCommitFailedException.class, mockOffsetCommitCallback.exception);
@ -2605,7 +2605,7 @@ public abstract class ConsumerCoordinatorTest {
MockCommitCallback cb = new MockCommitCallback();
prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.COORDINATOR_NOT_AVAILABLE);
coordinator.commitOffsetsAsync(singletonMap(t1p, new OffsetAndMetadata(100L)), cb);
assertEquals(coordinator.inFlightAsyncCommits.get(), 0);
assertEquals(0, coordinator.inFlightAsyncCommits.get());
coordinator.invokeCompletedOffsetCommitCallbacks();
assertTrue(coordinator.coordinatorUnknown());
@ -2622,7 +2622,7 @@ public abstract class ConsumerCoordinatorTest {
MockCommitCallback cb = new MockCommitCallback();
prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.NOT_COORDINATOR);
coordinator.commitOffsetsAsync(singletonMap(t1p, new OffsetAndMetadata(100L)), cb);
assertEquals(coordinator.inFlightAsyncCommits.get(), 0);
assertEquals(0, coordinator.inFlightAsyncCommits.get());
coordinator.invokeCompletedOffsetCommitCallbacks();
assertTrue(coordinator.coordinatorUnknown());
@ -2639,7 +2639,7 @@ public abstract class ConsumerCoordinatorTest {
MockCommitCallback cb = new MockCommitCallback();
prepareOffsetCommitRequestDisconnect(singletonMap(t1p, 100L));
coordinator.commitOffsetsAsync(singletonMap(t1p, new OffsetAndMetadata(100L)), cb);
assertEquals(coordinator.inFlightAsyncCommits.get(), 0);
assertEquals(0, coordinator.inFlightAsyncCommits.get());
coordinator.invokeCompletedOffsetCommitCallbacks();
assertTrue(coordinator.coordinatorUnknown());
@ -2703,7 +2703,7 @@ public abstract class ConsumerCoordinatorTest {
}
};
assertEquals(coordinator.inFlightAsyncCommits.get(), 1);
assertEquals(1, coordinator.inFlightAsyncCommits.get());
thread.start();
client.waitForRequests(2, 5000);
@ -2711,7 +2711,7 @@ public abstract class ConsumerCoordinatorTest {
respondToOffsetCommitRequest(singletonMap(t1p, secondOffset.offset()), Errors.NONE);
thread.join();
assertEquals(coordinator.inFlightAsyncCommits.get(), 0);
assertEquals(0, coordinator.inFlightAsyncCommits.get());
assertEquals(Arrays.asList(firstOffset, secondOffset), committedOffsets);
}
@ -3100,7 +3100,7 @@ public abstract class ConsumerCoordinatorTest {
assertEquals(Collections.emptySet(), subscriptions.initializingPartitions());
assertFalse(subscriptions.hasAllFetchPositions());
assertTrue(subscriptions.awaitingValidation(t1p));
assertEquals(subscriptions.position(t1p).offset, 100L);
assertEquals(100L, subscriptions.position(t1p).offset);
assertNull(subscriptions.validPosition(t1p));
}
@ -3470,7 +3470,7 @@ public abstract class ConsumerCoordinatorTest {
assertThrows(FencedInstanceIdException.class, this::receiveFencedInstanceIdException);
assertThrows(FencedInstanceIdException.class, () ->
coordinator.commitOffsetsAsync(singletonMap(t1p, new OffsetAndMetadata(100L)), new MockCommitCallback()));
assertEquals(coordinator.inFlightAsyncCommits.get(), 0);
assertEquals(0, coordinator.inFlightAsyncCommits.get());
assertThrows(FencedInstanceIdException.class, () ->
coordinator.commitOffsetsSync(singletonMap(t1p, new OffsetAndMetadata(100L)), time.timer(Long.MAX_VALUE)));
}
@ -3739,7 +3739,7 @@ public abstract class ConsumerCoordinatorTest {
prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.FENCED_INSTANCE_ID);
coordinator.commitOffsetsAsync(singletonMap(t1p, new OffsetAndMetadata(100L)), new MockCommitCallback());
assertEquals(coordinator.inFlightAsyncCommits.get(), 0);
assertEquals(0, coordinator.inFlightAsyncCommits.get());
coordinator.invokeCompletedOffsetCommitCallbacks();
}

View File

@ -71,6 +71,8 @@ import java.util.stream.Stream;
import static org.apache.kafka.clients.consumer.internals.AbstractMembershipManager.TOPIC_PARTITION_COMPARATOR;
import static org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer.invokeRebalanceCallbacks;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP_PREFIX;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.COORDINATOR_METRICS_SUFFIX;
import static org.apache.kafka.common.requests.ConsumerGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH;
import static org.apache.kafka.common.utils.Utils.mkEntry;
import static org.apache.kafka.common.utils.Utils.mkMap;
@ -125,7 +127,7 @@ public class ConsumerMembershipManagerTest {
time = new MockTime(0);
backgroundEventHandler = new BackgroundEventHandler(backgroundEventQueue, time, mock(AsyncConsumerMetrics.class));
metrics = new Metrics(time);
rebalanceMetricsManager = new ConsumerRebalanceMetricsManager(metrics);
rebalanceMetricsManager = new ConsumerRebalanceMetricsManager(metrics, subscriptionState);
when(commitRequestManager.maybeAutoCommitSyncBeforeRebalance(anyLong())).thenReturn(CompletableFuture.completedFuture(null));
}
@ -181,6 +183,15 @@ public class ConsumerMembershipManagerTest {
assertEquals(Optional.of("rack1"), membershipManager.rackId());
}
@Test
public void testAssignedPartitionCountMetricRegistered() {
MetricName metricName = metrics.metricName(
"assigned-partitions",
CONSUMER_METRIC_GROUP_PREFIX + COORDINATOR_METRICS_SUFFIX
);
assertNotNull(metrics.metric(metricName), "Metric assigned-partitions should have been registered");
}
@Test
public void testMembershipManagerInitSupportsEmptyGroupInstanceId() {
createMembershipManagerJoiningGroup();

View File

@ -41,7 +41,6 @@ import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.test.TestUtils;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Test;
import java.time.Duration;
@ -266,45 +265,34 @@ public class ConsumerNetworkClientTest {
assertEquals(metadataException, exc);
}
@Disabled("KAFKA-17554")
@Test
public void testFutureCompletionOutsidePoll() throws Exception {
// Tests the scenario in which the request that is being awaited in one thread
// is received and completed in another thread.
final CountDownLatch t1TheardCountDownLatch = new CountDownLatch(1);
final CountDownLatch t2ThreadCountDownLatch = new CountDownLatch(2);
final RequestFuture<ClientResponse> future = consumerClient.send(node, heartbeat());
consumerClient.pollNoWakeup(); // dequeue and send the request
CountDownLatch bothThreadsReady = new CountDownLatch(2);
client.enableBlockingUntilWakeup(2);
Thread t1 = new Thread(() -> {
t1TheardCountDownLatch.countDown();
Thread t1 = new Thread(() -> {
bothThreadsReady.countDown();
consumerClient.pollNoWakeup();
t2ThreadCountDownLatch.countDown();
});
t1.start();
Thread t2 = new Thread(() -> {
try {
t2ThreadCountDownLatch.await();
consumerClient.poll(future);
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
bothThreadsReady.countDown();
consumerClient.poll(future);
});
t1.start();
t2.start();
// Simulate a network response and return from the poll in t1
// Wait until both threads are blocked in poll
bothThreadsReady.await();
client.respond(heartbeatResponse(Errors.NONE));
// Wait for t1 to block in poll
t1TheardCountDownLatch.await();
client.wakeup();
// while t1 is blocked in poll, t2 should be able to complete the future
t2ThreadCountDownLatch.countDown();
// Both threads should complete since t1 should wakeup t2
t1.join();

View File

@ -1815,7 +1815,7 @@ public class FetchRequestManagerTest {
assertEquals(1, oorExceptions.size());
OffsetOutOfRangeException oor = oorExceptions.get(0);
assertTrue(oor.offsetOutOfRangePartitions().containsKey(tp0));
assertEquals(oor.offsetOutOfRangePartitions().size(), 1);
assertEquals(1, oor.offsetOutOfRangePartitions().size());
fetchRecordsInto(fetchedRecords);
@ -2359,7 +2359,7 @@ public class FetchRequestManagerTest {
Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> fetchedRecords = fetchRecords();
assertTrue(fetchedRecords.containsKey(tp0));
assertEquals(fetchedRecords.get(tp0).size(), 2);
assertEquals(2, fetchedRecords.get(tp0).size());
}
@Test
@ -2477,7 +2477,7 @@ public class FetchRequestManagerTest {
Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> fetchedRecords = fetchRecords();
assertTrue(fetchedRecords.containsKey(tp0));
assertEquals(fetchedRecords.get(tp0).size(), 2);
assertEquals(2, fetchedRecords.get(tp0).size());
List<ConsumerRecord<byte[], byte[]>> fetchedConsumerRecords = fetchedRecords.get(tp0);
Set<String> expectedCommittedKeys = Set.of("commit1-1", "commit1-2");
Set<String> actuallyCommittedKeys = new HashSet<>();
@ -2854,7 +2854,7 @@ public class FetchRequestManagerTest {
Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> fetchedRecords = fetchRecords();
assertTrue(fetchedRecords.containsKey(tp0));
assertEquals(fetchedRecords.get(tp0).size(), 2);
assertEquals(2, fetchedRecords.get(tp0).size());
}
private MemoryRecords buildRecords(long baseOffset, int count, long firstMessageId) {
@ -2939,8 +2939,8 @@ public class FetchRequestManagerTest {
Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecords = fetchRecords();
assertTrue(partitionRecords.containsKey(tp0));
assertEquals(subscriptions.position(tp0).offset, 3L);
assertOptional(subscriptions.position(tp0).offsetEpoch, value -> assertEquals(value.intValue(), 1));
assertEquals(3L, subscriptions.position(tp0).offset);
assertOptional(subscriptions.position(tp0).offsetEpoch, value -> assertEquals(1, value.intValue()));
}
@Test
@ -3110,7 +3110,7 @@ public class FetchRequestManagerTest {
fetchRecords();
Node selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds());
assertEquals(selected.id(), 1);
assertEquals(1, selected.id());
assertEquals(1, sendFetches());
assertFalse(fetcher.hasCompletedFetches());
@ -3124,7 +3124,7 @@ public class FetchRequestManagerTest {
fetchRecords();
selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds());
assertEquals(selected.id(), -1);
assertEquals(-1, selected.id());
}
@Test

View File

@ -1802,7 +1802,7 @@ public class FetcherTest {
assertEquals(1, oorExceptions.size());
OffsetOutOfRangeException oor = oorExceptions.get(0);
assertTrue(oor.offsetOutOfRangePartitions().containsKey(tp0));
assertEquals(oor.offsetOutOfRangePartitions().size(), 1);
assertEquals(1, oor.offsetOutOfRangePartitions().size());
fetchRecordsInto(fetchedRecords);
@ -2346,7 +2346,7 @@ public class FetcherTest {
Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> fetchedRecords = fetchRecords();
assertTrue(fetchedRecords.containsKey(tp0));
assertEquals(fetchedRecords.get(tp0).size(), 2);
assertEquals(2, fetchedRecords.get(tp0).size());
}
@Test
@ -2464,7 +2464,7 @@ public class FetcherTest {
Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> fetchedRecords = fetchRecords();
assertTrue(fetchedRecords.containsKey(tp0));
assertEquals(fetchedRecords.get(tp0).size(), 2);
assertEquals(2, fetchedRecords.get(tp0).size());
List<ConsumerRecord<byte[], byte[]>> fetchedConsumerRecords = fetchedRecords.get(tp0);
Set<String> expectedCommittedKeys = Set.of("commit1-1", "commit1-2");
Set<String> actuallyCommittedKeys = new HashSet<>();
@ -3054,7 +3054,7 @@ public class FetcherTest {
Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> fetchedRecords = fetchRecords();
assertTrue(fetchedRecords.containsKey(tp0));
assertEquals(fetchedRecords.get(tp0).size(), 2);
assertEquals(2, fetchedRecords.get(tp0).size());
}
private MemoryRecords buildRecords(long baseOffset, int count, long firstMessageId) {
@ -3139,8 +3139,8 @@ public class FetcherTest {
Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecords = fetchRecords();
assertTrue(partitionRecords.containsKey(tp0));
assertEquals(subscriptions.position(tp0).offset, 3L);
assertOptional(subscriptions.position(tp0).offsetEpoch, value -> assertEquals(value.intValue(), 1));
assertEquals(3L, subscriptions.position(tp0).offset);
assertOptional(subscriptions.position(tp0).offsetEpoch, value -> assertEquals(1, value.intValue()));
}
@Test
@ -3217,8 +3217,8 @@ public class FetcherTest {
Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecords = fetchRecords();
assertTrue(partitionRecords.containsKey(tp0));
assertEquals(subscriptions.position(tp0).offset, 3L);
assertOptional(subscriptions.position(tp0).offsetEpoch, value -> assertEquals(value.intValue(), 1));
assertEquals(3L, subscriptions.position(tp0).offset);
assertOptional(subscriptions.position(tp0).offsetEpoch, value -> assertEquals(1, value.intValue()));
}
@Test
@ -3388,7 +3388,7 @@ public class FetcherTest {
fetchRecords();
Node selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds());
assertEquals(selected.id(), 1);
assertEquals(1, selected.id());
assertEquals(1, sendFetches());
assertFalse(fetcher.hasCompletedFetches());
@ -3402,7 +3402,7 @@ public class FetcherTest {
fetchRecords();
selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds());
assertEquals(selected.id(), -1);
assertEquals(-1, selected.id());
}
@Test

View File

@ -99,8 +99,8 @@ class KafkaConsumerMetricsTest {
private void assertMetricValue(final String name) {
assertEquals(
metrics.metric(metrics.metricName(name, CONSUMER_METRIC_GROUP)).metricValue(),
(double) METRIC_VALUE
(double) METRIC_VALUE,
metrics.metric(metrics.metricName(name, CONSUMER_METRIC_GROUP)).metricValue()
);
}
}

View File

@ -246,7 +246,7 @@ public class OffsetFetcherTest {
assertTrue(subscriptions.hasValidPosition(tp0));
assertFalse(subscriptions.isOffsetResetNeeded(tp0));
assertTrue(subscriptions.isFetchable(tp0));
assertEquals(subscriptions.position(tp0).offset, 5L);
assertEquals(5L, subscriptions.position(tp0).offset);
}
@Test
@ -395,7 +395,7 @@ public class OffsetFetcherTest {
assertFalse(subscriptions.isOffsetResetNeeded(tp0));
assertTrue(metadata.updateRequested());
assertOptional(metadata.lastSeenLeaderEpoch(tp0), epoch -> assertEquals((long) epoch, 2));
assertOptional(metadata.lastSeenLeaderEpoch(tp0), epoch -> assertEquals(2, (long) epoch));
}
@Test
@ -902,7 +902,7 @@ public class OffsetFetcherTest {
ListOffsetsRequest offsetRequest = (ListOffsetsRequest) body;
int epoch = offsetRequest.topics().get(0).partitions().get(0).currentLeaderEpoch();
assertTrue(epoch != ListOffsetsResponse.UNKNOWN_EPOCH, "Expected Fetcher to set leader epoch in request");
assertEquals(epoch, 99, "Expected leader epoch to match epoch from metadata update");
assertEquals(99, epoch, "Expected leader epoch to match epoch from metadata update");
return true;
} else {
fail("Should have seen ListOffsetRequest");

View File

@ -102,8 +102,8 @@ public class OffsetForLeaderEpochClientTest {
assertTrue(result.partitionsToRetry().isEmpty());
assertTrue(result.endOffsets().containsKey(tp0));
assertEquals(result.endOffsets().get(tp0).errorCode(), Errors.NONE.code());
assertEquals(result.endOffsets().get(tp0).leaderEpoch(), 1);
assertEquals(result.endOffsets().get(tp0).endOffset(), 10L);
assertEquals(1, result.endOffsets().get(tp0).leaderEpoch());
assertEquals(10L, result.endOffsets().get(tp0).endOffset());
}
@Test
@ -121,7 +121,7 @@ public class OffsetForLeaderEpochClientTest {
consumerClient.pollNoWakeup();
assertTrue(future.failed());
assertEquals(future.exception().getClass(), TopicAuthorizationException.class);
assertEquals(TopicAuthorizationException.class, future.exception().getClass());
assertTrue(((TopicAuthorizationException) future.exception()).unauthorizedTopics().contains(tp0.topic()));
}

View File

@ -213,18 +213,19 @@ public class ShareConsumerImplTest {
props.put(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG, "an.invalid.class");
final ConsumerConfig config = new ConsumerConfig(props);
LogCaptureAppender appender = LogCaptureAppender.createAndRegister();
KafkaException ce = assertThrows(
try (LogCaptureAppender appender = LogCaptureAppender.createAndRegister()) {
KafkaException ce = assertThrows(
KafkaException.class,
() -> newConsumer(config));
assertTrue(ce.getMessage().contains("Failed to construct Kafka share consumer"), "Unexpected exception message: " + ce.getMessage());
assertTrue(ce.getCause().getMessage().contains("Class an.invalid.class cannot be found"), "Unexpected cause: " + ce.getCause());
assertTrue(ce.getMessage().contains("Failed to construct Kafka share consumer"), "Unexpected exception message: " + ce.getMessage());
assertTrue(ce.getCause().getMessage().contains("Class an.invalid.class cannot be found"), "Unexpected cause: " + ce.getCause());
boolean npeLogged = appender.getEvents().stream()
boolean npeLogged = appender.getEvents().stream()
.flatMap(event -> event.getThrowableInfo().stream())
.anyMatch(str -> str.contains("NullPointerException"));
assertFalse(npeLogged, "Unexpected NullPointerException during consumer construction");
assertFalse(npeLogged, "Unexpected NullPointerException during consumer construction");
}
}
@Test

View File

@ -439,14 +439,10 @@ class StreamsGroupHeartbeatRequestManagerTest {
try (
final MockedConstruction<HeartbeatRequestState> heartbeatRequestStateMockedConstruction = mockConstruction(
HeartbeatRequestState.class,
(mock, context) -> {
when(mock.heartbeatIntervalMs()).thenReturn(heartbeatIntervalMs);
});
(mock, context) -> when(mock.heartbeatIntervalMs()).thenReturn(heartbeatIntervalMs));
final MockedConstruction<Timer> pollTimerMockedConstruction = mockConstruction(
Timer.class,
(mock, context) -> {
when(mock.isExpired()).thenReturn(true);
});
(mock, context) -> when(mock.isExpired()).thenReturn(true));
final MockedConstruction<StreamsGroupHeartbeatRequestManager.HeartbeatState> heartbeatStateMockedConstruction = mockConstruction(
StreamsGroupHeartbeatRequestManager.HeartbeatState.class)
) {
@ -473,14 +469,10 @@ class StreamsGroupHeartbeatRequestManagerTest {
try (
final MockedConstruction<HeartbeatRequestState> heartbeatRequestStateMockedConstruction = mockConstruction(
HeartbeatRequestState.class,
(mock, context) -> {
when(mock.timeToNextHeartbeatMs(time.milliseconds())).thenReturn(timeToNextHeartbeatMs);
});
(mock, context) -> when(mock.timeToNextHeartbeatMs(time.milliseconds())).thenReturn(timeToNextHeartbeatMs));
final MockedConstruction<Timer> pollTimerMockedConstruction = mockConstruction(
Timer.class,
(mock, context) -> {
when(mock.isExpired()).thenReturn(true);
});
(mock, context) -> when(mock.isExpired()).thenReturn(true));
final MockedConstruction<StreamsGroupHeartbeatRequestManager.HeartbeatState> heartbeatStateMockedConstruction = mockConstruction(
StreamsGroupHeartbeatRequestManager.HeartbeatState.class)
) {
@ -508,14 +500,10 @@ class StreamsGroupHeartbeatRequestManagerTest {
try (
final MockedConstruction<HeartbeatRequestState> heartbeatRequestStateMockedConstruction = mockConstruction(
HeartbeatRequestState.class,
(mock, context) -> {
when(mock.canSendRequest(time.milliseconds())).thenReturn(true);
});
(mock, context) -> when(mock.canSendRequest(time.milliseconds())).thenReturn(true));
final MockedConstruction<Timer> pollTimerMockedConstruction = mockConstruction(
Timer.class,
(mock, context) -> {
when(mock.isExpired()).thenReturn(true);
})
(mock, context) -> when(mock.isExpired()).thenReturn(true))
) {
final StreamsGroupHeartbeatRequestManager heartbeatRequestManager = createStreamsGroupHeartbeatRequestManager();
final HeartbeatRequestState heartbeatRequestState = heartbeatRequestStateMockedConstruction.constructed().get(0);
@ -551,9 +539,7 @@ class StreamsGroupHeartbeatRequestManagerTest {
try (
final MockedConstruction<HeartbeatRequestState> heartbeatRequestStateMockedConstruction = mockConstruction(
HeartbeatRequestState.class,
(mock, context) -> {
when(mock.canSendRequest(time.milliseconds())).thenReturn(true);
})
(mock, context) -> when(mock.canSendRequest(time.milliseconds())).thenReturn(true))
) {
final StreamsGroupHeartbeatRequestManager heartbeatRequestManager = createStreamsGroupHeartbeatRequestManager();
final HeartbeatRequestState heartbeatRequestState = heartbeatRequestStateMockedConstruction.constructed().get(0);
@ -1001,9 +987,7 @@ class StreamsGroupHeartbeatRequestManagerTest {
try (
final MockedConstruction<HeartbeatRequestState> heartbeatRequestStateMockedConstruction = mockConstruction(
HeartbeatRequestState.class,
(mock, context) -> {
when(mock.canSendRequest(time.milliseconds())).thenReturn(true);
});
(mock, context) -> when(mock.canSendRequest(time.milliseconds())).thenReturn(true));
final MockedConstruction<StreamsGroupHeartbeatRequestManager.HeartbeatState> heartbeatStateMockedConstruction = mockConstruction(
StreamsGroupHeartbeatRequestManager.HeartbeatState.class)
) {
@ -1032,9 +1016,7 @@ class StreamsGroupHeartbeatRequestManagerTest {
try (
final MockedConstruction<HeartbeatRequestState> heartbeatRequestStateMockedConstruction = mockConstruction(
HeartbeatRequestState.class,
(mock, context) -> {
when(mock.canSendRequest(time.milliseconds())).thenReturn(true);
});
(mock, context) -> when(mock.canSendRequest(time.milliseconds())).thenReturn(true));
final MockedConstruction<StreamsGroupHeartbeatRequestManager.HeartbeatState> heartbeatStateMockedConstruction = mockConstruction(
StreamsGroupHeartbeatRequestManager.HeartbeatState.class)
) {
@ -1073,9 +1055,7 @@ class StreamsGroupHeartbeatRequestManagerTest {
try (
final MockedConstruction<HeartbeatRequestState> heartbeatRequestStateMockedConstruction = mockConstruction(
HeartbeatRequestState.class,
(mock, context) -> {
when(mock.canSendRequest(time.milliseconds())).thenReturn(true);
});
(mock, context) -> when(mock.canSendRequest(time.milliseconds())).thenReturn(true));
final MockedConstruction<StreamsGroupHeartbeatRequestManager.HeartbeatState> heartbeatStateMockedConstruction = mockConstruction(
StreamsGroupHeartbeatRequestManager.HeartbeatState.class)
) {
@ -1111,9 +1091,7 @@ class StreamsGroupHeartbeatRequestManagerTest {
try (
final MockedConstruction<HeartbeatRequestState> heartbeatRequestStateMockedConstruction = mockConstruction(
HeartbeatRequestState.class,
(mock, context) -> {
when(mock.canSendRequest(time.milliseconds())).thenReturn(true);
});
(mock, context) -> when(mock.canSendRequest(time.milliseconds())).thenReturn(true));
final MockedConstruction<StreamsGroupHeartbeatRequestManager.HeartbeatState> heartbeatStateMockedConstruction = mockConstruction(
StreamsGroupHeartbeatRequestManager.HeartbeatState.class)
) {
@ -1145,9 +1123,7 @@ class StreamsGroupHeartbeatRequestManagerTest {
try (
final MockedConstruction<HeartbeatRequestState> heartbeatRequestStateMockedConstruction = mockConstruction(
HeartbeatRequestState.class,
(mock, context) -> {
when(mock.canSendRequest(time.milliseconds())).thenReturn(true);
});
(mock, context) -> when(mock.canSendRequest(time.milliseconds())).thenReturn(true));
final MockedConstruction<StreamsGroupHeartbeatRequestManager.HeartbeatState> heartbeatStateMockedConstruction = mockConstruction(
StreamsGroupHeartbeatRequestManager.HeartbeatState.class)
) {
@ -1173,9 +1149,7 @@ class StreamsGroupHeartbeatRequestManagerTest {
try (
final MockedConstruction<HeartbeatRequestState> heartbeatRequestStateMockedConstruction = mockConstruction(
HeartbeatRequestState.class,
(mock, context) -> {
when(mock.canSendRequest(time.milliseconds())).thenReturn(true);
});
(mock, context) -> when(mock.canSendRequest(time.milliseconds())).thenReturn(true));
final MockedConstruction<StreamsGroupHeartbeatRequestManager.HeartbeatState> heartbeatStateMockedConstruction = mockConstruction(
StreamsGroupHeartbeatRequestManager.HeartbeatState.class);
final LogCaptureAppender logAppender = LogCaptureAppender.createAndRegister(StreamsGroupHeartbeatRequestManager.class)
@ -1212,9 +1186,7 @@ class StreamsGroupHeartbeatRequestManagerTest {
try (
final MockedConstruction<HeartbeatRequestState> heartbeatRequestStateMockedConstruction = mockConstruction(
HeartbeatRequestState.class,
(mock, context) -> {
when(mock.canSendRequest(time.milliseconds())).thenReturn(true);
});
(mock, context) -> when(mock.canSendRequest(time.milliseconds())).thenReturn(true));
final MockedConstruction<StreamsGroupHeartbeatRequestManager.HeartbeatState> heartbeatStateMockedConstruction = mockConstruction(
StreamsGroupHeartbeatRequestManager.HeartbeatState.class);
final LogCaptureAppender logAppender = LogCaptureAppender.createAndRegister(StreamsGroupHeartbeatRequestManager.class)
@ -1261,9 +1233,7 @@ class StreamsGroupHeartbeatRequestManagerTest {
try (
final MockedConstruction<HeartbeatRequestState> heartbeatRequestStateMockedConstruction = mockConstruction(
HeartbeatRequestState.class,
(mock, context) -> {
when(mock.canSendRequest(time.milliseconds())).thenReturn(true);
});
(mock, context) -> when(mock.canSendRequest(time.milliseconds())).thenReturn(true));
final MockedConstruction<StreamsGroupHeartbeatRequestManager.HeartbeatState> heartbeatStateMockedConstruction = mockConstruction(
StreamsGroupHeartbeatRequestManager.HeartbeatState.class);
final LogCaptureAppender logAppender = LogCaptureAppender.createAndRegister(StreamsGroupHeartbeatRequestManager.class)
@ -1312,9 +1282,7 @@ class StreamsGroupHeartbeatRequestManagerTest {
try (
final MockedConstruction<HeartbeatRequestState> heartbeatRequestStateMockedConstruction = mockConstruction(
HeartbeatRequestState.class,
(mock, context) -> {
when(mock.canSendRequest(time.milliseconds())).thenReturn(true);
});
(mock, context) -> when(mock.canSendRequest(time.milliseconds())).thenReturn(true));
final MockedConstruction<StreamsGroupHeartbeatRequestManager.HeartbeatState> heartbeatStateMockedConstruction = mockConstruction(
StreamsGroupHeartbeatRequestManager.HeartbeatState.class)
) {
@ -1343,9 +1311,7 @@ class StreamsGroupHeartbeatRequestManagerTest {
try (
final MockedConstruction<HeartbeatRequestState> heartbeatRequestStateMockedConstruction = mockConstruction(
HeartbeatRequestState.class,
(mock, context) -> {
when(mock.canSendRequest(time.milliseconds())).thenReturn(true);
});
(mock, context) -> when(mock.canSendRequest(time.milliseconds())).thenReturn(true));
final MockedConstruction<StreamsGroupHeartbeatRequestManager.HeartbeatState> heartbeatStateMockedConstruction = mockConstruction(
StreamsGroupHeartbeatRequestManager.HeartbeatState.class);
final LogCaptureAppender logAppender = LogCaptureAppender.createAndRegister(StreamsGroupHeartbeatRequestManager.class)
@ -1424,14 +1390,11 @@ class StreamsGroupHeartbeatRequestManagerTest {
@Test
public void testMaximumTimeToWaitPollTimerExpired() {
try (
final MockedConstruction<Timer> timerMockedConstruction = mockConstruction(Timer.class, (mock, context) -> {
when(mock.isExpired()).thenReturn(true);
});
final MockedConstruction<Timer> timerMockedConstruction =
mockConstruction(Timer.class, (mock, context) -> when(mock.isExpired()).thenReturn(true));
final MockedConstruction<HeartbeatRequestState> heartbeatRequestStateMockedConstruction = mockConstruction(
HeartbeatRequestState.class,
(mock, context) -> {
when(mock.requestInFlight()).thenReturn(false);
})
(mock, context) -> when(mock.requestInFlight()).thenReturn(false))
) {
final StreamsGroupHeartbeatRequestManager heartbeatRequestManager = createStreamsGroupHeartbeatRequestManager();
final Timer pollTimer = timerMockedConstruction.constructed().get(0);
@ -1450,9 +1413,7 @@ class StreamsGroupHeartbeatRequestManagerTest {
final MockedConstruction<Timer> timerMockedConstruction = mockConstruction(Timer.class);
final MockedConstruction<HeartbeatRequestState> heartbeatRequestStateMockedConstruction = mockConstruction(
HeartbeatRequestState.class,
(mock, context) -> {
when(mock.requestInFlight()).thenReturn(false);
})
(mock, context) -> when(mock.requestInFlight()).thenReturn(false))
) {
final StreamsGroupHeartbeatRequestManager heartbeatRequestManager = createStreamsGroupHeartbeatRequestManager();
final Timer pollTimer = timerMockedConstruction.constructed().get(0);
@ -1473,9 +1434,8 @@ class StreamsGroupHeartbeatRequestManagerTest {
final long remainingMs = 12L;
final long timeToNextHeartbeatMs = 6L;
try (
final MockedConstruction<Timer> timerMockedConstruction = mockConstruction(Timer.class, (mock, context) -> {
when(mock.remainingMs()).thenReturn(remainingMs);
});
final MockedConstruction<Timer> timerMockedConstruction =
mockConstruction(Timer.class, (mock, context) -> when(mock.remainingMs()).thenReturn(remainingMs));
final MockedConstruction<HeartbeatRequestState> heartbeatRequestStateMockedConstruction = mockConstruction(
HeartbeatRequestState.class,
(mock, context) -> {
@ -1500,14 +1460,11 @@ class StreamsGroupHeartbeatRequestManagerTest {
public void testMaximumTimeToWaitSelectingMinimumWaitTime(final long remainingMs,
final long timeToNextHeartbeatMs) {
try (
final MockedConstruction<Timer> timerMockedConstruction = mockConstruction(Timer.class, (mock, context) -> {
when(mock.remainingMs()).thenReturn(remainingMs);
});
final MockedConstruction<Timer> timerMockedConstruction =
mockConstruction(Timer.class, (mock, context) -> when(mock.remainingMs()).thenReturn(remainingMs));
final MockedConstruction<HeartbeatRequestState> heartbeatRequestStateMockedConstruction = mockConstruction(
HeartbeatRequestState.class,
(mock, context) -> {
when(mock.timeToNextHeartbeatMs(anyLong())).thenReturn(timeToNextHeartbeatMs);
})
(mock, context) -> when(mock.timeToNextHeartbeatMs(anyLong())).thenReturn(timeToNextHeartbeatMs))
) {
final StreamsGroupHeartbeatRequestManager heartbeatRequestManager = createStreamsGroupHeartbeatRequestManager();
final Timer pollTimer = timerMockedConstruction.constructed().get(0);

View File

@ -67,6 +67,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
import static org.junit.jupiter.api.Assertions.assertNotEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.ArgumentMatchers.any;
@ -131,6 +132,15 @@ public class StreamsMembershipManagerTest {
verifyInStateUnsubscribed(membershipManager);
}
@Test
public void testAssignedPartitionCountMetricRegistered() {
MetricName metricName = metrics.metricName(
"assigned-partitions",
CONSUMER_METRIC_GROUP_PREFIX + COORDINATOR_METRICS_SUFFIX
);
assertNotNull(metrics.metric(metricName), "Metric assigned-partitions should have been registered");
}
@Test
public void testUnexpectedErrorInHeartbeatResponse() {
final String errorMessage = "Nobody expects the Spanish Inquisition!";

View File

@ -62,7 +62,7 @@ public class StreamsRebalanceDataTest {
public void testTaskIdCompareTo() {
final StreamsRebalanceData.TaskId task = new StreamsRebalanceData.TaskId("subtopologyId1", 1);
assertTrue(task.compareTo(new StreamsRebalanceData.TaskId(task.subtopologyId(), task.partitionId())) == 0);
assertEquals(0, task.compareTo(new StreamsRebalanceData.TaskId(task.subtopologyId(), task.partitionId())));
assertTrue(task.compareTo(new StreamsRebalanceData.TaskId(task.subtopologyId() + "1", task.partitionId())) < 0);
assertTrue(task.compareTo(new StreamsRebalanceData.TaskId(task.subtopologyId(), task.partitionId() + 1)) < 0);
assertTrue(new StreamsRebalanceData.TaskId(task.subtopologyId() + "1", task.partitionId()).compareTo(task) > 0);

View File

@ -0,0 +1,276 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.clients.consumer.internals;
import org.apache.kafka.common.errors.InterruptException;
import org.apache.kafka.common.errors.WakeupException;
import org.apache.kafka.common.utils.LogContext;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.Mock;
import org.mockito.junit.jupiter.MockitoExtension;
import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness;
import java.util.Set;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
@ExtendWith(MockitoExtension.class)
@MockitoSettings(strictness = Strictness.STRICT_STUBS)
public class StreamsRebalanceListenerInvokerTest {
@Mock
private StreamsRebalanceListener mockListener;
@Mock
private StreamsRebalanceData streamsRebalanceData;
private StreamsRebalanceListenerInvoker invoker;
private final LogContext logContext = new LogContext();
@BeforeEach
public void setup() {
invoker = new StreamsRebalanceListenerInvoker(logContext, streamsRebalanceData);
}
@Test
public void testSetRebalanceListenerWithNull() {
NullPointerException exception = assertThrows(NullPointerException.class,
() -> invoker.setRebalanceListener(null));
assertEquals("StreamsRebalanceListener cannot be null", exception.getMessage());
}
@Test
public void testSetRebalanceListenerOverwritesExisting() {
StreamsRebalanceListener firstListener = org.mockito.Mockito.mock(StreamsRebalanceListener.class);
StreamsRebalanceListener secondListener = org.mockito.Mockito.mock(StreamsRebalanceListener.class);
StreamsRebalanceData.Assignment mockAssignment = createMockAssignment();
when(streamsRebalanceData.reconciledAssignment()).thenReturn(mockAssignment);
// Set first listener
invoker.setRebalanceListener(firstListener);
// Overwrite with second listener
invoker.setRebalanceListener(secondListener);
// Should use second listener
invoker.invokeAllTasksRevoked();
verify(firstListener, never()).onTasksRevoked(any());
verify(secondListener).onTasksRevoked(eq(mockAssignment.activeTasks()));
}
@Test
public void testInvokeMethodsWithNoListener() {
assertNull(invoker.invokeAllTasksRevoked());
assertNull(invoker.invokeTasksAssigned(createMockAssignment()));
assertNull(invoker.invokeTasksRevoked(createMockTasks()));
assertNull(invoker.invokeAllTasksLost());
}
@Test
public void testInvokeAllTasksRevokedWithListener() {
invoker.setRebalanceListener(mockListener);
StreamsRebalanceData.Assignment mockAssignment = createMockAssignment();
when(streamsRebalanceData.reconciledAssignment()).thenReturn(mockAssignment);
Exception result = invoker.invokeAllTasksRevoked();
assertNull(result);
verify(mockListener).onTasksRevoked(eq(mockAssignment.activeTasks()));
}
@Test
public void testInvokeTasksAssignedWithListener() {
invoker.setRebalanceListener(mockListener);
StreamsRebalanceData.Assignment assignment = createMockAssignment();
Exception result = invoker.invokeTasksAssigned(assignment);
assertNull(result);
verify(mockListener).onTasksAssigned(eq(assignment));
}
@Test
public void testInvokeTasksAssignedWithWakeupException() {
invoker.setRebalanceListener(mockListener);
StreamsRebalanceData.Assignment assignment = createMockAssignment();
WakeupException wakeupException = new WakeupException();
doThrow(wakeupException).when(mockListener).onTasksAssigned(assignment);
WakeupException thrownException = assertThrows(WakeupException.class,
() -> invoker.invokeTasksAssigned(assignment));
assertEquals(wakeupException, thrownException);
verify(mockListener).onTasksAssigned(eq(assignment));
}
@Test
public void testInvokeTasksAssignedWithInterruptException() {
invoker.setRebalanceListener(mockListener);
StreamsRebalanceData.Assignment assignment = createMockAssignment();
InterruptException interruptException = new InterruptException("Test interrupt");
doThrow(interruptException).when(mockListener).onTasksAssigned(assignment);
InterruptException thrownException = assertThrows(InterruptException.class,
() -> invoker.invokeTasksAssigned(assignment));
assertEquals(interruptException, thrownException);
verify(mockListener).onTasksAssigned(eq(assignment));
}
@Test
public void testInvokeTasksAssignedWithOtherException() {
invoker.setRebalanceListener(mockListener);
StreamsRebalanceData.Assignment assignment = createMockAssignment();
RuntimeException runtimeException = new RuntimeException("Test exception");
doThrow(runtimeException).when(mockListener).onTasksAssigned(assignment);
Exception result = invoker.invokeTasksAssigned(assignment);
assertEquals(runtimeException, result);
verify(mockListener).onTasksAssigned(eq(assignment));
}
@Test
public void testInvokeTasksRevokedWithListener() {
invoker.setRebalanceListener(mockListener);
Set<StreamsRebalanceData.TaskId> tasks = createMockTasks();
Exception result = invoker.invokeTasksRevoked(tasks);
assertNull(result);
verify(mockListener).onTasksRevoked(eq(tasks));
}
@Test
public void testInvokeTasksRevokedWithWakeupException() {
invoker.setRebalanceListener(mockListener);
Set<StreamsRebalanceData.TaskId> tasks = createMockTasks();
WakeupException wakeupException = new WakeupException();
doThrow(wakeupException).when(mockListener).onTasksRevoked(tasks);
WakeupException thrownException = assertThrows(WakeupException.class,
() -> invoker.invokeTasksRevoked(tasks));
assertEquals(wakeupException, thrownException);
verify(mockListener).onTasksRevoked(eq(tasks));
}
@Test
public void testInvokeTasksRevokedWithInterruptException() {
invoker.setRebalanceListener(mockListener);
Set<StreamsRebalanceData.TaskId> tasks = createMockTasks();
InterruptException interruptException = new InterruptException("Test interrupt");
doThrow(interruptException).when(mockListener).onTasksRevoked(tasks);
InterruptException thrownException = assertThrows(InterruptException.class,
() -> invoker.invokeTasksRevoked(tasks));
assertEquals(interruptException, thrownException);
verify(mockListener).onTasksRevoked(eq(tasks));
}
@Test
public void testInvokeTasksRevokedWithOtherException() {
invoker.setRebalanceListener(mockListener);
Set<StreamsRebalanceData.TaskId> tasks = createMockTasks();
RuntimeException runtimeException = new RuntimeException("Test exception");
doThrow(runtimeException).when(mockListener).onTasksRevoked(tasks);
Exception result = invoker.invokeTasksRevoked(tasks);
assertEquals(runtimeException, result);
verify(mockListener).onTasksRevoked(eq(tasks));
}
@Test
public void testInvokeAllTasksLostWithListener() {
invoker.setRebalanceListener(mockListener);
Exception result = invoker.invokeAllTasksLost();
assertNull(result);
verify(mockListener).onAllTasksLost();
}
@Test
public void testInvokeAllTasksLostWithWakeupException() {
invoker.setRebalanceListener(mockListener);
WakeupException wakeupException = new WakeupException();
doThrow(wakeupException).when(mockListener).onAllTasksLost();
WakeupException thrownException = assertThrows(WakeupException.class,
() -> invoker.invokeAllTasksLost());
assertEquals(wakeupException, thrownException);
verify(mockListener).onAllTasksLost();
}
@Test
public void testInvokeAllTasksLostWithInterruptException() {
invoker.setRebalanceListener(mockListener);
InterruptException interruptException = new InterruptException("Test interrupt");
doThrow(interruptException).when(mockListener).onAllTasksLost();
InterruptException thrownException = assertThrows(InterruptException.class,
() -> invoker.invokeAllTasksLost());
assertEquals(interruptException, thrownException);
verify(mockListener).onAllTasksLost();
}
@Test
public void testInvokeAllTasksLostWithOtherException() {
invoker.setRebalanceListener(mockListener);
RuntimeException runtimeException = new RuntimeException("Test exception");
doThrow(runtimeException).when(mockListener).onAllTasksLost();
Exception result = invoker.invokeAllTasksLost();
assertEquals(runtimeException, result);
verify(mockListener).onAllTasksLost();
}
private StreamsRebalanceData.Assignment createMockAssignment() {
Set<StreamsRebalanceData.TaskId> activeTasks = createMockTasks();
Set<StreamsRebalanceData.TaskId> standbyTasks = Set.of();
Set<StreamsRebalanceData.TaskId> warmupTasks = Set.of();
return new StreamsRebalanceData.Assignment(activeTasks, standbyTasks, warmupTasks);
}
private Set<StreamsRebalanceData.TaskId> createMockTasks() {
return Set.of(
new StreamsRebalanceData.TaskId("subtopology1", 0),
new StreamsRebalanceData.TaskId("subtopology1", 1)
);
}
}

View File

@ -113,13 +113,13 @@ public class AsyncConsumerMetricsTest {
// Then:
assertEquals(
(double) 10,
metrics.metric(
metrics.metricName(
"application-event-queue-size",
groupName
)
).metricValue(),
(double) 10
).metricValue()
);
}
@ -156,13 +156,13 @@ public class AsyncConsumerMetricsTest {
// Then:
assertEquals(
(double) 10,
metrics.metric(
metrics.metricName(
"unsent-requests-queue-size",
groupName
)
).metricValue(),
(double) 10
).metricValue()
);
}
@ -187,13 +187,13 @@ public class AsyncConsumerMetricsTest {
// Then:
assertEquals(
(double) 10,
metrics.metric(
metrics.metricName(
"background-event-queue-size",
groupName
)
).metricValue(),
(double) 10
).metricValue()
);
}
@ -223,13 +223,13 @@ public class AsyncConsumerMetricsTest {
private void assertMetricValue(final String name, final String groupName) {
assertEquals(
(double) METRIC_VALUE,
metrics.metric(
metrics.metricName(
name,
groupName
)
).metricValue(),
(double) METRIC_VALUE
).metricValue()
);
}
}

View File

@ -0,0 +1,63 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.clients.consumer.internals.metrics;
import org.apache.kafka.clients.consumer.internals.AutoOffsetResetStrategy;
import org.apache.kafka.clients.consumer.internals.SubscriptionState;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Time;
import org.junit.jupiter.api.Test;
import java.util.Optional;
import java.util.Set;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.mockito.Mockito.mock;
class ConsumerRebalanceMetricsManagerTest {
private final Time time = new MockTime();
private final Metrics metrics = new Metrics(time);
@Test
public void testAssignedPartitionCountMetric() {
SubscriptionState subscriptionState = new SubscriptionState(mock(LogContext.class), AutoOffsetResetStrategy.EARLIEST);
ConsumerRebalanceMetricsManager consumerRebalanceMetricsManager = new ConsumerRebalanceMetricsManager(metrics, subscriptionState);
assertNotNull(metrics.metric(consumerRebalanceMetricsManager.assignedPartitionsCount), "Metric assigned-partitions has not been registered as expected");
// Check for manually assigned partitions
subscriptionState.assignFromUser(Set.of(new TopicPartition("topic", 0), new TopicPartition("topic", 1)));
assertEquals(2.0d, metrics.metric(consumerRebalanceMetricsManager.assignedPartitionsCount).metricValue());
subscriptionState.assignFromUser(Set.of());
assertEquals(0.0d, metrics.metric(consumerRebalanceMetricsManager.assignedPartitionsCount).metricValue());
subscriptionState.unsubscribe();
assertEquals(0.0d, metrics.metric(consumerRebalanceMetricsManager.assignedPartitionsCount).metricValue());
// Check for automatically assigned partitions
subscriptionState.subscribe(Set.of("topic"), Optional.empty());
subscriptionState.assignFromSubscribed(Set.of(new TopicPartition("topic", 0)));
assertEquals(1.0d, metrics.metric(consumerRebalanceMetricsManager.assignedPartitionsCount).metricValue());
}
}

View File

@ -219,7 +219,7 @@ public class KafkaProducerTest {
}
@AfterEach
public void detectLeaks() {
public void detectLeaks() throws InterruptedException {
// Assert no thread leakage of Kafka producer.
TestUtils.assertNoLeakedThreadsWithNameAndDaemonStatus(NETWORK_THREAD_PREFIX, Boolean.TRUE);
}

View File

@ -724,10 +724,10 @@ public class MockProducerTest {
buildMockProducer(false);
Future<RecordMetadata> metadata = producer.send(record2, (md, exception) -> {
assertNotNull(md);
assertEquals(md.offset(), -1L, "Invalid offset");
assertEquals(md.timestamp(), RecordBatch.NO_TIMESTAMP, "Invalid timestamp");
assertEquals(md.serializedKeySize(), -1L, "Invalid Serialized Key size");
assertEquals(md.serializedValueSize(), -1L, "Invalid Serialized value size");
assertEquals(-1L, md.offset(), "Invalid offset");
assertEquals(RecordBatch.NO_TIMESTAMP, md.timestamp(), "Invalid timestamp");
assertEquals(-1L, md.serializedKeySize(), "Invalid Serialized Key size");
assertEquals(-1L, md.serializedValueSize(), "Invalid Serialized value size");
});
IllegalArgumentException e = new IllegalArgumentException("dummy exception");
assertTrue(producer.errorNext(e), "Complete the second request with an error");

View File

@ -219,7 +219,7 @@ public class BufferPoolTest {
t1.join();
t2.join();
// both the allocate() called by threads t1 and t2 should have been interrupted and the waiters queue should be empty
assertEquals(pool.queued(), 0);
assertEquals(0, pool.queued());
}
@Test
@ -332,7 +332,7 @@ public class BufferPoolTest {
}
assertEquals(bufferPool.availableMemory(), 1024);
assertEquals(1024, bufferPool.availableMemory());
}
public static class StressTestThread extends Thread {

View File

@ -121,8 +121,8 @@ class KafkaProducerMetricsTest {
private void assertMetricValue(final String name) {
assertEquals(
metrics.metric(metrics.metricName(name, KafkaProducerMetrics.GROUP)).metricValue(),
(double) METRIC_VALUE
(double) METRIC_VALUE,
metrics.metric(metrics.metricName(name, KafkaProducerMetrics.GROUP)).metricValue()
);
}
}

View File

@ -892,7 +892,7 @@ public class RecordAccumulatorTest {
readyNodes = accum.ready(metadataCache, time.milliseconds()).readyNodes;
assertEquals(Collections.singleton(node1), readyNodes, "Our partition's leader should be ready");
Map<Integer, List<ProducerBatch>> drained = accum.drain(metadataCache, readyNodes, Integer.MAX_VALUE, time.milliseconds());
assertEquals(drained.get(node1.id()).size(), 1, "There should be only one batch.");
assertEquals(1, drained.get(node1.id()).size(), "There should be only one batch.");
time.sleep(1000L);
accum.reenqueue(drained.get(node1.id()).get(0), time.milliseconds());
@ -1788,6 +1788,6 @@ public class RecordAccumulatorTest {
}
// Verify all original records are accounted for (no data loss)
assertEquals(keyFoundMap.size(), 100, "All original 100 records should be present after splitting");
assertEquals(100, keyFoundMap.size(), "All original 100 records should be present after splitting");
}
}

Some files were not shown because too many files have changed in this diff Show More