Compare commits

...

26 Commits

Author SHA1 Message Date
Chang-Yu Huang 6969ca94d2
Merge f9703f7e8b into 4a5aa37169 2025-10-07 15:53:47 -04:00
Chang-Chi Hsu 4a5aa37169
MINOR: Move ReconfigurableQuorumIntegrationTest from core module to server module (#20636)
CI / build (push) Waiting to run Details
It moves the `ReconfigurableQuorumIntegrationTest` class to the
`org.apache.kafka.server` package and consolidates two related tests,
`RemoveAndAddVoterWithValidClusterId` and
`RemoveAndAddVoterWithInconsistentClusterId`, into a single file. This
improves code organization and reduces redundancy.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-10-08 01:10:58 +08:00
lucliu1108 2938c4242e
KAFKA-19754: Add RPC-level integration test for StreamsGroupDescribeRequest (#20632)
CI / build (push) Waiting to run Details
Test the `StreamsGroupDescribeRequest` RPC and corresponding responses
for situations where
- `streams.version` not upgraded to 1
- `streams.version` enabled, multiple groups listening to the same
topic.

Reviewers: Lucas Brutschy <lucasbru@apache.org>
2025-10-07 15:47:32 +02:00
Ken Huang ebae768bd8
KAFKA-18193 Refactor Kafka Streams CloseOptions to Fluent API Style (#19955)
In Kafka Streams, configuration classes typically follow a fluent API
pattern to ensure a consistent and intuitive developer experience.
However, the current implementation of
`org.apache.kafka.streams.KafkaStreams$CloseOptions` deviates from this
convention by exposing a public constructor, breaking the uniformity
expected across the API.

To address this inconsistency, we propose introducing a new
`CloseOptions` class that adheres to the fluent API style, replacing the
existing implementation. The new class will retain the existing
`timeout(Duration)` and `leaveGroup(boolean)` methods but will enforce
fluent instantiation and configuration. Given the design shift, we will
not maintain backward compatibility with the current class.

This change aligns with the goal of standardizing configuration objects
across Kafka Streams, offering developers a more cohesive and
predictable API.

Reviewers: Bill Bejeck<bbejeck@apache.org>
2025-10-07 08:50:18 -04:00
Jhen-Yung Hsu fa2496bb91
MINOR: skip ci-complete on 4.0 (#20644)
Since that 4.0 branch does not include
[KAFKA-18748](https://issues.apache.org/jira/browse/KAFKA-18748), it is
unable to find the related scan reports, but the ci-complete workflow is
still being triggered on the 4.0 branch. Disable this on the 4.0 branch,
as its reports can be safely ignored.

See https://github.com/apache/kafka/pull/20616#issuecomment-3370876779.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-10-07 20:27:29 +08:00
Dejan Stojadinović c6bbbbe24d
KAFKA-19174 Gradle version upgrade 8 -->> 9 (#19513)
List of changes:
- prerequisite Jira ticket:
  - [KAFKA-19591](https://issues.apache.org/jira/browse/KAFKA-19591)
- mandatory version upgrades:
  - Gradle version: 8.14.3 -->> 9.1.0
  - Gradle Shadow plugin: 8.3.6 -->> 8.3.9
  - Gradle dependencycheck plugin: 8.2.1 -->> 12.1.3
  - Gradle spotbugs plugin: 6.2.3 -->> 6.2.5
  - Gradle spotless plugin: 6.25.0 -->> 7.2.1
- build logic will be refactored to accommodate Gradle 9 breaking
changes
- (optional): a dozen of Gradle plugins versions will also be upgraded
- other JIRA tickets that had to be solved all along:
  - [KAFKA-16801](https://issues.apache.org/jira/browse/KAFKA-16801)
  - [KAFKA-19654](https://issues.apache.org/jira/browse/KAFKA-19654)

 **Related links:**
- https://gradle.org/whats-new/gradle-9
- https://github.com/gradle/gradle/releases/tag/v9.0.0
- https://docs.gradle.org/9.0.0/release-notes.html
- https://docs.gradle.org/9.0.0/userguide/upgrading_major_version_9.html
- https://docs.gradle.org/9.1.0/release-notes.html

Notes:
- new Gradle version brings up some breaking changes, as always 😃
- Kafka build with Gradle 9 has same issues as other projects:
  - https://github.com/redhat-developer/vscode-java/issues/4018
  - https://github.com/gradle/gradle/pull/32597

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-10-07 18:20:04 +08:00
Shashank f5a87b3703
KAFKA-19748: Add a note in docs about memory leak in Kafka Streams 4.1.0 (#20639)
CI / build (push) Waiting to run Details
Added a note regarding the memory leak bug in the documentation.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2025-10-06 11:59:51 -07:00
Arpit Goyal 2da02d9fcf
KAFKA-19723 Adding consumer rebalance metrics test (#20565)
Added Testcases for consumer rebalance metric manager test.

Reviewers: Lianet Magrans <lmagrans@confluent.io>, TengYao Chi
 <frankvicky@apache.org>, Hong-Yi Chen <apalan60@gmail.com>
2025-10-06 14:27:50 -04:00
Calvin Liu 162db130f6
MINOR: simplify last known elr update (#20629)
Simplify the last known elr update logic. This way can make a more
robust logic.

Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-10-06 10:02:50 -07:00
Ken Huang f68a149a18
KAFKA-18509 Move StateChangeLogger to server-common module (#20637)
We can rewrite this class from scala to java and move to server-common
module.  To maintain backward compatibility, we should keep the logger
name `state.change.logger`.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-10-06 22:56:57 +08:00
majialong 24cad50840
MINOR: Adjust the timing for creating connect config (#20612)
CI / build (push) Waiting to run Details
[In this PR](https://github.com/apache/kafka/pull/20334), we added some
validation checks for the connect config, such as ensuring that
`plugin.path` cannot be empty.

 However, currently, Connect first loads the plugin and then creates the
configuration. Even if `plugin.path` is empty, it still attempts to load
the plugin first, and then throws an exception when creating the
configuration.

The approach should be to first create a configuration to validate that
the config meet the requirements, and then load the plugin only if the
validation passes. This allows for early detection of problems and
avoids unnecessary plugin loading processes.

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-10-06 22:07:51 +08:00
Sean Quah 71a7d85955
KAFKA-19431: Ensure consumer and share assignment consistency with subscriptions (#20055)
Filter out unsubscribed topics during reconciliation.

This eliminates the window where a consumer group assignment could
contain unsubscribed topics when a member unsubscribes from a topic
while it has unrevoked partitions.

We also apply filtering in a few other cases that would arise when
client-side assignors are implemented, since new assignments would no
longer be available immediately. This is important for mixed groups,
since clients on the classic protocol will rejoin if they receive a
topic in their assignment that is no longer in their subscription.

Regex subscriptions have a window where the regex is not resolved and we
cannot know which topics are part of the subscription. We opt to be
conservative and treat unresolved regexes as matching no topics.

The same change is applied to share groups, since the reconciliation
process is similar.

To gauge the performance impact of the change, we add a jmh benchmark.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, Lianet Magran  <lmagrans@confluent.io>, Sushant Mahajan <smahajan@confluent.io>, Dongnuo Lyu <dlyu@confluent.io>, David Jacot <djacot@confluent.io>
2025-10-06 14:57:44 +02:00
Jinhe Zhang 611f4128b4
MINOR: Enable streams rebalance protocol in EosIntegrationTest (#20592)
CI / build (push) Waiting to run Details
Remove stalling instance in EOSIntegrationTest, since it doesn’t matter
what it thinks what the assignment is but blocks the test with streams
group protocol

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2025-10-05 15:55:59 +02:00
Chang-Yu Huang f9703f7e8b
Merge branch 'apache:trunk' into KAFKA-17103 2025-08-20 15:08:56 -04:00
ChangYu Huang cf1f13b0c2 Remove additional thread in testTopicExpiryInMetadata 2025-08-20 12:30:27 -04:00
ChangYu Huang ff404d01ec Remove additional thread in testTopicNotExistingInMetadata 2025-08-20 12:24:59 -04:00
ChangYu Huang 69975c02ae Remove additional thread in testTopicRefreshInMetadata 2025-08-20 12:18:01 -04:00
ChangYu Huang 4fa33f2d0f Add a prepared metadata count 2025-08-20 12:09:34 -04:00
ChangYu Huang f15d2ef460 Fix value mistake 2025-08-20 11:39:37 -04:00
ChangYu Huang fe3df3c168 Remove debug message 2025-08-19 22:09:40 -04:00
Chang-Yu Huang 5817251104
Merge branch 'apache:trunk' into KAFKA-17103 2025-08-19 21:33:23 -04:00
ChangYu Huang d1ab4b41b9 Advance mock time to prevent sender waiting for transactional response infinitely 2025-08-19 16:39:55 -04:00
ChangYu Huang f91dc5a239 Remove busy waiting in testTopicExpiryInMetadata 2025-08-19 16:24:09 -04:00
ChangYu Huang 2b68e44a35 Remove busy waiting in testTopicNotExistingInMetadata 2025-08-19 12:44:27 -04:00
ChangYu Huang 9922d9dacd Remove busy waiting testTopicRefreshInMetadata 2025-08-19 12:44:27 -04:00
ChangYu Huang 39fb8a0f95 Add more controls in MockClient 2025-08-19 12:44:08 -04:00
42 changed files with 2770 additions and 607 deletions

View File

@ -42,7 +42,7 @@ runs:
distribution: temurin
java-version: ${{ inputs.java-version }}
- name: Setup Gradle
uses: gradle/actions/setup-gradle@94baf225fe0a508e581a564467443d0e2379123b # v4.3.0
uses: gradle/actions/setup-gradle@748248ddd2a24f49513d8f472f81c3a07d4d50e1 # v4.4.4
env:
GRADLE_BUILD_ACTION_CACHE_DEBUG_ENABLED: true
with:

View File

@ -38,7 +38,7 @@ run-name: Build Scans for ${{ github.event.workflow_run.display_title}}
jobs:
upload-build-scan:
# Skip this workflow if the CI run was skipped or cancelled
if: (github.event.workflow_run.conclusion == 'success' || github.event.workflow_run.conclusion == 'failure')
if: (github.event.workflow_run.conclusion == 'success' || github.event.workflow_run.conclusion == 'failure') && github.event.workflow_run.head_branch != '4.0'
runs-on: ubuntu-latest
strategy:
fail-fast: false

View File

@ -29,22 +29,21 @@ buildscript {
}
plugins {
id 'com.github.ben-manes.versions' version '0.48.0'
id 'com.github.ben-manes.versions' version '0.52.0'
id 'idea'
id 'jacoco'
id 'java-library'
id 'org.owasp.dependencycheck' version '8.2.1'
id 'org.owasp.dependencycheck' version '12.1.3'
id 'org.nosphere.apache.rat' version "0.8.1"
id "io.swagger.core.v3.swagger-gradle-plugin" version "${swaggerVersion}"
id "com.github.spotbugs" version '6.2.3' apply false
id "com.github.spotbugs" version '6.2.5' apply false
id 'org.scoverage' version '8.0.3' apply false
id 'com.gradleup.shadow' version '8.3.6' apply false
id 'com.diffplug.spotless' version "6.25.0"
id 'com.gradleup.shadow' version '8.3.9' apply false
id 'com.diffplug.spotless' version "7.2.1"
}
ext {
gradleVersion = versions.gradle
minClientJavaVersion = 11
minNonClientJavaVersion = 17
modulesNeedingJava11 = [":clients", ":generator", ":streams", ":streams:test-utils", ":streams:examples", ":streams-scala", ":test-common:test-common-util"]
@ -297,7 +296,7 @@ if (repo != null) {
} else {
rat.enabled = false
}
println("Starting build with version $version (commit id ${commitId == null ? "null" : commitId.take(8)}) using Gradle $gradleVersion, Java ${JavaVersion.current()} and Scala ${versions.scala}")
println("Starting build with version $version (commit id ${commitId == null ? "null" : commitId.take(8)}) using Gradle $versions.gradle, Java ${JavaVersion.current()} and Scala ${versions.scala}")
println("Build properties: ignoreFailures=$userIgnoreFailures, maxParallelForks=$maxTestForks, maxScalacThreads=$maxScalacThreads, maxTestRetries=$userMaxTestRetries")
subprojects {
@ -328,6 +327,16 @@ subprojects {
tasks.register('uploadArchives').configure { dependsOn(publish) }
}
tasks.withType(AbstractArchiveTask).configureEach {
reproducibleFileOrder = false
preserveFileTimestamps = true
useFileSystemPermissions()
}
tasks.withType(AbstractTestTask).configureEach {
failOnNoDiscoveredTests = false
}
// apply the eclipse plugin only to subprojects that hold code. 'connect' is just a folder.
if (!project.name.equals('connect')) {
apply plugin: 'eclipse'

View File

@ -1,132 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.clients.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

@ -39,6 +39,8 @@ import java.util.Queue;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentLinkedDeque;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.function.Consumer;
import java.util.stream.Collectors;
@ -72,6 +74,8 @@ public class MockClient implements KafkaClient {
private int correlation;
private Runnable wakeupHook;
private boolean advanceTimeDuringPoll;
private boolean shouldUpdateWithCurrentMetadata = true;
private CountDownLatch isMetadataUpdateNeeded = new CountDownLatch(1);
private final Time time;
private final MockMetadataUpdater metadataUpdater;
private final Map<String, ConnectionState> connections = new HashMap<>();
@ -192,6 +196,10 @@ public class MockClient implements KafkaClient {
this.disconnectFuture = disconnectFuture;
}
public void setShouldUpdateWithCurrentMetadata(boolean shouldUpdateWithCurrentMetadata) {
this.shouldUpdateWithCurrentMetadata = shouldUpdateWithCurrentMetadata;
}
@Override
public void disconnect(String node) {
disconnect(node, false);
@ -329,8 +337,10 @@ public class MockClient implements KafkaClient {
MetadataUpdate metadataUpdate = metadataUpdates.poll();
if (metadataUpdate != null) {
metadataUpdater.update(time, metadataUpdate);
} else {
} else if (shouldUpdateWithCurrentMetadata) {
metadataUpdater.updateWithCurrentMetadata(time);
} else {
isMetadataUpdateNeeded.countDown();
}
}
@ -350,6 +360,14 @@ public class MockClient implements KafkaClient {
return copy;
}
public boolean awaitMetadataUpdateRequest(long timeoutMs) throws InterruptedException {
if (isMetadataUpdateNeeded.await(timeoutMs, TimeUnit.MILLISECONDS)) {
isMetadataUpdateNeeded = new CountDownLatch(1);
return true;
}
return false;
}
private long elapsedTimeMs(long currentTimeMs, long startTimeMs) {
return Math.max(0, currentTimeMs - startTimeMs);
}
@ -523,6 +541,10 @@ public class MockClient implements KafkaClient {
metadataUpdates.add(new MetadataUpdate(updateResponse, expectMatchMetadataTopics));
}
public int preparedMetadataUpdatesCount() {
return metadataUpdates.size();
}
public void updateMetadata(MetadataResponse updateResponse) {
metadataUpdater.update(time, new MetadataUpdate(updateResponse, false));
}

View File

@ -19,45 +19,323 @@ 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.MetricConfig;
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.AfterEach;
import org.junit.jupiter.api.BeforeEach;
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.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.Mockito.mock;
class ConsumerRebalanceMetricsManagerTest {
private final Time time = new MockTime();
private final Metrics metrics = new Metrics(time);
private Time time;
private Metrics metrics;
private SubscriptionState subscriptionState;
private ConsumerRebalanceMetricsManager metricsManager;
private MetricConfig metricConfig;
private long windowSizeMs;
private int numSamples;
@BeforeEach
public void setUp() {
time = new MockTime();
// Use MetricConfig with its default values
windowSizeMs = 30000; // 30 seconds - default value
numSamples = 2; // default value
metricConfig = new MetricConfig()
.samples(numSamples)
.timeWindow(windowSizeMs, java.util.concurrent.TimeUnit.MILLISECONDS);
metrics = new Metrics(metricConfig, time);
subscriptionState = new SubscriptionState(mock(LogContext.class), AutoOffsetResetStrategy.EARLIEST);
metricsManager = new ConsumerRebalanceMetricsManager(metrics, subscriptionState);
}
@AfterEach
public void tearDown() {
metrics.close();
}
@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");
assertNotNull(metrics.metric(metricsManager.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());
assertEquals(2.0d, metrics.metric(metricsManager.assignedPartitionsCount).metricValue());
subscriptionState.assignFromUser(Set.of());
assertEquals(0.0d, metrics.metric(consumerRebalanceMetricsManager.assignedPartitionsCount).metricValue());
assertEquals(0.0d, metrics.metric(metricsManager.assignedPartitionsCount).metricValue());
subscriptionState.unsubscribe();
assertEquals(0.0d, metrics.metric(consumerRebalanceMetricsManager.assignedPartitionsCount).metricValue());
assertEquals(0.0d, metrics.metric(metricsManager.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());
assertEquals(1.0d, metrics.metric(metricsManager.assignedPartitionsCount).metricValue());
}
@Test
public void testRebalanceTimingMetrics() {
// Verify timing metrics are registered
assertNotNull(metrics.metric(metricsManager.rebalanceLatencyAvg));
assertNotNull(metrics.metric(metricsManager.rebalanceLatencyMax));
assertNotNull(metrics.metric(metricsManager.rebalanceLatencyTotal));
assertNotNull(metrics.metric(metricsManager.rebalanceTotal));
// Record first rebalance (10ms duration)
metricsManager.recordRebalanceStarted(time.milliseconds());
time.sleep(10);
metricsManager.recordRebalanceEnded(time.milliseconds());
// Verify metrics after first rebalance
assertEquals(10.0d, metrics.metric(metricsManager.rebalanceLatencyAvg).metricValue());
assertEquals(10.0d, metrics.metric(metricsManager.rebalanceLatencyMax).metricValue());
assertEquals(10.0d, metrics.metric(metricsManager.rebalanceLatencyTotal).metricValue());
assertEquals(1.0d, metrics.metric(metricsManager.rebalanceTotal).metricValue());
// Record second rebalance (30ms duration)
metricsManager.recordRebalanceStarted(time.milliseconds());
time.sleep(30);
metricsManager.recordRebalanceEnded(time.milliseconds());
// Verify metrics after second rebalance
assertEquals(20.0d, metrics.metric(metricsManager.rebalanceLatencyAvg).metricValue(),
"Average latency should be (10 + 30) / 2 = 20ms");
assertEquals(30.0d, metrics.metric(metricsManager.rebalanceLatencyMax).metricValue(),
"Max latency should be max(10, 30) = 30ms");
assertEquals(40.0d, metrics.metric(metricsManager.rebalanceLatencyTotal).metricValue(),
"Total latency should be 10 + 30 = 40ms");
assertEquals(2.0d, metrics.metric(metricsManager.rebalanceTotal).metricValue());
// Record third rebalance (50ms duration)
metricsManager.recordRebalanceStarted(time.milliseconds());
time.sleep(50);
metricsManager.recordRebalanceEnded(time.milliseconds());
// Verify metrics after third rebalance
assertEquals(30.0d, metrics.metric(metricsManager.rebalanceLatencyAvg).metricValue(),
"Average latency should be (10 + 30 + 50) / 3 = 30ms");
assertEquals(50.0d, metrics.metric(metricsManager.rebalanceLatencyMax).metricValue(),
"Max latency should be max(10, 30, 50) = 50ms");
assertEquals(90.0d, metrics.metric(metricsManager.rebalanceLatencyTotal).metricValue(),
"Total latency should be 10 + 30 + 50 = 90ms");
assertEquals(3.0d, metrics.metric(metricsManager.rebalanceTotal).metricValue());
}
@Test
public void testRebalanceRateMetric() {
// Verify rate metric is registered
assertNotNull(metrics.metric(metricsManager.rebalanceRatePerHour));
// Record 3 rebalances within 30ms total (3 x 10ms)
int rebalanceCount = 3;
long startTime = time.milliseconds();
for (int i = 0; i < rebalanceCount; i++) {
metricsManager.recordRebalanceStarted(time.milliseconds());
time.sleep(10);
metricsManager.recordRebalanceEnded(time.milliseconds());
}
long endTime = time.milliseconds();
long actualElapsedMs = endTime - startTime;
double ratePerHour = (Double) metrics.metric(metricsManager.rebalanceRatePerHour).metricValue();
// The Rate metric calculation:
// - Uses elapsed time from the oldest sample
// - Ensures minimum window size of (numSamples - 1) * windowSizeMs
// - With default config: minWindow = (2-1) * 30000 = 30000ms
long minWindowMs = (numSamples - 1) * windowSizeMs; // (2-1) * 30000 = 30000ms
// Since actualElapsedMs (30ms) is much less than minWindowMs (30000ms),
// the rate calculation will use minWindowMs as the window
// Rate per hour = count / (windowMs / 1000) * 3600
double expectedRatePerHour = (double) rebalanceCount / (minWindowMs / 1000.0) * 3600.0;
assertEquals(expectedRatePerHour, ratePerHour, 1.0,
String.format("With %d rebalances in %dms, min window %dms: expecting %.1f rebalances/hour",
rebalanceCount, actualElapsedMs, minWindowMs, expectedRatePerHour));
}
@Test
public void testFailedRebalanceMetrics() {
// Verify failed rebalance metrics are registered
assertNotNull(metrics.metric(metricsManager.failedRebalanceTotal));
assertNotNull(metrics.metric(metricsManager.failedRebalanceRate));
assertEquals(0.0d, metrics.metric(metricsManager.failedRebalanceTotal).metricValue(),
"Initially, there should be no failed rebalances");
// Start a rebalance but don't complete it
metricsManager.recordRebalanceStarted(time.milliseconds());
time.sleep(10);
metricsManager.maybeRecordRebalanceFailed();
assertEquals(1.0d, metrics.metric(metricsManager.failedRebalanceTotal).metricValue(),
"Failed rebalance count should increment to 1 after recording failure");
// Complete a successful rebalance
metricsManager.recordRebalanceStarted(time.milliseconds());
time.sleep(10);
metricsManager.recordRebalanceEnded(time.milliseconds());
metricsManager.maybeRecordRebalanceFailed();
assertEquals(1.0d, metrics.metric(metricsManager.failedRebalanceTotal).metricValue(),
"Failed count should not increment after successful rebalance completes");
// Start another rebalance, don't complete it, then record failure
time.sleep(10);
metricsManager.recordRebalanceStarted(time.milliseconds());
assertTrue(metricsManager.rebalanceStarted(), "Rebalance should be in progress");
time.sleep(10);
// Don't call recordRebalanceEnded() to simulate an incomplete rebalance
metricsManager.maybeRecordRebalanceFailed();
assertEquals(2.0d, metrics.metric(metricsManager.failedRebalanceTotal).metricValue());
double failedRate = (Double) metrics.metric(metricsManager.failedRebalanceRate).metricValue();
// Calculate expected failed rate based on Rate metric behavior
// We had 2 failures over ~40ms, but minimum window is (numSamples - 1) * windowSizeMs
long minWindowMs = (numSamples - 1) * windowSizeMs; // (2-1) * 30000 = 30000ms
double expectedFailedRatePerHour = 2.0 / (minWindowMs / 1000.0) * 3600.0;
assertEquals(expectedFailedRatePerHour, failedRate, 1.0,
String.format("With 2 failures, min window %dms: expecting %.1f failures/hour",
minWindowMs, expectedFailedRatePerHour));
}
@Test
public void testLastRebalanceSecondsAgoMetric() {
// Verify metric is registered
assertNotNull(metrics.metric(metricsManager.lastRebalanceSecondsAgo));
assertEquals(-1.0d, metrics.metric(metricsManager.lastRebalanceSecondsAgo).metricValue(),
"Should return -1 when no rebalance has occurred");
// Complete a rebalance
metricsManager.recordRebalanceStarted(time.milliseconds());
time.sleep(10);
metricsManager.recordRebalanceEnded(time.milliseconds());
assertEquals(0.0d, metrics.metric(metricsManager.lastRebalanceSecondsAgo).metricValue(),
"Should return 0 immediately after rebalance completes");
// Advance time by 5 seconds
time.sleep(5000);
assertEquals(5.0d, metrics.metric(metricsManager.lastRebalanceSecondsAgo).metricValue());
// Advance time by another 10 seconds
time.sleep(10000);
assertEquals(15.0d, metrics.metric(metricsManager.lastRebalanceSecondsAgo).metricValue());
// Complete another rebalance
metricsManager.recordRebalanceStarted(time.milliseconds());
time.sleep(20);
metricsManager.recordRebalanceEnded(time.milliseconds());
assertEquals(0.0d, metrics.metric(metricsManager.lastRebalanceSecondsAgo).metricValue(),
"Should reset to 0 after a new rebalance completes");
}
@Test
public void testRebalanceStartedFlag() {
assertFalse(metricsManager.rebalanceStarted(),
"Initially, no rebalance should be in progress");
metricsManager.recordRebalanceStarted(time.milliseconds());
assertTrue(metricsManager.rebalanceStarted(),
"Rebalance should be marked as started after recordRebalanceStarted()");
time.sleep(10);
metricsManager.recordRebalanceEnded(time.milliseconds());
assertFalse(metricsManager.rebalanceStarted(),
"Rebalance should not be in progress after recordRebalanceEnded()");
// Start another rebalance - advance time first
time.sleep(100);
metricsManager.recordRebalanceStarted(time.milliseconds());
assertTrue(metricsManager.rebalanceStarted(),
"New rebalance should be marked as started");
}
@Test
public void testMultipleConsecutiveFailures() {
// Record multiple consecutive failures
for (int i = 0; i < 5; i++) {
metricsManager.recordRebalanceStarted(time.milliseconds());
time.sleep(10);
metricsManager.maybeRecordRebalanceFailed();
}
assertEquals(5.0d, metrics.metric(metricsManager.failedRebalanceTotal).metricValue(),
"Should have recorded 5 consecutive failed rebalances");
assertEquals(0.0d, metrics.metric(metricsManager.rebalanceTotal).metricValue(),
"Successful rebalance count should remain 0 when only failures occur");
}
@Test
public void testMixedSuccessAndFailureScenarios() {
// Success -> Failure -> Success -> Failure pattern
// First success
metricsManager.recordRebalanceStarted(time.milliseconds());
time.sleep(20);
metricsManager.recordRebalanceEnded(time.milliseconds());
assertEquals(1.0d, metrics.metric(metricsManager.rebalanceTotal).metricValue());
// First failure
time.sleep(10);
metricsManager.recordRebalanceStarted(time.milliseconds());
assertTrue(metricsManager.rebalanceStarted(), "First failure rebalance should be in progress");
time.sleep(30);
metricsManager.maybeRecordRebalanceFailed();
double failedAfterFirst = (Double) metrics.metric(metricsManager.failedRebalanceTotal).metricValue();
assertEquals(1.0d, failedAfterFirst, "Should have recorded one failed rebalance after first failure");
// Second success
time.sleep(10);
metricsManager.recordRebalanceStarted(time.milliseconds());
time.sleep(40);
metricsManager.recordRebalanceEnded(time.milliseconds());
assertEquals(2.0d, metrics.metric(metricsManager.rebalanceTotal).metricValue());
// Second failure
time.sleep(10);
metricsManager.recordRebalanceStarted(time.milliseconds());
assertTrue(metricsManager.rebalanceStarted(), "Second failure rebalance should be in progress");
time.sleep(50);
metricsManager.maybeRecordRebalanceFailed();
assertEquals(2.0d, metrics.metric(metricsManager.rebalanceTotal).metricValue(),
"Should have 2 successful rebalances in mixed scenario");
assertEquals(2.0d, metrics.metric(metricsManager.failedRebalanceTotal).metricValue(),
"Should have 2 failed rebalances in mixed scenario");
assertEquals(30.0d, metrics.metric(metricsManager.rebalanceLatencyAvg).metricValue(),
"Average latency should only include successful rebalances: (20 + 40) / 2 = 30ms");
assertEquals(40.0d, metrics.metric(metricsManager.rebalanceLatencyMax).metricValue(),
"Max latency should be 40ms from successful rebalances only");
assertEquals(60.0d, metrics.metric(metricsManager.rebalanceLatencyTotal).metricValue(),
"Total latency should only include successful rebalances: 20 + 40 = 60ms");
}
}

View File

@ -128,13 +128,11 @@ import java.util.Map;
import java.util.Optional;
import java.util.Properties;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Exchanger;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
@ -961,10 +959,9 @@ public class KafkaProducerTest {
@Test
public void testTopicRefreshInMetadata() throws InterruptedException {
Map<String, Object> configs = new HashMap<>();
final String maxBlockMs = "600000";
configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999");
configs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, "600000");
// test under normal producer for simplicity
configs.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, false);
configs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, maxBlockMs);
long refreshBackoffMs = 500L;
long refreshBackoffMaxMs = 5000L;
long metadataExpireMs = 60000L;
@ -972,35 +969,35 @@ public class KafkaProducerTest {
final Time time = new MockTime();
final ProducerMetadata metadata = new ProducerMetadata(refreshBackoffMs, refreshBackoffMaxMs, metadataExpireMs, metadataIdleMs,
new LogContext(), new ClusterResourceListeners(), time);
final String warmupTopic = "warmup-topic";
final String topic = "topic";
MockClient client = new MockClient(time, metadata);
client.setShouldUpdateWithCurrentMetadata(false);
client.advanceTimeDuringPoll(true);
try (KafkaProducer<String, String> producer = kafkaProducer(configs,
new StringSerializer(), new StringSerializer(), metadata, new MockClient(time, metadata), null, time)) {
new StringSerializer(), new StringSerializer(), metadata, client, null, time)) {
AtomicBoolean running = new AtomicBoolean(true);
Thread t = new Thread(() -> {
long startTimeMs = System.currentTimeMillis();
while (running.get()) {
while (!metadata.updateRequested() && System.currentTimeMillis() - startTimeMs < 100)
Thread.yield();
MetadataResponse updateResponse = RequestTestUtils.metadataUpdateWith("kafka-cluster", 1,
singletonMap(topic, Errors.UNKNOWN_TOPIC_OR_PARTITION), emptyMap());
metadata.updateWithCurrentRequestVersion(updateResponse, false, time.milliseconds());
time.sleep(60 * 1000L);
}
});
t.start();
MetadataResponse updateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap(warmupTopic, 1));
client.updateMetadata(updateResponse);
final int preparedUpdatesCount = 100;
updateResponse = RequestTestUtils.metadataUpdateWith("kafka-cluster", 1,
singletonMap(topic, Errors.UNKNOWN_TOPIC_OR_PARTITION), emptyMap());
for (int i = 0; i < preparedUpdatesCount; i++) {
client.prepareMetadataUpdate(updateResponse);
}
Throwable throwable = assertThrows(TimeoutException.class, () -> producer.partitionsFor(topic));
assertInstanceOf(UnknownTopicOrPartitionException.class, throwable.getCause());
running.set(false);
t.join();
assertTrue(preparedUpdatesCount > client.preparedMetadataUpdatesCount());
}
}
@Test
public void testTopicNotExistingInMetadata() throws InterruptedException {
public void testTopicNotExistingInMetadata() {
Map<String, Object> configs = new HashMap<>();
final String maxBlockMs = "30000";
configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999");
configs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, "30000");
configs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, maxBlockMs);
long refreshBackoffMs = 500L;
long refreshBackoffMaxMs = 5000L;
long metadataExpireMs = 60000L;
@ -1008,39 +1005,33 @@ public class KafkaProducerTest {
final Time time = new MockTime();
final ProducerMetadata metadata = new ProducerMetadata(refreshBackoffMs, refreshBackoffMaxMs, metadataExpireMs, metadataIdleMs,
new LogContext(), new ClusterResourceListeners(), time);
final String warmupTopic = "warmup-topic";
final String topic = "topic";
MockClient client = new MockClient(time, metadata);
client.setShouldUpdateWithCurrentMetadata(false);
client.advanceTimeDuringPoll(true);
try (KafkaProducer<String, String> producer = kafkaProducer(configs, new StringSerializer(),
new StringSerializer(), metadata, new MockClient(time, metadata), null, time)) {
new StringSerializer(), metadata, client, null, time)) {
Exchanger<Void> exchanger = new Exchanger<>();
MetadataResponse updateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap(warmupTopic, 1));
client.updateMetadata(updateResponse);
updateResponse = RequestTestUtils.metadataUpdateWith("kafka-cluster", 1,
singletonMap(topic, Errors.UNKNOWN_TOPIC_OR_PARTITION), emptyMap());
client.prepareMetadataUpdate(updateResponse);
Thread t = new Thread(() -> {
try {
// Update the metadata with non-existing topic.
MetadataResponse updateResponse = RequestTestUtils.metadataUpdateWith("kafka-cluster", 1,
singletonMap(topic, Errors.UNKNOWN_TOPIC_OR_PARTITION), emptyMap());
metadata.updateWithCurrentRequestVersion(updateResponse, false, time.milliseconds());
exchanger.exchange(null);
while (!metadata.updateRequested())
Thread.sleep(100);
time.sleep(30 * 1000L);
} catch (Exception e) {
throw new RuntimeException(e);
}
});
t.start();
exchanger.exchange(null);
Throwable throwable = assertThrows(TimeoutException.class, () -> producer.partitionsFor(topic));
assertInstanceOf(UnknownTopicOrPartitionException.class, throwable.getCause());
t.join();
assertEquals(0, client.preparedMetadataUpdatesCount());
}
}
@Test
public void testTopicExpiryInMetadata() throws InterruptedException {
Map<String, Object> configs = new HashMap<>();
final String maxBlockMs = "300000";
configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999");
configs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, "30000");
configs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, maxBlockMs);
long refreshBackoffMs = 500L;
long refreshBackoffMaxMs = 5000L;
long metadataExpireMs = 60000L;
@ -1049,39 +1040,28 @@ public class KafkaProducerTest {
final ProducerMetadata metadata = new ProducerMetadata(refreshBackoffMs, refreshBackoffMaxMs, metadataExpireMs, metadataIdleMs,
new LogContext(), new ClusterResourceListeners(), time);
final String topic = "topic";
final String warmupTopic = "warmup-topic";
MockClient client = new MockClient(time, metadata);
client.setShouldUpdateWithCurrentMetadata(false);
client.advanceTimeDuringPoll(true);
try (KafkaProducer<String, String> producer = kafkaProducer(configs, new StringSerializer(),
new StringSerializer(), metadata, new MockClient(time, metadata), null, time)) {
new StringSerializer(), metadata, client, null, time)) {
Exchanger<Void> exchanger = new Exchanger<>();
MetadataResponse updateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap(warmupTopic, 1));
client.updateMetadata(updateResponse);
Thread t = new Thread(() -> {
try {
exchanger.exchange(null); // 1
while (!metadata.updateRequested())
Thread.sleep(100);
MetadataResponse updateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap(topic, 1));
metadata.updateWithCurrentRequestVersion(updateResponse, false, time.milliseconds());
exchanger.exchange(null); // 2
time.sleep(120 * 1000L);
// Update the metadata again, but it should be expired at this point.
updateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap(topic, 1));
metadata.updateWithCurrentRequestVersion(updateResponse, false, time.milliseconds());
exchanger.exchange(null); // 3
while (!metadata.updateRequested())
Thread.sleep(100);
time.sleep(30 * 1000L);
} catch (Exception e) {
throw new RuntimeException(e);
}
});
t.start();
exchanger.exchange(null); // 1
updateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap(topic, 1));
client.prepareMetadataUpdate(updateResponse);
assertNotNull(producer.partitionsFor(topic));
exchanger.exchange(null); // 2
exchanger.exchange(null); // 3
// Update the metadata again, but it should be expired at this point.
time.sleep(120 * 1000L);
updateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap(warmupTopic, 1));
client.updateMetadata(updateResponse);
assertFalse(client.awaitMetadataUpdateRequest(0));
assertThrows(TimeoutException.class, () -> producer.partitionsFor(topic));
t.join();
assertTrue(client.awaitMetadataUpdateRequest(0));
}
}

View File

@ -233,13 +233,13 @@ public class MirrorMaker {
private void addHerder(SourceAndTarget sourceAndTarget) {
log.info("creating herder for {}", sourceAndTarget.toString());
Map<String, String> workerProps = config.workerConfig(sourceAndTarget);
DistributedConfig distributedConfig = new DistributedConfig(workerProps);
String encodedSource = encodePath(sourceAndTarget.source());
String encodedTarget = encodePath(sourceAndTarget.target());
List<String> restNamespace = List.of(encodedSource, encodedTarget);
String workerId = generateWorkerId(sourceAndTarget);
Plugins plugins = new Plugins(workerProps);
plugins.compareAndSwapWithDelegatingLoader();
DistributedConfig distributedConfig = new DistributedConfig(workerProps);
String kafkaClusterId = distributedConfig.kafkaClusterId();
String clientIdBase = ConnectUtils.clientIdBase(distributedConfig);
// Create the admin client to be shared by all backing stores for this herder

View File

@ -114,14 +114,15 @@ public abstract class AbstractConnectCli<H extends Herder, T extends WorkerConfi
log.info("Kafka Connect worker initializing ...");
long initStart = time.hiResClockMs();
T config = createConfig(workerProps);
log.debug("Kafka cluster ID: {}", config.kafkaClusterId());
WorkerInfo initInfo = new WorkerInfo();
initInfo.logAll();
log.info("Scanning for plugin classes. This might take a moment ...");
Plugins plugins = new Plugins(workerProps);
plugins.compareAndSwapWithDelegatingLoader();
T config = createConfig(workerProps);
log.debug("Kafka cluster ID: {}", config.kafkaClusterId());
RestClient restClient = new RestClient(config);

View File

@ -20,7 +20,6 @@ import java.lang.{Long => JLong}
import java.util.concurrent.locks.ReentrantReadWriteLock
import java.util.Optional
import java.util.concurrent.{CompletableFuture, ConcurrentHashMap, CopyOnWriteArrayList}
import kafka.controller.StateChangeLogger
import kafka.log._
import kafka.server._
import kafka.server.share.DelayedShareFetch
@ -37,6 +36,7 @@ import org.apache.kafka.common.record.{FileRecords, MemoryRecords, RecordBatch}
import org.apache.kafka.common.requests._
import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET}
import org.apache.kafka.common.utils.Time
import org.apache.kafka.logger.StateChangeLogger
import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState, MetadataCache, PartitionRegistration}
import org.apache.kafka.server.common.RequestLocal
import org.apache.kafka.server.log.remote.TopicPartitionLog
@ -322,7 +322,7 @@ class Partition(val topicPartition: TopicPartition,
def topic: String = topicPartition.topic
def partitionId: Int = topicPartition.partition
private val stateChangeLogger = new StateChangeLogger(localBrokerId, inControllerContext = false, None)
private val stateChangeLogger = new StateChangeLogger(localBrokerId)
private val remoteReplicasMap = new ConcurrentHashMap[Int, Replica]
// The read lock is only required when multiple reads are executed and needs to be in a consistent manner
private val leaderIsrUpdateLock = new ReentrantReadWriteLock

View File

@ -1,45 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.controller
import com.typesafe.scalalogging.Logger
import kafka.utils.Logging
object StateChangeLogger {
private val logger = Logger("state.change.logger")
}
/**
* Simple class that sets `logIdent` appropriately depending on whether the state change logger is being used in the
* context of the KafkaController or not (e.g. ReplicaManager and MetadataCache log to the state change logger
* irrespective of whether the broker is the Controller).
*/
class StateChangeLogger(brokerId: Int, inControllerContext: Boolean, controllerEpoch: Option[Int]) extends Logging {
if (controllerEpoch.isDefined && !inControllerContext)
throw new IllegalArgumentException("Controller epoch should only be defined if inControllerContext is true")
override lazy val logger: Logger = StateChangeLogger.logger
locally {
val prefix = if (inControllerContext) "Controller" else "Broker"
val epochEntry = controllerEpoch.fold("")(epoch => s" epoch=$epoch")
logIdent = s"[$prefix id=$brokerId$epochEntry] "
}
}

View File

@ -18,7 +18,6 @@ package kafka.server
import com.yammer.metrics.core.Meter
import kafka.cluster.{Partition, PartitionListener}
import kafka.controller.StateChangeLogger
import kafka.log.LogManager
import kafka.server.HostedPartition.Online
import kafka.server.QuotaFactory.QuotaManagers
@ -48,6 +47,7 @@ import org.apache.kafka.common.requests._
import org.apache.kafka.common.utils.{Exit, Time, Utils}
import org.apache.kafka.coordinator.transaction.{AddPartitionsToTxnConfig, TransactionLogConfig}
import org.apache.kafka.image.{LocalReplicaChanges, MetadataImage, TopicsDelta}
import org.apache.kafka.logger.StateChangeLogger
import org.apache.kafka.metadata.LeaderConstants.NO_LEADER
import org.apache.kafka.metadata.MetadataCache
import org.apache.kafka.server.common.{DirectoryEventHandler, RequestLocal, StopPartition}
@ -272,7 +272,7 @@ class ReplicaManager(val config: KafkaConfig,
@volatile private var isInControlledShutdown = false
this.logIdent = s"[ReplicaManager broker=$localBrokerId] "
protected val stateChangeLogger = new StateChangeLogger(localBrokerId, inControllerContext = false, None)
protected val stateChangeLogger = new StateChangeLogger(localBrokerId)
private var logDirFailureHandler: LogDirFailureHandler = _
@ -789,9 +789,9 @@ class ReplicaManager(val config: KafkaConfig,
hasCustomErrorMessage = customException.isDefined
)
}
// In non-transaction paths, errorResults is typically empty, so we can
// In non-transaction paths, errorResults is typically empty, so we can
// directly use entriesPerPartition instead of creating a new filtered collection
val entriesWithoutErrorsPerPartition =
val entriesWithoutErrorsPerPartition =
if (errorResults.nonEmpty) entriesPerPartition.filter { case (key, _) => !errorResults.contains(key) }
else entriesPerPartition
@ -1637,13 +1637,13 @@ class ReplicaManager(val config: KafkaConfig,
remoteFetchPartitionStatus: Seq[(TopicIdPartition, FetchPartitionStatus)]): Unit = {
val remoteFetchTasks = new util.HashMap[TopicIdPartition, Future[Void]]
val remoteFetchResults = new util.HashMap[TopicIdPartition, CompletableFuture[RemoteLogReadResult]]
remoteFetchInfos.forEach { (topicIdPartition, remoteFetchInfo) =>
val (task, result) = processRemoteFetch(remoteFetchInfo)
remoteFetchTasks.put(topicIdPartition, task)
remoteFetchResults.put(topicIdPartition, result)
}
val remoteFetchMaxWaitMs = config.remoteLogManagerConfig.remoteFetchMaxWaitMs().toLong
val remoteFetch = new DelayedRemoteFetch(remoteFetchTasks, remoteFetchResults, remoteFetchInfos, remoteFetchMaxWaitMs,
remoteFetchPartitionStatus, params, logReadResults, this, responseCallback)

View File

@ -26,9 +26,9 @@ import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity
import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse
import org.apache.kafka.common.message.SyncGroupRequestData.SyncGroupRequestAssignment
import org.apache.kafka.common.message.WriteTxnMarkersRequestData.{WritableTxnMarker, WritableTxnMarkerTopic}
import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AddOffsetsToTxnResponseData, ConsumerGroupDescribeRequestData, ConsumerGroupDescribeResponseData, ConsumerGroupHeartbeatRequestData, ConsumerGroupHeartbeatResponseData, DeleteGroupsRequestData, DeleteGroupsResponseData, DescribeGroupsRequestData, DescribeGroupsResponseData, EndTxnRequestData, HeartbeatRequestData, HeartbeatResponseData, InitProducerIdRequestData, JoinGroupRequestData, JoinGroupResponseData, LeaveGroupResponseData, ListGroupsRequestData, ListGroupsResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteRequestData, OffsetDeleteResponseData, OffsetFetchRequestData, OffsetFetchResponseData, ShareGroupDescribeRequestData, ShareGroupDescribeResponseData, ShareGroupHeartbeatRequestData, ShareGroupHeartbeatResponseData, SyncGroupRequestData, SyncGroupResponseData, TxnOffsetCommitRequestData, TxnOffsetCommitResponseData, WriteTxnMarkersRequestData}
import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AddOffsetsToTxnResponseData, ConsumerGroupDescribeRequestData, ConsumerGroupDescribeResponseData, ConsumerGroupHeartbeatRequestData, ConsumerGroupHeartbeatResponseData, DeleteGroupsRequestData, DeleteGroupsResponseData, DescribeGroupsRequestData, DescribeGroupsResponseData, EndTxnRequestData, HeartbeatRequestData, HeartbeatResponseData, InitProducerIdRequestData, JoinGroupRequestData, JoinGroupResponseData, LeaveGroupResponseData, ListGroupsRequestData, ListGroupsResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteRequestData, OffsetDeleteResponseData, OffsetFetchRequestData, OffsetFetchResponseData, ShareGroupDescribeRequestData, ShareGroupDescribeResponseData, ShareGroupHeartbeatRequestData, ShareGroupHeartbeatResponseData, StreamsGroupDescribeRequestData, StreamsGroupDescribeResponseData, StreamsGroupHeartbeatRequestData, StreamsGroupHeartbeatResponseData, SyncGroupRequestData, SyncGroupResponseData, TxnOffsetCommitRequestData, TxnOffsetCommitResponseData, WriteTxnMarkersRequestData}
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, AddOffsetsToTxnRequest, AddOffsetsToTxnResponse, ConsumerGroupDescribeRequest, ConsumerGroupDescribeResponse, ConsumerGroupHeartbeatRequest, ConsumerGroupHeartbeatResponse, DeleteGroupsRequest, DeleteGroupsResponse, DescribeGroupsRequest, DescribeGroupsResponse, EndTxnRequest, EndTxnResponse, HeartbeatRequest, HeartbeatResponse, InitProducerIdRequest, InitProducerIdResponse, JoinGroupRequest, JoinGroupResponse, LeaveGroupRequest, LeaveGroupResponse, ListGroupsRequest, ListGroupsResponse, OffsetCommitRequest, OffsetCommitResponse, OffsetDeleteRequest, OffsetDeleteResponse, OffsetFetchRequest, OffsetFetchResponse, ShareGroupDescribeRequest, ShareGroupDescribeResponse, ShareGroupHeartbeatRequest, ShareGroupHeartbeatResponse, SyncGroupRequest, SyncGroupResponse, TxnOffsetCommitRequest, TxnOffsetCommitResponse, WriteTxnMarkersRequest, WriteTxnMarkersResponse}
import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, AddOffsetsToTxnRequest, AddOffsetsToTxnResponse, ConsumerGroupDescribeRequest, ConsumerGroupDescribeResponse, ConsumerGroupHeartbeatRequest, ConsumerGroupHeartbeatResponse, DeleteGroupsRequest, DeleteGroupsResponse, DescribeGroupsRequest, DescribeGroupsResponse, EndTxnRequest, EndTxnResponse, HeartbeatRequest, HeartbeatResponse, InitProducerIdRequest, InitProducerIdResponse, JoinGroupRequest, JoinGroupResponse, LeaveGroupRequest, LeaveGroupResponse, ListGroupsRequest, ListGroupsResponse, OffsetCommitRequest, OffsetCommitResponse, OffsetDeleteRequest, OffsetDeleteResponse, OffsetFetchRequest, OffsetFetchResponse, ShareGroupDescribeRequest, ShareGroupDescribeResponse, ShareGroupHeartbeatRequest, ShareGroupHeartbeatResponse, StreamsGroupDescribeRequest, StreamsGroupDescribeResponse, StreamsGroupHeartbeatRequest, StreamsGroupHeartbeatResponse, SyncGroupRequest, SyncGroupResponse, TxnOffsetCommitRequest, TxnOffsetCommitResponse, WriteTxnMarkersRequest, WriteTxnMarkersResponse}
import org.apache.kafka.common.serialization.StringSerializer
import org.apache.kafka.common.test.ClusterInstance
import org.apache.kafka.common.utils.ProducerIdAndEpoch
@ -768,6 +768,21 @@ class GroupCoordinatorBaseRequestTest(cluster: ClusterInstance) {
shareGroupDescribeResponse.data.groups.asScala.toList
}
protected def streamsGroupDescribe(
groupIds: List[String],
includeAuthorizedOperations: Boolean = false,
version: Short = ApiKeys.STREAMS_GROUP_DESCRIBE.latestVersion(isUnstableApiEnabled)
): List[StreamsGroupDescribeResponseData.DescribedGroup] = {
val streamsGroupDescribeRequest = new StreamsGroupDescribeRequest.Builder(
new StreamsGroupDescribeRequestData()
.setGroupIds(groupIds.asJava)
.setIncludeAuthorizedOperations(includeAuthorizedOperations)
).build(version)
val streamsGroupDescribeResponse = connectAndReceive[StreamsGroupDescribeResponse](streamsGroupDescribeRequest)
streamsGroupDescribeResponse.data.groups.asScala.toList
}
protected def heartbeat(
groupId: String,
generationId: Int,
@ -855,6 +870,41 @@ class GroupCoordinatorBaseRequestTest(cluster: ClusterInstance) {
shareGroupHeartbeatResponse.data
}
protected def streamsGroupHeartbeat(
groupId: String,
memberId: String = "",
memberEpoch: Int = 0,
rebalanceTimeoutMs: Int = -1,
activeTasks: List[StreamsGroupHeartbeatRequestData.TaskIds] = null,
standbyTasks: List[StreamsGroupHeartbeatRequestData.TaskIds] = null,
warmupTasks: List[StreamsGroupHeartbeatRequestData.TaskIds] = null,
topology: StreamsGroupHeartbeatRequestData.Topology = null,
expectedError: Errors = Errors.NONE,
version: Short = ApiKeys.STREAMS_GROUP_HEARTBEAT.latestVersion(isUnstableApiEnabled)
): StreamsGroupHeartbeatResponseData = {
val streamsGroupHeartbeatRequest = new StreamsGroupHeartbeatRequest.Builder(
new StreamsGroupHeartbeatRequestData()
.setGroupId(groupId)
.setMemberId(memberId)
.setMemberEpoch(memberEpoch)
.setRebalanceTimeoutMs(rebalanceTimeoutMs)
.setActiveTasks(activeTasks.asJava)
.setStandbyTasks(standbyTasks.asJava)
.setWarmupTasks(warmupTasks.asJava)
.setTopology(topology)
).build(version)
// Send the request until receiving a successful response. There is a delay
// here because the group coordinator is loaded in the background.
var streamsGroupHeartbeatResponse: StreamsGroupHeartbeatResponse = null
TestUtils.waitUntilTrue(() => {
streamsGroupHeartbeatResponse = connectAndReceive[StreamsGroupHeartbeatResponse](streamsGroupHeartbeatRequest)
streamsGroupHeartbeatResponse.data.errorCode == expectedError.code
}, msg = s"Could not heartbeat successfully. Last response $streamsGroupHeartbeatResponse.")
streamsGroupHeartbeatResponse.data
}
protected def leaveGroupWithNewProtocol(
groupId: String,
memberId: String

View File

@ -0,0 +1,316 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.server
import kafka.utils.TestUtils
import org.apache.kafka.common.message.{StreamsGroupDescribeRequestData, StreamsGroupDescribeResponseData, StreamsGroupHeartbeatRequestData, StreamsGroupHeartbeatResponseData}
import org.apache.kafka.common.utils.Utils
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.requests.{StreamsGroupDescribeRequest, StreamsGroupDescribeResponse}
import org.apache.kafka.common.resource.ResourceType
import org.apache.kafka.common.test.ClusterInstance
import org.apache.kafka.common.test.api._
import scala.jdk.CollectionConverters._
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.security.authorizer.AclEntry
import org.apache.kafka.server.common.Feature
import org.junit.Assert.{assertEquals, assertTrue}
import java.lang.{Byte => JByte}
@ClusterTestDefaults(
types = Array(Type.KRAFT),
brokers = 1,
serverProperties = Array(
new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"),
new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "1")
)
)
class StreamsGroupDescribeRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) {
@ClusterTest(
features = Array(
new ClusterFeature(feature = Feature.STREAMS_VERSION, version = 0)
)
)
def testStreamsGroupDescribeWhenFeatureFlagNotEnabled(): Unit = {
val streamsGroupDescribeRequest = new StreamsGroupDescribeRequest.Builder(
new StreamsGroupDescribeRequestData().setGroupIds(List("grp-mock-1", "grp-mock-2").asJava)
).build(ApiKeys.STREAMS_GROUP_DESCRIBE.latestVersion(isUnstableApiEnabled))
val streamsGroupDescribeResponse = connectAndReceive[StreamsGroupDescribeResponse](streamsGroupDescribeRequest)
val expectedResponse = new StreamsGroupDescribeResponseData()
expectedResponse.groups().add(
new StreamsGroupDescribeResponseData.DescribedGroup()
.setGroupId("grp-mock-1")
.setErrorCode(Errors.UNSUPPORTED_VERSION.code)
)
expectedResponse.groups().add(
new StreamsGroupDescribeResponseData.DescribedGroup()
.setGroupId("grp-mock-2")
.setErrorCode(Errors.UNSUPPORTED_VERSION.code)
)
assertEquals(expectedResponse, streamsGroupDescribeResponse.data)
}
@ClusterTest(
serverProperties = Array(
new ClusterConfigProperty(key = GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = "classic,consumer,streams"),
new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"),
new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "1")
)
)
def testStreamsGroupDescribeGroupsWithNewGroupCoordinator(): Unit = {
// Creates the __consumer_offsets topics because it won't be created automatically
// in this test because it does not use FindCoordinator API.
createOffsetsTopic()
val admin = cluster.admin()
val topicName = "foo"
try {
TestUtils.createTopicWithAdminRaw(
admin = admin,
topic = topicName,
numPartitions = 3
)
TestUtils.waitUntilTrue(() => {
admin.listTopics().names().get().contains(topicName)
}, msg = s"Topic $topicName is not available to the group coordinator")
val timeoutMs = 5 * 60 * 1000
val clientId = "client-id"
val clientHost = "/127.0.0.1"
val authorizedOperationsInt = Utils.to32BitField(
AclEntry.supportedOperations(ResourceType.GROUP).asScala
.map(_.code.asInstanceOf[JByte]).asJava)
var grp1Member1Response: StreamsGroupHeartbeatResponseData = null
var grp1Member2Response: StreamsGroupHeartbeatResponseData = null
var grp2Member1Response: StreamsGroupHeartbeatResponseData = null
var grp2Member2Response: StreamsGroupHeartbeatResponseData = null
// grp-1 with 2 members
TestUtils.waitUntilTrue(() => {
grp1Member1Response = streamsGroupHeartbeat(
groupId = "grp-1",
memberId = "member-1",
rebalanceTimeoutMs = timeoutMs,
activeTasks = List.empty,
standbyTasks = List.empty,
warmupTasks = List.empty,
topology = new StreamsGroupHeartbeatRequestData.Topology()
.setEpoch(1)
.setSubtopologies(List(
new StreamsGroupHeartbeatRequestData.Subtopology()
.setSubtopologyId("subtopology-1")
.setSourceTopics(List(topicName).asJava)
.setRepartitionSinkTopics(List.empty.asJava)
.setRepartitionSourceTopics(List.empty.asJava)
.setStateChangelogTopics(List.empty.asJava)
).asJava)
)
grp1Member2Response = streamsGroupHeartbeat(
groupId = "grp-1",
memberId = "member-2",
rebalanceTimeoutMs = timeoutMs,
activeTasks = List.empty,
standbyTasks = List.empty,
warmupTasks = List.empty,
topology = new StreamsGroupHeartbeatRequestData.Topology()
.setEpoch(1)
.setSubtopologies(List(
new StreamsGroupHeartbeatRequestData.Subtopology()
.setSubtopologyId("subtopology-1")
.setSourceTopics(List(topicName).asJava)
.setRepartitionSinkTopics(List.empty.asJava)
.setRepartitionSourceTopics(List.empty.asJava)
.setStateChangelogTopics(List.empty.asJava)
).asJava)
)
val groupsDescription1 = streamsGroupDescribe(
groupIds = List("grp-1"),
includeAuthorizedOperations = true
)
grp1Member1Response.errorCode == Errors.NONE.code && grp1Member2Response.errorCode == Errors.NONE.code &&
groupsDescription1.size == 1 && groupsDescription1.head.members.size == 2
}, msg = s"Could not create grp-1 with 2 members successfully")
// grp-2 with 2 members
TestUtils.waitUntilTrue(() => {
grp2Member1Response = streamsGroupHeartbeat(
groupId = "grp-2",
memberId = "member-3",
rebalanceTimeoutMs = timeoutMs,
activeTasks = List.empty,
standbyTasks = List.empty,
warmupTasks = List.empty,
topology = new StreamsGroupHeartbeatRequestData.Topology()
.setEpoch(1)
.setSubtopologies(List(
new StreamsGroupHeartbeatRequestData.Subtopology()
.setSubtopologyId("subtopology-1")
.setSourceTopics(List(topicName).asJava)
.setRepartitionSinkTopics(List.empty.asJava)
.setRepartitionSourceTopics(List.empty.asJava)
.setStateChangelogTopics(List.empty.asJava)
).asJava)
)
grp2Member2Response = streamsGroupHeartbeat(
groupId = "grp-2",
memberId = "member-4",
rebalanceTimeoutMs = timeoutMs,
activeTasks = List.empty,
standbyTasks = List.empty,
warmupTasks = List.empty,
topology = new StreamsGroupHeartbeatRequestData.Topology()
.setEpoch(1)
.setSubtopologies(List(
new StreamsGroupHeartbeatRequestData.Subtopology()
.setSubtopologyId("subtopology-1")
.setSourceTopics(List(topicName).asJava)
.setRepartitionSinkTopics(List.empty.asJava)
.setRepartitionSourceTopics(List.empty.asJava)
.setStateChangelogTopics(List.empty.asJava)
).asJava)
)
val groupsDescription2 = streamsGroupDescribe(
groupIds = List("grp-2"),
includeAuthorizedOperations = true,
version = ApiKeys.STREAMS_GROUP_DESCRIBE.latestVersion(isUnstableApiEnabled).toShort
)
grp2Member1Response.errorCode == Errors.NONE.code && grp2Member2Response.errorCode == Errors.NONE.code &&
groupsDescription2.size == 1 && groupsDescription2.head.members.size == 2
}, msg = s"Could not create grp-2 with 2 members successfully")
// Send follow-up heartbeats until both groups are stable
TestUtils.waitUntilTrue(() => {
grp1Member1Response = streamsGroupHeartbeat(
groupId = "grp-1",
memberId = grp1Member1Response.memberId,
memberEpoch = grp1Member1Response.memberEpoch,
rebalanceTimeoutMs = timeoutMs,
activeTasks = convertTaskIds(grp1Member1Response.activeTasks),
standbyTasks = convertTaskIds(grp1Member1Response.standbyTasks),
warmupTasks = convertTaskIds(grp1Member1Response.warmupTasks),
topology = null
)
grp1Member2Response = streamsGroupHeartbeat(
groupId = "grp-1",
memberId = grp1Member2Response.memberId,
memberEpoch = grp1Member2Response.memberEpoch,
rebalanceTimeoutMs = timeoutMs,
activeTasks = convertTaskIds(grp1Member2Response.activeTasks),
standbyTasks = convertTaskIds(grp1Member2Response.standbyTasks),
warmupTasks = convertTaskIds(grp1Member2Response.warmupTasks),
topology = null
)
grp2Member1Response = streamsGroupHeartbeat(
groupId = "grp-2",
memberId = grp2Member1Response.memberId,
memberEpoch = grp2Member1Response.memberEpoch,
rebalanceTimeoutMs = timeoutMs,
activeTasks = convertTaskIds(grp2Member1Response.activeTasks),
standbyTasks = convertTaskIds(grp2Member1Response.standbyTasks),
warmupTasks = convertTaskIds(grp2Member1Response.warmupTasks),
topology = null
)
grp2Member2Response = streamsGroupHeartbeat(
groupId = "grp-2",
memberId = grp2Member2Response.memberId,
memberEpoch = grp2Member2Response.memberEpoch,
rebalanceTimeoutMs = timeoutMs,
activeTasks = convertTaskIds(grp2Member2Response.activeTasks),
standbyTasks = convertTaskIds(grp2Member2Response.standbyTasks),
warmupTasks = convertTaskIds(grp2Member2Response.warmupTasks),
topology = null
)
val actual = streamsGroupDescribe(
groupIds = List("grp-1","grp-2"),
includeAuthorizedOperations = true,
version = ApiKeys.STREAMS_GROUP_DESCRIBE.latestVersion(isUnstableApiEnabled).toShort
)
actual.head.groupState() == "Stable" && actual(1).groupState() == "Stable" &&
actual.head.members.size == 2 && actual(1).members.size == 2
}, "Two groups did not stabilize with 2 members each in time")
// Test the describe request for both groups in stable state
for (version <- ApiKeys.STREAMS_GROUP_DESCRIBE.oldestVersion() to ApiKeys.STREAMS_GROUP_DESCRIBE.latestVersion(isUnstableApiEnabled)) {
val actual = streamsGroupDescribe(
groupIds = List("grp-1","grp-2"),
includeAuthorizedOperations = true,
version = version.toShort
)
assertEquals(2, actual.size)
assertEquals(actual.map(_.groupId).toSet, Set("grp-1", "grp-2"))
for (describedGroup <- actual) {
assertEquals("Stable", describedGroup.groupState)
assertTrue("Group epoch is not equal to the assignment epoch", describedGroup.groupEpoch == describedGroup.assignmentEpoch)
// Verify topology
assertEquals(1, describedGroup.topology.epoch)
assertEquals(1, describedGroup.topology.subtopologies.size)
assertEquals("subtopology-1", describedGroup.topology.subtopologies.get(0).subtopologyId)
assertEquals(List(topicName).asJava, describedGroup.topology.subtopologies.get(0).sourceTopics)
// Verify members
assertEquals(2, describedGroup.members.size)
val expectedMemberIds = describedGroup.groupId match {
case "grp-1" => Set(grp1Member1Response.memberId, grp1Member2Response.memberId)
case "grp-2" => Set(grp2Member1Response.memberId, grp2Member2Response.memberId)
case unexpected => throw new AssertionError(s"Unexpected group ID: $unexpected")
}
val actualMemberIds = describedGroup.members.asScala.map(_.memberId).toSet
assertEquals(expectedMemberIds, actualMemberIds)
assertEquals(authorizedOperationsInt, describedGroup.authorizedOperations)
describedGroup.members.asScala.foreach { member =>
assertTrue("Group epoch is not equal to the member epoch", member.memberEpoch == describedGroup.assignmentEpoch)
assertEquals(1, member.topologyEpoch)
assertEquals(member.targetAssignment, member.assignment)
assertEquals(clientId, member.clientId())
assertEquals(clientHost, member.clientHost())
}
// Verify all partitions 0, 1, 2 are assigned exactly once
val allAssignedPartitions = describedGroup.members.asScala.flatMap { member =>
member.assignment.activeTasks.asScala.flatMap(_.partitions.asScala)
}.toList
assertEquals(List(0, 1, 2).sorted, allAssignedPartitions.sorted)
}
}
} finally{
admin.close()
}
}
private def convertTaskIds(responseTasks: java.util.List[StreamsGroupHeartbeatResponseData.TaskIds]): List[StreamsGroupHeartbeatRequestData.TaskIds] = {
if (responseTasks == null) {
List.empty
} else {
responseTasks.asScala.map { responseTask =>
new StreamsGroupHeartbeatRequestData.TaskIds()
.setSubtopologyId(responseTask.subtopologyId)
.setPartitions(responseTask.partitions)
}.toList
}
}
}

View File

@ -110,6 +110,8 @@
<h3><a id="streams_api_changes_410" href="#streams_api_changes_410">Streams API changes in 4.1.0</a></h3>
<p><b>Note:</b> Kafka Streams 4.1.0 contains a critical memory leak bug (<a href="https://issues.apache.org/jira/browse/KAFKA-19748">KAFKA-19748</a>) that affects users of range scans and certain DSL operators (session windows, sliding windows, stream-stream joins, foreign-key joins). Users running Kafka Streams should consider upgrading directly to 4.1.1 when available.</p>
<h4>Early Access of the Streams Rebalance Protocol</h4>
<p>

View File

@ -189,10 +189,19 @@
A new metric <code>AvgIdleRatio</code> has been added to the <code>ControllerEventManager</code> group. This metric measures the average idle ratio of the controller event queue thread,
providing visibility into how much time the controller spends waiting for events versus processing them. The metric value ranges from 0.0 (always busy) to 1.0 (always idle).
</li>
<li>
Deprecated <code>org.apache.kafka.streams.KafkaStreams$CloseOptions</code> and its related methods, such as
<code>KafkaStreams#close(org.apache.kafka.streams.KafkaStreams$CloseOptions)</code>.
As a replacement, please use <code>org.apache.kafka.streams.CloseOptions</code> and
<code>KafkaStreams#close(org.apache.kafka.streams.CloseOptions)</code>.
For further details, please refer to <a href="https://cwiki.apache.org/confluence/x/QAq9F">KIP-1153</a>.
</li>
</ul>
<h4><a id="upgrade_4_1_0" href="#upgrade_4_1_0">Upgrading to 4.1.0</a></h4>
<p><b>Note:</b> Kafka Streams 4.1.0 contains a critical memory leak bug (<a href="https://issues.apache.org/jira/browse/KAFKA-19748">KAFKA-19748</a>) that affects users of range scans and certain DSL operators (session windows, sliding windows, stream-stream joins, foreign-key joins). Users running Kafka Streams should consider upgrading directly to 4.1.1 when available.</p>
<h5><a id="upgrade_4_1_0_from" href="#upgrade_4_1_0_from">Upgrading Servers to 4.1.0 from any version 3.3.x through 4.0.x</a></h5>
<h5><a id="upgrade_410_notable" href="#upgrade_410_notable">Notable changes in 4.1.0</a></h5>
<ul>

View File

@ -60,7 +60,7 @@ versions += [
commonsLang: "3.18.0",
commonsValidator: "1.10.0",
classgraph: "4.8.179",
gradle: "8.14.3",
gradle: "9.1.0",
grgit: "4.1.1",
httpclient: "4.5.14",
jackson: "2.19.0",
@ -125,7 +125,7 @@ versions += [
snappy: "1.1.10.7",
spotbugs: "4.9.4",
mockOAuth2Server: "2.2.1",
zinc: "1.9.2",
zinc: "1.10.8",
// When updating the zstd version, please do as well in docker/native/native-image-configs/resource-config.json
// Also make sure the compression levels in org.apache.kafka.common.record.CompressionType are still valid
zstd: "1.5.6-10",

View File

@ -1,7 +1,7 @@
distributionBase=GRADLE_USER_HOME
distributionPath=wrapper/dists
distributionSha256Sum=bd71102213493060956ec229d946beee57158dbd89d0e62b91bca0fa2c5f3531
distributionUrl=https\://services.gradle.org/distributions/gradle-8.14.3-bin.zip
distributionSha256Sum=a17ddd85a26b6a7f5ddb71ff8b05fc5104c0202c6e64782429790c933686c806
distributionUrl=https\://services.gradle.org/distributions/gradle-9.1.0-bin.zip
networkTimeout=10000
validateDistributionUrl=true
zipStoreBase=GRADLE_USER_HOME

41
gradlew vendored
View File

@ -1,7 +1,7 @@
#!/bin/sh
#
# Copyright © 2015-2021 the original authors.
# Copyright © 2015 the original authors.
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
@ -15,6 +15,8 @@
# See the License for the specific language governing permissions and
# limitations under the License.
#
# SPDX-License-Identifier: Apache-2.0
#
##############################################################################
#
@ -55,7 +57,7 @@
# Darwin, MinGW, and NonStop.
#
# (3) This script is generated from the Groovy template
# https://github.com/gradle/gradle/blob/HEAD/subprojects/plugins/src/main/resources/org/gradle/api/internal/plugins/unixStartScript.txt
# https://github.com/gradle/gradle/blob/HEAD/platforms/jvm/plugins-application/src/main/resources/org/gradle/api/internal/plugins/unixStartScript.txt
# within the Gradle project.
#
# You can find Gradle at https://github.com/gradle/gradle/.
@ -84,7 +86,7 @@ done
# shellcheck disable=SC2034
APP_BASE_NAME=${0##*/}
# Discard cd standard output in case $CDPATH is set (https://github.com/gradle/gradle/issues/25036)
APP_HOME=$( cd "${APP_HOME:-./}" > /dev/null && pwd -P ) || exit
APP_HOME=$( cd -P "${APP_HOME:-./}" > /dev/null && printf '%s\n' "$PWD" ) || exit
# Use the maximum available, or set MAX_FD != -1 to use that value.
MAX_FD=maximum
@ -113,20 +115,6 @@ case "$( uname )" in #(
esac
# Loop in case we encounter an error.
for attempt in 1 2 3; do
if [ ! -e "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" ]; then
if ! curl -s -S --retry 3 -L -o "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" "https://raw.githubusercontent.com/gradle/gradle/v8.14.3/gradle/wrapper/gradle-wrapper.jar"; then
rm -f "$APP_HOME/gradle/wrapper/gradle-wrapper.jar"
# Pause for a bit before looping in case the server throttled us.
sleep 5
continue
fi
fi
done
CLASSPATH=$APP_HOME/gradle/wrapper/gradle-wrapper.jar
# Determine the Java command to use to start the JVM.
if [ -n "$JAVA_HOME" ] ; then
@ -183,7 +171,6 @@ fi
# For Cygwin or MSYS, switch paths to Windows format before running java
if "$cygwin" || "$msys" ; then
APP_HOME=$( cygpath --path --mixed "$APP_HOME" )
CLASSPATH=$( cygpath --path --mixed "$CLASSPATH" )
JAVACMD=$( cygpath --unix "$JAVACMD" )
@ -212,19 +199,31 @@ if "$cygwin" || "$msys" ; then
fi
# Loop in case we encounter an error.
for attempt in 1 2 3; do
if [ ! -e "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" ]; then
if ! curl -s -S --retry 3 -L -o "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" "https://raw.githubusercontent.com/gradle/gradle/v9.1.0/gradle/wrapper/gradle-wrapper.jar"; then
rm -f "$APP_HOME/gradle/wrapper/gradle-wrapper.jar"
# Pause for a bit before looping in case the server throttled us.
sleep 5
continue
fi
fi
done
# Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script.
DEFAULT_JVM_OPTS='"-Xmx64m" "-Xms64m"'
# Collect all arguments for the java command:
# * DEFAULT_JVM_OPTS, JAVA_OPTS, JAVA_OPTS, and optsEnvironmentVar are not allowed to contain shell fragments,
# * DEFAULT_JVM_OPTS, JAVA_OPTS, and optsEnvironmentVar are not allowed to contain shell fragments,
# and any embedded shellness will be escaped.
# * For example: A user cannot expect ${Hostname} to be expanded, as it is an environment variable and will be
# treated as '${Hostname}' itself on the command line.
set -- \
"-Dorg.gradle.appname=$APP_BASE_NAME" \
-classpath "$CLASSPATH" \
org.gradle.wrapper.GradleWrapperMain \
-jar "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" \
"$@"
# Stop when "xargs" is not available.

View File

@ -2249,18 +2249,13 @@ public class GroupMetadataManager {
.setClassicMemberMetadata(null)
.build();
// If the group is newly created, we must ensure that it moves away from
// epoch 0 and that it is fully initialized.
boolean bumpGroupEpoch = group.groupEpoch() == 0;
bumpGroupEpoch |= hasMemberSubscriptionChanged(
boolean subscribedTopicNamesChanged = hasMemberSubscriptionChanged(
groupId,
member,
updatedMember,
records
);
bumpGroupEpoch |= maybeUpdateRegularExpressions(
UpdateRegularExpressionsResult updateRegularExpressionsResult = maybeUpdateRegularExpressions(
context,
group,
member,
@ -2268,9 +2263,24 @@ public class GroupMetadataManager {
records
);
// The subscription has changed when either the subscribed topic names or subscribed topic
// regex has changed.
boolean hasSubscriptionChanged = subscribedTopicNamesChanged || updateRegularExpressionsResult.regexUpdated();
int groupEpoch = group.groupEpoch();
SubscriptionType subscriptionType = group.subscriptionType();
boolean bumpGroupEpoch =
// If the group is newly created, we must ensure that it moves away from
// epoch 0 and that it is fully initialized.
groupEpoch == 0 ||
// Bumping the group epoch signals that the target assignment should be updated. We bump
// the group epoch when the member has changed its subscribed topic names or the member
// has changed its subscribed topic regex to a regex that is already resolved. We avoid
// bumping the group epoch when the new subscribed topic regex has not been resolved
// yet, since we will have to update the target assignment again later.
subscribedTopicNamesChanged ||
updateRegularExpressionsResult == UpdateRegularExpressionsResult.REGEX_UPDATED_AND_RESOLVED;
if (bumpGroupEpoch || group.hasMetadataExpired(currentTimeMs)) {
// The subscription metadata is updated in two cases:
// 1) The member has updated its subscriptions;
@ -2315,6 +2325,9 @@ public class GroupMetadataManager {
group::currentPartitionEpoch,
targetAssignmentEpoch,
targetAssignment,
group.resolvedRegularExpressions(),
// Force consistency with the subscription when the subscription has changed.
hasSubscriptionChanged,
ownedTopicPartitions,
records
);
@ -2468,6 +2481,8 @@ public class GroupMetadataManager {
group::currentPartitionEpoch,
group.assignmentEpoch(),
group.targetAssignment(updatedMember.memberId(), updatedMember.instanceId()),
group.resolvedRegularExpressions(),
bumpGroupEpoch,
toTopicPartitions(subscription.ownedPartitions(), metadataImage),
records
);
@ -2511,6 +2526,9 @@ public class GroupMetadataManager {
group::currentPartitionEpoch,
targetAssignmentEpoch,
targetAssignment,
group.resolvedRegularExpressions(),
// Force consistency with the subscription when the subscription has changed.
bumpGroupEpoch,
toTopicPartitions(subscription.ownedPartitions(), metadataImage),
records
);
@ -2669,6 +2687,8 @@ public class GroupMetadataManager {
updatedMember,
targetAssignmentEpoch,
targetAssignment,
// Force consistency with the subscription when the subscription has changed.
bumpGroupEpoch,
records
);
@ -3108,6 +3128,16 @@ public class GroupMetadataManager {
return value != null && !value.isEmpty();
}
private enum UpdateRegularExpressionsResult {
NO_CHANGE,
REGEX_UPDATED,
REGEX_UPDATED_AND_RESOLVED;
public boolean regexUpdated() {
return this == REGEX_UPDATED || this == REGEX_UPDATED_AND_RESOLVED;
}
}
/**
* Check whether the member has updated its subscribed topic regular expression and
* may trigger the resolution/the refresh of all the regular expressions in the
@ -3119,9 +3149,9 @@ public class GroupMetadataManager {
* @param member The old member.
* @param updatedMember The new member.
* @param records The records accumulator.
* @return Whether a rebalance must be triggered.
* @return The result of the update.
*/
private boolean maybeUpdateRegularExpressions(
private UpdateRegularExpressionsResult maybeUpdateRegularExpressions(
AuthorizableRequestContext context,
ConsumerGroup group,
ConsumerGroupMember member,
@ -3134,14 +3164,17 @@ public class GroupMetadataManager {
String oldSubscribedTopicRegex = member.subscribedTopicRegex();
String newSubscribedTopicRegex = updatedMember.subscribedTopicRegex();
boolean bumpGroupEpoch = false;
boolean requireRefresh = false;
UpdateRegularExpressionsResult updateRegularExpressionsResult = UpdateRegularExpressionsResult.NO_CHANGE;
// Check whether the member has changed its subscribed regex.
if (!Objects.equals(oldSubscribedTopicRegex, newSubscribedTopicRegex)) {
boolean subscribedTopicRegexChanged = !Objects.equals(oldSubscribedTopicRegex, newSubscribedTopicRegex);
if (subscribedTopicRegexChanged) {
log.debug("[GroupId {}] Member {} updated its subscribed regex to: {}.",
groupId, memberId, newSubscribedTopicRegex);
updateRegularExpressionsResult = UpdateRegularExpressionsResult.REGEX_UPDATED;
if (isNotEmpty(oldSubscribedTopicRegex) && group.numSubscribedMembers(oldSubscribedTopicRegex) == 1) {
// If the member was the last one subscribed to the regex, we delete the
// resolved regular expression.
@ -3160,7 +3193,9 @@ public class GroupMetadataManager {
} else {
// If the new regex is already resolved, we trigger a rebalance
// by bumping the group epoch.
bumpGroupEpoch = group.resolvedRegularExpression(newSubscribedTopicRegex).isPresent();
if (group.resolvedRegularExpression(newSubscribedTopicRegex).isPresent()) {
updateRegularExpressionsResult = UpdateRegularExpressionsResult.REGEX_UPDATED_AND_RESOLVED;
}
}
}
}
@ -3176,20 +3211,20 @@ public class GroupMetadataManager {
// 0. The group is subscribed to regular expressions. We also take the one
// that the current may have just introduced.
if (!requireRefresh && group.subscribedRegularExpressions().isEmpty()) {
return bumpGroupEpoch;
return updateRegularExpressionsResult;
}
// 1. There is no ongoing refresh for the group.
String key = group.groupId() + "-regex";
if (executor.isScheduled(key)) {
return bumpGroupEpoch;
return updateRegularExpressionsResult;
}
// 2. The last refresh is older than 10s. If the group does not have any regular
// expressions but the current member just brought a new one, we should continue.
long lastRefreshTimeMs = group.lastResolvedRegularExpressionRefreshTimeMs();
if (currentTimeMs <= lastRefreshTimeMs + REGEX_BATCH_REFRESH_MIN_INTERVAL_MS) {
return bumpGroupEpoch;
return updateRegularExpressionsResult;
}
// 3.1 The group has unresolved regular expressions.
@ -3218,7 +3253,7 @@ public class GroupMetadataManager {
);
}
return bumpGroupEpoch;
return updateRegularExpressionsResult;
}
/**
@ -3492,16 +3527,18 @@ public class GroupMetadataManager {
/**
* Reconciles the current assignment of the member towards the target assignment if needed.
*
* @param groupId The group id.
* @param member The member to reconcile.
* @param currentPartitionEpoch The function returning the current epoch of
* a given partition.
* @param targetAssignmentEpoch The target assignment epoch.
* @param targetAssignment The target assignment.
* @param ownedTopicPartitions The list of partitions owned by the member. This
* is reported in the ConsumerGroupHeartbeat API and
* it could be null if not provided.
* @param records The list to accumulate any new records.
* @param groupId The group id.
* @param member The member to reconcile.
* @param currentPartitionEpoch The function returning the current epoch of
* a given partition.
* @param targetAssignmentEpoch The target assignment epoch.
* @param targetAssignment The target assignment.
* @param resolvedRegularExpressions The resolved regular expressions.
* @param hasSubscriptionChanged Whether the member has changed its subscription on the current heartbeat.
* @param ownedTopicPartitions The list of partitions owned by the member. This
* is reported in the ConsumerGroupHeartbeat API and
* it could be null if not provided.
* @param records The list to accumulate any new records.
* @return The received member if no changes have been made; or a new
* member containing the new assignment.
*/
@ -3511,15 +3548,20 @@ public class GroupMetadataManager {
BiFunction<Uuid, Integer, Integer> currentPartitionEpoch,
int targetAssignmentEpoch,
Assignment targetAssignment,
Map<String, ResolvedRegularExpression> resolvedRegularExpressions,
boolean hasSubscriptionChanged,
List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions,
List<CoordinatorRecord> records
) {
if (member.isReconciledTo(targetAssignmentEpoch)) {
if (!hasSubscriptionChanged && member.isReconciledTo(targetAssignmentEpoch)) {
return member;
}
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(targetAssignmentEpoch, targetAssignment)
.withHasSubscriptionChanged(hasSubscriptionChanged)
.withResolvedRegularExpressions(resolvedRegularExpressions)
.withCurrentPartitionEpoch(currentPartitionEpoch)
.withOwnedTopicPartitions(ownedTopicPartitions)
.build();
@ -3556,11 +3598,12 @@ public class GroupMetadataManager {
/**
* Reconciles the current assignment of the member towards the target assignment if needed.
*
* @param groupId The group id.
* @param member The member to reconcile.
* @param targetAssignmentEpoch The target assignment epoch.
* @param targetAssignment The target assignment.
* @param records The list to accumulate any new records.
* @param groupId The group id.
* @param member The member to reconcile.
* @param targetAssignmentEpoch The target assignment epoch.
* @param targetAssignment The target assignment.
* @param hasSubscriptionChanged Whether the member has changed its subscription on the current heartbeat.
* @param records The list to accumulate any new records.
* @return The received member if no changes have been made; or a new
* member containing the new assignment.
*/
@ -3569,14 +3612,17 @@ public class GroupMetadataManager {
ShareGroupMember member,
int targetAssignmentEpoch,
Assignment targetAssignment,
boolean hasSubscriptionChanged,
List<CoordinatorRecord> records
) {
if (member.isReconciledTo(targetAssignmentEpoch)) {
if (!hasSubscriptionChanged && member.isReconciledTo(targetAssignmentEpoch)) {
return member;
}
ShareGroupMember updatedMember = new ShareGroupAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(targetAssignmentEpoch, targetAssignment)
.withHasSubscriptionChanged(hasSubscriptionChanged)
.build();
if (!updatedMember.equals(member)) {

View File

@ -19,8 +19,11 @@ package org.apache.kafka.coordinator.group.modern.consumer;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.errors.FencedMemberEpochException;
import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.group.modern.Assignment;
import org.apache.kafka.coordinator.group.modern.MemberState;
import org.apache.kafka.coordinator.group.modern.TopicIds;
import org.apache.kafka.coordinator.group.modern.UnionSet;
import java.util.HashMap;
import java.util.HashSet;
@ -41,6 +44,11 @@ public class CurrentAssignmentBuilder {
*/
private final ConsumerGroupMember member;
/**
* The metadata image.
*/
private CoordinatorMetadataImage metadataImage = CoordinatorMetadataImage.EMPTY;
/**
* The target assignment epoch.
*/
@ -51,6 +59,16 @@ public class CurrentAssignmentBuilder {
*/
private Assignment targetAssignment;
/**
* Whether the member has changed its subscription on the current heartbeat.
*/
private boolean hasSubscriptionChanged;
/**
* The resolved regular expressions.
*/
private Map<String, ResolvedRegularExpression> resolvedRegularExpressions = Map.of();
/**
* A function which returns the current epoch of a topic-partition or -1 if the
* topic-partition is not assigned. The current epoch is the epoch of the current owner.
@ -73,6 +91,19 @@ public class CurrentAssignmentBuilder {
this.member = Objects.requireNonNull(member);
}
/**
* Sets the metadata image.
*
* @param metadataImage The metadata image.
* @return This object.
*/
public CurrentAssignmentBuilder withMetadataImage(
CoordinatorMetadataImage metadataImage
) {
this.metadataImage = metadataImage;
return this;
}
/**
* Sets the target assignment epoch and the target assignment that the
* consumer group member must be reconciled to.
@ -90,6 +121,32 @@ public class CurrentAssignmentBuilder {
return this;
}
/**
* Sets whether the member has changed its subscription on the current heartbeat.
*
* @param hasSubscriptionChanged If true, always removes unsubscribed topics from the current assignment.
* @return This object.
*/
public CurrentAssignmentBuilder withHasSubscriptionChanged(
boolean hasSubscriptionChanged
) {
this.hasSubscriptionChanged = hasSubscriptionChanged;
return this;
}
/**
* Sets the resolved regular expressions.
*
* @param resolvedRegularExpressions The resolved regular expressions.
* @return This object.
*/
public CurrentAssignmentBuilder withResolvedRegularExpressions(
Map<String, ResolvedRegularExpression> resolvedRegularExpressions
) {
this.resolvedRegularExpressions = resolvedRegularExpressions;
return this;
}
/**
* Sets a BiFunction which allows to retrieve the current epoch of a
* partition. This is used by the state machine to determine if a
@ -132,12 +189,15 @@ public class CurrentAssignmentBuilder {
case STABLE:
// When the member is in the STABLE state, we verify if a newer
// epoch (or target assignment) is available. If it is, we can
// reconcile the member towards it. Otherwise, we return.
// reconcile the member towards it. Otherwise, we ensure the
// assignment is consistent with the subscribed topics, if changed.
if (member.memberEpoch() != targetAssignmentEpoch) {
return computeNextAssignment(
member.memberEpoch(),
member.assignedPartitions()
);
} else if (hasSubscriptionChanged) {
return updateCurrentAssignment(member.assignedPartitions());
} else {
return member;
}
@ -147,18 +207,27 @@ public class CurrentAssignmentBuilder {
// until the member has revoked the necessary partitions. They are
// considered revoked when they are not anymore reported in the
// owned partitions set in the ConsumerGroupHeartbeat API.
// Additional partitions may need revoking when the member's
// subscription changes.
// If the member provides its owned partitions. We verify if it still
// owns any of the revoked partitions. If it does, we cannot progress.
if (ownsRevokedPartitions(member.partitionsPendingRevocation())) {
return member;
if (hasSubscriptionChanged) {
return updateCurrentAssignment(member.assignedPartitions());
} else {
return member;
}
}
// When the member has revoked all the pending partitions, it can
// transition to the next epoch (current + 1) and we can reconcile
// its state towards the latest target assignment.
return computeNextAssignment(
member.memberEpoch() + 1,
// When we enter UNREVOKED_PARTITIONS due to a subscription change,
// we must not advance the member epoch when the new target
// assignment is not available yet.
Math.min(member.memberEpoch() + 1, targetAssignmentEpoch),
member.assignedPartitions()
);
@ -215,6 +284,71 @@ public class CurrentAssignmentBuilder {
return false;
}
/**
* Updates the current assignment, removing any partitions that are not part of the subscribed topics.
* This method is a lot faster than running the full reconciliation logic in computeNextAssignment.
*
* @param memberAssignedPartitions The assigned partitions of the member to use.
* @return A new ConsumerGroupMember.
*/
private ConsumerGroupMember updateCurrentAssignment(
Map<Uuid, Set<Integer>> memberAssignedPartitions
) {
Set<Uuid> subscribedTopicIds = subscribedTopicIds();
// Reuse the original map if no topics need to be removed.
Map<Uuid, Set<Integer>> newAssignedPartitions;
Map<Uuid, Set<Integer>> newPartitionsPendingRevocation;
if (subscribedTopicIds.isEmpty() && member.partitionsPendingRevocation().isEmpty()) {
newAssignedPartitions = Map.of();
newPartitionsPendingRevocation = memberAssignedPartitions;
} else {
newAssignedPartitions = memberAssignedPartitions;
newPartitionsPendingRevocation = new HashMap<>(member.partitionsPendingRevocation());
for (Map.Entry<Uuid, Set<Integer>> entry : memberAssignedPartitions.entrySet()) {
if (!subscribedTopicIds.contains(entry.getKey())) {
if (newAssignedPartitions == memberAssignedPartitions) {
newAssignedPartitions = new HashMap<>(memberAssignedPartitions);
newPartitionsPendingRevocation = new HashMap<>(member.partitionsPendingRevocation());
}
newAssignedPartitions.remove(entry.getKey());
newPartitionsPendingRevocation.merge(
entry.getKey(),
entry.getValue(),
(existing, additional) -> {
existing = new HashSet<>(existing);
existing.addAll(additional);
return existing;
}
);
}
}
}
if (newAssignedPartitions == memberAssignedPartitions) {
// If no partitions were removed, we can return the member as is.
return member;
}
if (!newPartitionsPendingRevocation.isEmpty() && ownsRevokedPartitions(newPartitionsPendingRevocation)) {
return new ConsumerGroupMember.Builder(member)
.setState(MemberState.UNREVOKED_PARTITIONS)
.setAssignedPartitions(newAssignedPartitions)
.setPartitionsPendingRevocation(newPartitionsPendingRevocation)
.build();
} else {
// There were partitions removed, but they were already revoked.
// Keep the member in the current state and shrink the assigned partitions.
// We do not expect to be in the UNREVOKED_PARTITIONS state here. The full
// reconciliation logic should handle the case where the member has revoked all its
// partitions pending revocation.
return new ConsumerGroupMember.Builder(member)
.setAssignedPartitions(newAssignedPartitions)
.build();
}
}
/**
* Computes the next assignment.
*
@ -227,6 +361,8 @@ public class CurrentAssignmentBuilder {
int memberEpoch,
Map<Uuid, Set<Integer>> memberAssignedPartitions
) {
Set<Uuid> subscribedTopicIds = subscribedTopicIds();
boolean hasUnreleasedPartitions = false;
Map<Uuid, Set<Integer>> newAssignedPartitions = new HashMap<>();
Map<Uuid, Set<Integer>> newPartitionsPendingRevocation = new HashMap<>();
@ -241,6 +377,11 @@ public class CurrentAssignmentBuilder {
Set<Integer> currentAssignedPartitions = memberAssignedPartitions
.getOrDefault(topicId, Set.of());
// If the member is no longer subscribed to the topic, treat its target assignment as empty.
if (!subscribedTopicIds.contains(topicId)) {
target = Set.of();
}
// New Assigned Partitions = Previous Assigned Partitions Target
Set<Integer> assignedPartitions = new HashSet<>(currentAssignedPartitions);
assignedPartitions.retainAll(target);
@ -317,4 +458,28 @@ public class CurrentAssignmentBuilder {
.build();
}
}
/**
* Gets the set of topic IDs that the member is subscribed to.
*
* @return The set of topic IDs that the member is subscribed to.
*/
private Set<Uuid> subscribedTopicIds() {
Set<String> subscriptions = member.subscribedTopicNames();
String subscribedTopicRegex = member.subscribedTopicRegex();
if (subscribedTopicRegex != null && !subscribedTopicRegex.isEmpty()) {
ResolvedRegularExpression resolvedRegularExpression = resolvedRegularExpressions.get(subscribedTopicRegex);
if (resolvedRegularExpression != null) {
if (subscriptions.isEmpty()) {
subscriptions = resolvedRegularExpression.topics();
} else if (!resolvedRegularExpression.topics().isEmpty()) {
subscriptions = new UnionSet<>(subscriptions, resolvedRegularExpression.topics());
}
} else {
// Treat an unresolved regex as matching no topics, to be conservative.
}
}
return new TopicIds(subscriptions, metadataImage);
}
}

View File

@ -16,10 +16,16 @@
*/
package org.apache.kafka.coordinator.group.modern.share;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.group.modern.Assignment;
import org.apache.kafka.coordinator.group.modern.MemberState;
import org.apache.kafka.coordinator.group.modern.TopicIds;
import java.util.HashMap;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
/**
* The ShareGroupAssignmentBuilder class encapsulates the reconciliation engine of the
@ -32,6 +38,11 @@ public class ShareGroupAssignmentBuilder {
*/
private final ShareGroupMember member;
/**
* The metadata image.
*/
private CoordinatorMetadataImage metadataImage = CoordinatorMetadataImage.EMPTY;
/**
* The target assignment epoch.
*/
@ -42,6 +53,11 @@ public class ShareGroupAssignmentBuilder {
*/
private Assignment targetAssignment;
/**
* Whether the member has changed its subscription on the current heartbeat.
*/
private boolean hasSubscriptionChanged;
/**
* Constructs the ShareGroupAssignmentBuilder based on the current state of the
* provided share group member.
@ -52,6 +68,19 @@ public class ShareGroupAssignmentBuilder {
this.member = Objects.requireNonNull(member);
}
/**
* Sets the metadata image.
*
* @param metadataImage The metadata image.
* @return This object.
*/
public ShareGroupAssignmentBuilder withMetadataImage(
CoordinatorMetadataImage metadataImage
) {
this.metadataImage = metadataImage;
return this;
}
/**
* Sets the target assignment epoch and the target assignment that the
* share group member must be reconciled to.
@ -69,6 +98,19 @@ public class ShareGroupAssignmentBuilder {
return this;
}
/**
* Sets whether the member has changed its subscription on the current heartbeat.
*
* @param hasSubscriptionChanged If true, always removes unsubscribed topics from the current assignment.
* @return This object.
*/
public ShareGroupAssignmentBuilder withHasSubscriptionChanged(
boolean hasSubscriptionChanged
) {
this.hasSubscriptionChanged = hasSubscriptionChanged;
return this;
}
/**
* Builds the next state for the member or keep the current one if it
* is not possible to move forward with the current state.
@ -83,11 +125,38 @@ public class ShareGroupAssignmentBuilder {
// when the member is updated.
return new ShareGroupMember.Builder(member)
.setState(MemberState.STABLE)
.setAssignedPartitions(targetAssignment.partitions())
// If we have client-side assignors, the latest target assignment may not
// be consistent with the latest subscribed topics, so we must always
// filter the assigned partitions to ensure they are consistent with the
// subscribed topics.
.setAssignedPartitions(filterAssignedPartitions(targetAssignment.partitions(), member.subscribedTopicNames()))
.updateMemberEpoch(targetAssignmentEpoch)
.build();
} else if (hasSubscriptionChanged) {
return new ShareGroupMember.Builder(member)
.setAssignedPartitions(filterAssignedPartitions(targetAssignment.partitions(), member.subscribedTopicNames()))
.build();
} else {
return member;
}
}
return member;
private Map<Uuid, Set<Integer>> filterAssignedPartitions(
Map<Uuid, Set<Integer>> partitions,
Set<String> subscribedTopicNames
) {
TopicIds subscribedTopicIds = new TopicIds(member.subscribedTopicNames(), metadataImage);
// Reuse the original map if no topics need to be removed.
Map<Uuid, Set<Integer>> filteredPartitions = partitions;
for (Map.Entry<Uuid, Set<Integer>> entry : partitions.entrySet()) {
if (!subscribedTopicIds.contains(entry.getKey())) {
if (filteredPartitions == partitions) {
filteredPartitions = new HashMap<>(partitions);
}
filteredPartitions.remove(entry.getKey());
}
}
return filteredPartitions;
}
}

View File

@ -74,6 +74,7 @@ public class ShareGroupMember extends ModernGroupMember {
this.memberId = Objects.requireNonNull(newMemberId);
this.memberEpoch = member.memberEpoch;
this.previousMemberEpoch = member.previousMemberEpoch;
this.state = member.state;
this.rackId = member.rackId;
this.clientId = member.clientId;
this.clientHost = member.clientHost;

View File

@ -20604,7 +20604,7 @@ public class GroupMetadataManagerTest {
.build();
// Member 1 updates its new regular expression.
CoordinatorResult<ConsumerGroupHeartbeatResponseData, CoordinatorRecord> result = context.consumerGroupHeartbeat(
CoordinatorResult<ConsumerGroupHeartbeatResponseData, CoordinatorRecord> result1 = context.consumerGroupHeartbeat(
new ConsumerGroupHeartbeatRequestData()
.setGroupId(groupId)
.setMemberId(memberId1)
@ -20620,19 +20620,15 @@ public class GroupMetadataManagerTest {
.setMemberEpoch(10)
.setHeartbeatIntervalMs(5000)
.setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()
.setTopicPartitions(List.of(
new ConsumerGroupHeartbeatResponseData.TopicPartitions()
.setTopicId(fooTopicId)
.setPartitions(List.of(0, 1, 2, 3, 4, 5))
))
.setTopicPartitions(List.of())
),
result.response()
result1.response()
);
ConsumerGroupMember expectedMember1 = new ConsumerGroupMember.Builder(memberId1)
.setState(MemberState.STABLE)
.setMemberEpoch(10)
.setPreviousMemberEpoch(0)
.setPreviousMemberEpoch(10)
.setClientId(DEFAULT_CLIENT_ID)
.setClientHost(DEFAULT_CLIENT_ADDRESS.toString())
.setRebalanceTimeoutMs(5000)
@ -20644,10 +20640,12 @@ public class GroupMetadataManagerTest {
// The member subscription is updated.
GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember1),
// The previous regular expression is deleted.
GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionTombstone(groupId, "foo*")
GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionTombstone(groupId, "foo*"),
// The member assignment is updated.
GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedMember1)
);
assertRecordsEquals(expectedRecords, result.records());
assertRecordsEquals(expectedRecords, result1.records());
// Execute pending tasks.
List<MockCoordinatorExecutor.ExecutorResult<CoordinatorRecord>> tasks = context.processTasks();
@ -20675,6 +20673,65 @@ public class GroupMetadataManagerTest {
),
task.result().records()
);
assignor.prepareGroupAssignment(new GroupAssignment(Map.of(
memberId1, new MemberAssignmentImpl(mkAssignment(
mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5),
mkTopicAssignment(barTopicId, 0, 1, 2)
))
)));
// Member heartbeats again with the same regex.
CoordinatorResult<ConsumerGroupHeartbeatResponseData, CoordinatorRecord> result2 = context.consumerGroupHeartbeat(
new ConsumerGroupHeartbeatRequestData()
.setGroupId(groupId)
.setMemberId(memberId1)
.setMemberEpoch(10)
.setRebalanceTimeoutMs(5000)
.setSubscribedTopicRegex("foo*|bar*")
.setServerAssignor("range")
.setTopicPartitions(List.of()));
assertResponseEquals(
new ConsumerGroupHeartbeatResponseData()
.setMemberId(memberId1)
.setMemberEpoch(11)
.setHeartbeatIntervalMs(5000)
.setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()
.setTopicPartitions(List.of(
new ConsumerGroupHeartbeatResponseData.TopicPartitions()
.setTopicId(fooTopicId)
.setPartitions(List.of(0, 1, 2, 3, 4, 5)),
new ConsumerGroupHeartbeatResponseData.TopicPartitions()
.setTopicId(barTopicId)
.setPartitions(List.of(0, 1, 2))))),
result2.response()
);
ConsumerGroupMember expectedMember2 = new ConsumerGroupMember.Builder(memberId1)
.setState(MemberState.STABLE)
.setMemberEpoch(11)
.setPreviousMemberEpoch(10)
.setClientId(DEFAULT_CLIENT_ID)
.setClientHost(DEFAULT_CLIENT_ADDRESS.toString())
.setRebalanceTimeoutMs(5000)
.setSubscribedTopicRegex("foo*|bar*")
.setServerAssignorName("range")
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5),
mkTopicAssignment(barTopicId, 0, 1, 2)))
.build();
expectedRecords = List.of(
GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId1, mkAssignment(
mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5),
mkTopicAssignment(barTopicId, 0, 1, 2)
)),
GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochRecord(groupId, 11),
GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedMember2)
);
assertRecordsEquals(expectedRecords, result2.records());
}
@Test
@ -21077,10 +21134,7 @@ public class GroupMetadataManagerTest {
.setMemberEpoch(10)
.setHeartbeatIntervalMs(5000)
.setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()
.setTopicPartitions(List.of(
new ConsumerGroupHeartbeatResponseData.TopicPartitions()
.setTopicId(fooTopicId)
.setPartitions(List.of(3, 4, 5))))),
.setTopicPartitions(List.of())),
result1.response()
);
@ -21098,7 +21152,8 @@ public class GroupMetadataManagerTest {
assertRecordsEquals(
List.of(
GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember2),
GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionTombstone(groupId, "foo*")
GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionTombstone(groupId, "foo*"),
GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedMember2)
),
result1.records()
);
@ -21164,8 +21219,7 @@ public class GroupMetadataManagerTest {
.setRebalanceTimeoutMs(5000)
.setSubscribedTopicRegex("foo|bar*")
.setServerAssignorName("range")
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(fooTopicId, 3, 4, 5)))
.setAssignedPartitions(mkAssignment())
.build();
assertResponseEquals(
@ -21174,10 +21228,7 @@ public class GroupMetadataManagerTest {
.setMemberEpoch(11)
.setHeartbeatIntervalMs(5000)
.setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()
.setTopicPartitions(List.of(
new ConsumerGroupHeartbeatResponseData.TopicPartitions()
.setTopicId(fooTopicId)
.setPartitions(List.of(3, 4, 5))))),
.setTopicPartitions(List.of())),
result2.response()
);
@ -21306,10 +21357,7 @@ public class GroupMetadataManagerTest {
.setMemberEpoch(10)
.setHeartbeatIntervalMs(5000)
.setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()
.setTopicPartitions(List.of(
new ConsumerGroupHeartbeatResponseData.TopicPartitions()
.setTopicId(fooTopicId)
.setPartitions(List.of(3, 4, 5))))),
.setTopicPartitions(List.of())),
result1.response()
);
@ -21327,7 +21375,8 @@ public class GroupMetadataManagerTest {
assertRecordsEquals(
List.of(
GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember2),
GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionTombstone(groupId, "foo*")
GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionTombstone(groupId, "foo*"),
GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedMember2)
),
result1.records()
);
@ -21440,6 +21489,219 @@ public class GroupMetadataManagerTest {
);
}
@Test
public void testStaticConsumerGroupMemberJoinsWithUpdatedRegex() {
String groupId = "fooup";
String memberId1 = Uuid.randomUuid().toString();
String memberId2 = Uuid.randomUuid().toString();
String instanceId = "instance-id";
Uuid fooTopicId = Uuid.randomUuid();
String fooTopicName = "foo";
Uuid barTopicId = Uuid.randomUuid();
String barTopicName = "bar";
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
.buildCoordinatorMetadataImage(12345L);
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor))
.withMetadataImage(metadataImage)
.withConsumerGroup(new ConsumerGroupBuilder(groupId, 10)
.withMember(new ConsumerGroupMember.Builder(memberId1)
.setInstanceId(instanceId)
.setState(MemberState.STABLE)
.setMemberEpoch(10)
.setPreviousMemberEpoch(10)
.setClientId(DEFAULT_CLIENT_ID)
.setClientHost(DEFAULT_CLIENT_ADDRESS.toString())
.setRebalanceTimeoutMs(5000)
.setSubscribedTopicRegex("foo*|bar*")
.setServerAssignorName("range")
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5),
mkTopicAssignment(barTopicId, 0, 1, 2)))
.build())
.withAssignment(memberId1, mkAssignment(
mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5),
mkTopicAssignment(barTopicId, 0, 1, 2)))
.withAssignmentEpoch(10))
.build();
// Static member temporarily leaves the group.
CoordinatorResult<ConsumerGroupHeartbeatResponseData, CoordinatorRecord> result1 = context.consumerGroupHeartbeat(
new ConsumerGroupHeartbeatRequestData()
.setGroupId(groupId)
.setInstanceId(instanceId)
.setMemberId(memberId1)
.setMemberEpoch(LEAVE_GROUP_STATIC_MEMBER_EPOCH)
);
assertResponseEquals(
new ConsumerGroupHeartbeatResponseData()
.setMemberId(memberId1)
.setMemberEpoch(LEAVE_GROUP_STATIC_MEMBER_EPOCH),
result1.response()
);
// Static member joins the group with an updated regular expression.
CoordinatorResult<ConsumerGroupHeartbeatResponseData, CoordinatorRecord> result2 = context.consumerGroupHeartbeat(
new ConsumerGroupHeartbeatRequestData()
.setGroupId(groupId)
.setInstanceId(instanceId)
.setMemberId(memberId2)
.setMemberEpoch(0)
.setRebalanceTimeoutMs(5000)
.setSubscribedTopicRegex("foo*")
.setServerAssignor("range")
.setTopicPartitions(List.of()));
// The returned assignment does not contain topics not in the current regular expression.
assertResponseEquals(
new ConsumerGroupHeartbeatResponseData()
.setMemberId(memberId2)
.setMemberEpoch(10)
.setHeartbeatIntervalMs(5000)
.setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()
.setTopicPartitions(List.of())
),
result2.response()
);
ConsumerGroupMember expectedCopiedMember = new ConsumerGroupMember.Builder(memberId2)
.setState(MemberState.STABLE)
.setInstanceId(instanceId)
.setMemberEpoch(0)
.setPreviousMemberEpoch(0)
.setClientId(DEFAULT_CLIENT_ID)
.setClientHost(DEFAULT_CLIENT_ADDRESS.toString())
.setRebalanceTimeoutMs(5000)
.setSubscribedTopicRegex("foo*|bar*")
.setServerAssignorName("range")
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5),
mkTopicAssignment(barTopicId, 0, 1, 2)))
.build();
ConsumerGroupMember expectedMember1 = new ConsumerGroupMember.Builder(memberId2)
.setState(MemberState.STABLE)
.setInstanceId(instanceId)
.setMemberEpoch(10)
.setPreviousMemberEpoch(0)
.setClientId(DEFAULT_CLIENT_ID)
.setClientHost(DEFAULT_CLIENT_ADDRESS.toString())
.setRebalanceTimeoutMs(5000)
.setSubscribedTopicRegex("foo*")
.setServerAssignorName("range")
.build();
List<CoordinatorRecord> expectedRecords = List.of(
// The previous member is deleted.
GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentTombstoneRecord(groupId, memberId1),
GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentTombstoneRecord(groupId, memberId1),
GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionTombstoneRecord(groupId, memberId1),
// The previous member is replaced by the new one.
GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedCopiedMember),
GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId2, mkAssignment(
mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5),
mkTopicAssignment(barTopicId, 0, 1, 2)
)),
GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedCopiedMember),
// The member subscription is updated.
GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember1),
// The previous regular expression is deleted.
GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionTombstone(groupId, "foo*|bar*"),
// The member assignment is updated.
GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedMember1)
);
assertRecordsEquals(expectedRecords, result2.records());
// Execute pending tasks.
List<MockCoordinatorExecutor.ExecutorResult<CoordinatorRecord>> tasks = context.processTasks();
assertEquals(1, tasks.size());
MockCoordinatorExecutor.ExecutorResult<CoordinatorRecord> task = tasks.get(0);
assertEquals(groupId + "-regex", task.key());
assertRecordsEquals(
List.of(
// The resolution of the new regex is persisted.
GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionRecord(
groupId,
"foo*",
new ResolvedRegularExpression(
Set.of("foo"),
12345L,
context.time.milliseconds()
)
),
// The group epoch is bumped.
GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, computeGroupHash(Map.of(
fooTopicName, computeTopicHash(fooTopicName, metadataImage)
)))
),
task.result().records()
);
assignor.prepareGroupAssignment(new GroupAssignment(Map.of(
memberId2, new MemberAssignmentImpl(mkAssignment(
mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5)
))
)));
// Member heartbeats again with the same regex.
CoordinatorResult<ConsumerGroupHeartbeatResponseData, CoordinatorRecord> result3 = context.consumerGroupHeartbeat(
new ConsumerGroupHeartbeatRequestData()
.setGroupId(groupId)
.setInstanceId(instanceId)
.setMemberId(memberId2)
.setMemberEpoch(10)
.setRebalanceTimeoutMs(5000)
.setSubscribedTopicRegex("foo*")
.setServerAssignor("range")
.setTopicPartitions(List.of()));
assertResponseEquals(
new ConsumerGroupHeartbeatResponseData()
.setMemberId(memberId2)
.setMemberEpoch(11)
.setHeartbeatIntervalMs(5000)
.setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()
.setTopicPartitions(List.of(
new ConsumerGroupHeartbeatResponseData.TopicPartitions()
.setTopicId(fooTopicId)
.setPartitions(List.of(0, 1, 2, 3, 4, 5))))),
result3.response()
);
ConsumerGroupMember expectedMember2 = new ConsumerGroupMember.Builder(memberId2)
.setState(MemberState.STABLE)
.setInstanceId(instanceId)
.setMemberEpoch(11)
.setPreviousMemberEpoch(10)
.setClientId(DEFAULT_CLIENT_ID)
.setClientHost(DEFAULT_CLIENT_ADDRESS.toString())
.setRebalanceTimeoutMs(5000)
.setSubscribedTopicRegex("foo*|bar*")
.setServerAssignorName("range")
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5)))
.build();
expectedRecords = List.of(
GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId2, mkAssignment(
mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5)
)),
GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochRecord(groupId, 11),
GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedMember2)
);
assertRecordsEquals(expectedRecords, result3.records());
}
@Test
public void testResolvedRegularExpressionsRemovedWhenMembersLeaveOrFenced() {
String groupId = "fooup";

View File

@ -19,13 +19,19 @@ package org.apache.kafka.coordinator.group.modern.consumer;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.errors.FencedMemberEpochException;
import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.common.runtime.MetadataImageBuilder;
import org.apache.kafka.coordinator.group.modern.Assignment;
import org.apache.kafka.coordinator.group.modern.MemberState;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.CsvSource;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.Set;
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
@ -36,19 +42,28 @@ public class CurrentAssignmentBuilderTest {
@Test
public void testStableToStable() {
String topic1 = "topic1";
String topic2 = "topic2";
Uuid topicId1 = Uuid.randomUuid();
Uuid topicId2 = Uuid.randomUuid();
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(topicId1, topic1, 10)
.addTopic(topicId2, topic2, 10)
.buildCoordinatorMetadataImage();
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
.setState(MemberState.STABLE)
.setMemberEpoch(10)
.setPreviousMemberEpoch(10)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 1, 2, 3),
mkTopicAssignment(topicId2, 4, 5, 6)))
.build();
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(11, new Assignment(mkAssignment(
mkTopicAssignment(topicId1, 1, 2, 3),
mkTopicAssignment(topicId2, 4, 5, 6))))
@ -60,6 +75,7 @@ public class CurrentAssignmentBuilderTest {
.setState(MemberState.STABLE)
.setMemberEpoch(11)
.setPreviousMemberEpoch(10)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 1, 2, 3),
mkTopicAssignment(topicId2, 4, 5, 6)))
@ -70,19 +86,28 @@ public class CurrentAssignmentBuilderTest {
@Test
public void testStableToStableWithNewPartitions() {
String topic1 = "topic1";
String topic2 = "topic2";
Uuid topicId1 = Uuid.randomUuid();
Uuid topicId2 = Uuid.randomUuid();
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(topicId1, topic1, 10)
.addTopic(topicId2, topic2, 10)
.buildCoordinatorMetadataImage();
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
.setState(MemberState.STABLE)
.setMemberEpoch(10)
.setPreviousMemberEpoch(10)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 1, 2, 3),
mkTopicAssignment(topicId2, 4, 5, 6)))
.build();
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(11, new Assignment(mkAssignment(
mkTopicAssignment(topicId1, 1, 2, 3, 4),
mkTopicAssignment(topicId2, 4, 5, 6, 7))))
@ -94,6 +119,7 @@ public class CurrentAssignmentBuilderTest {
.setState(MemberState.STABLE)
.setMemberEpoch(11)
.setPreviousMemberEpoch(10)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 1, 2, 3, 4),
mkTopicAssignment(topicId2, 4, 5, 6, 7)))
@ -104,19 +130,28 @@ public class CurrentAssignmentBuilderTest {
@Test
public void testStableToUnrevokedPartitions() {
String topic1 = "topic1";
String topic2 = "topic2";
Uuid topicId1 = Uuid.randomUuid();
Uuid topicId2 = Uuid.randomUuid();
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(topicId1, topic1, 10)
.addTopic(topicId2, topic2, 10)
.buildCoordinatorMetadataImage();
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
.setState(MemberState.STABLE)
.setMemberEpoch(10)
.setPreviousMemberEpoch(10)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 1, 2, 3),
mkTopicAssignment(topicId2, 4, 5, 6)))
.build();
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(11, new Assignment(mkAssignment(
mkTopicAssignment(topicId1, 2, 3, 4),
mkTopicAssignment(topicId2, 5, 6, 7))))
@ -128,6 +163,7 @@ public class CurrentAssignmentBuilderTest {
.setState(MemberState.UNREVOKED_PARTITIONS)
.setMemberEpoch(10)
.setPreviousMemberEpoch(10)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 2, 3),
mkTopicAssignment(topicId2, 5, 6)))
@ -141,19 +177,28 @@ public class CurrentAssignmentBuilderTest {
@Test
public void testStableToUnreleasedPartitions() {
String topic1 = "topic1";
String topic2 = "topic2";
Uuid topicId1 = Uuid.randomUuid();
Uuid topicId2 = Uuid.randomUuid();
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(topicId1, topic1, 10)
.addTopic(topicId2, topic2, 10)
.buildCoordinatorMetadataImage();
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
.setState(MemberState.STABLE)
.setMemberEpoch(10)
.setPreviousMemberEpoch(10)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 1, 2, 3),
mkTopicAssignment(topicId2, 4, 5, 6)))
.build();
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(11, new Assignment(mkAssignment(
mkTopicAssignment(topicId1, 1, 2, 3, 4),
mkTopicAssignment(topicId2, 4, 5, 6, 7))))
@ -165,6 +210,7 @@ public class CurrentAssignmentBuilderTest {
.setState(MemberState.UNRELEASED_PARTITIONS)
.setMemberEpoch(11)
.setPreviousMemberEpoch(10)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 1, 2, 3),
mkTopicAssignment(topicId2, 4, 5, 6)))
@ -175,19 +221,28 @@ public class CurrentAssignmentBuilderTest {
@Test
public void testStableToUnreleasedPartitionsWithOwnedPartitionsNotHavingRevokedPartitions() {
String topic1 = "topic1";
String topic2 = "topic2";
Uuid topicId1 = Uuid.randomUuid();
Uuid topicId2 = Uuid.randomUuid();
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(topicId1, topic1, 10)
.addTopic(topicId2, topic2, 10)
.buildCoordinatorMetadataImage();
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
.setState(MemberState.STABLE)
.setMemberEpoch(10)
.setPreviousMemberEpoch(10)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 1, 2, 3),
mkTopicAssignment(topicId2, 4, 5, 6)))
.build();
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(11, new Assignment(mkAssignment(
mkTopicAssignment(topicId1, 1, 2, 3),
mkTopicAssignment(topicId2, 4, 5, 7))))
@ -202,6 +257,7 @@ public class CurrentAssignmentBuilderTest {
.setState(MemberState.UNRELEASED_PARTITIONS)
.setMemberEpoch(11)
.setPreviousMemberEpoch(10)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 1, 2, 3),
mkTopicAssignment(topicId2, 4, 5)))
@ -212,13 +268,21 @@ public class CurrentAssignmentBuilderTest {
@Test
public void testUnrevokedPartitionsToStable() {
String topic1 = "topic1";
String topic2 = "topic2";
Uuid topicId1 = Uuid.randomUuid();
Uuid topicId2 = Uuid.randomUuid();
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(topicId1, topic1, 10)
.addTopic(topicId2, topic2, 10)
.buildCoordinatorMetadataImage();
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
.setState(MemberState.UNREVOKED_PARTITIONS)
.setMemberEpoch(10)
.setPreviousMemberEpoch(10)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 2, 3),
mkTopicAssignment(topicId2, 5, 6)))
@ -228,6 +292,7 @@ public class CurrentAssignmentBuilderTest {
.build();
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(11, new Assignment(mkAssignment(
mkTopicAssignment(topicId1, 2, 3),
mkTopicAssignment(topicId2, 5, 6))))
@ -246,6 +311,7 @@ public class CurrentAssignmentBuilderTest {
.setState(MemberState.STABLE)
.setMemberEpoch(11)
.setPreviousMemberEpoch(10)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 2, 3),
mkTopicAssignment(topicId2, 5, 6)))
@ -256,13 +322,21 @@ public class CurrentAssignmentBuilderTest {
@Test
public void testRemainsInUnrevokedPartitions() {
String topic1 = "topic1";
String topic2 = "topic2";
Uuid topicId1 = Uuid.randomUuid();
Uuid topicId2 = Uuid.randomUuid();
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(topicId1, topic1, 10)
.addTopic(topicId2, topic2, 10)
.buildCoordinatorMetadataImage();
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
.setState(MemberState.UNREVOKED_PARTITIONS)
.setMemberEpoch(10)
.setPreviousMemberEpoch(10)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 2, 3),
mkTopicAssignment(topicId2, 5, 6)))
@ -272,6 +346,7 @@ public class CurrentAssignmentBuilderTest {
.build();
CurrentAssignmentBuilder currentAssignmentBuilder = new CurrentAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(12, new Assignment(mkAssignment(
mkTopicAssignment(topicId1, 3),
mkTopicAssignment(topicId2, 6))))
@ -311,15 +386,27 @@ public class CurrentAssignmentBuilderTest {
);
}
@Test
public void testUnrevokedPartitionsToUnrevokedPartitions() {
@ParameterizedTest
@CsvSource({
"10, 12, 11",
"10, 10, 10", // The member epoch must not advance past the target assignment epoch.
})
public void testUnrevokedPartitionsToUnrevokedPartitions(int memberEpoch, int targetAssignmentEpoch, int expectedMemberEpoch) {
String topic1 = "topic1";
String topic2 = "topic2";
Uuid topicId1 = Uuid.randomUuid();
Uuid topicId2 = Uuid.randomUuid();
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(topicId1, topic1, 10)
.addTopic(topicId2, topic2, 10)
.buildCoordinatorMetadataImage();
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
.setState(MemberState.UNREVOKED_PARTITIONS)
.setMemberEpoch(10)
.setPreviousMemberEpoch(10)
.setMemberEpoch(memberEpoch)
.setPreviousMemberEpoch(memberEpoch)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 2, 3),
mkTopicAssignment(topicId2, 5, 6)))
@ -329,7 +416,8 @@ public class CurrentAssignmentBuilderTest {
.build();
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
.withTargetAssignment(12, new Assignment(mkAssignment(
.withMetadataImage(metadataImage)
.withTargetAssignment(targetAssignmentEpoch, new Assignment(mkAssignment(
mkTopicAssignment(topicId1, 3),
mkTopicAssignment(topicId2, 6))))
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
@ -345,8 +433,9 @@ public class CurrentAssignmentBuilderTest {
assertEquals(
new ConsumerGroupMember.Builder("member")
.setState(MemberState.UNREVOKED_PARTITIONS)
.setMemberEpoch(11)
.setPreviousMemberEpoch(10)
.setMemberEpoch(expectedMemberEpoch)
.setPreviousMemberEpoch(memberEpoch)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 3),
mkTopicAssignment(topicId2, 6)))
@ -360,19 +449,28 @@ public class CurrentAssignmentBuilderTest {
@Test
public void testUnrevokedPartitionsToUnreleasedPartitions() {
String topic1 = "topic1";
String topic2 = "topic2";
Uuid topicId1 = Uuid.randomUuid();
Uuid topicId2 = Uuid.randomUuid();
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(topicId1, topic1, 10)
.addTopic(topicId2, topic2, 10)
.buildCoordinatorMetadataImage();
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
.setState(MemberState.UNREVOKED_PARTITIONS)
.setMemberEpoch(11)
.setPreviousMemberEpoch(10)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 2, 3),
mkTopicAssignment(topicId2, 5, 6)))
.build();
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(11, new Assignment(mkAssignment(
mkTopicAssignment(topicId1, 2, 3, 4),
mkTopicAssignment(topicId2, 5, 6, 7))))
@ -391,6 +489,7 @@ public class CurrentAssignmentBuilderTest {
.setState(MemberState.UNRELEASED_PARTITIONS)
.setMemberEpoch(11)
.setPreviousMemberEpoch(11)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 2, 3),
mkTopicAssignment(topicId2, 5, 6)))
@ -401,19 +500,28 @@ public class CurrentAssignmentBuilderTest {
@Test
public void testUnreleasedPartitionsToStable() {
String topic1 = "topic1";
String topic2 = "topic2";
Uuid topicId1 = Uuid.randomUuid();
Uuid topicId2 = Uuid.randomUuid();
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(topicId1, topic1, 10)
.addTopic(topicId2, topic2, 10)
.buildCoordinatorMetadataImage();
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
.setState(MemberState.UNRELEASED_PARTITIONS)
.setMemberEpoch(11)
.setPreviousMemberEpoch(11)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 2, 3),
mkTopicAssignment(topicId2, 5, 6)))
.build();
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(12, new Assignment(mkAssignment(
mkTopicAssignment(topicId1, 2, 3),
mkTopicAssignment(topicId2, 5, 6))))
@ -425,6 +533,7 @@ public class CurrentAssignmentBuilderTest {
.setState(MemberState.STABLE)
.setMemberEpoch(12)
.setPreviousMemberEpoch(11)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 2, 3),
mkTopicAssignment(topicId2, 5, 6)))
@ -435,19 +544,28 @@ public class CurrentAssignmentBuilderTest {
@Test
public void testUnreleasedPartitionsToStableWithNewPartitions() {
String topic1 = "topic1";
String topic2 = "topic2";
Uuid topicId1 = Uuid.randomUuid();
Uuid topicId2 = Uuid.randomUuid();
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(topicId1, topic1, 10)
.addTopic(topicId2, topic2, 10)
.buildCoordinatorMetadataImage();
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
.setState(MemberState.UNRELEASED_PARTITIONS)
.setMemberEpoch(11)
.setPreviousMemberEpoch(11)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 2, 3),
mkTopicAssignment(topicId2, 5, 6)))
.build();
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(11, new Assignment(mkAssignment(
mkTopicAssignment(topicId1, 2, 3, 4),
mkTopicAssignment(topicId2, 5, 6, 7))))
@ -459,6 +577,7 @@ public class CurrentAssignmentBuilderTest {
.setState(MemberState.STABLE)
.setMemberEpoch(11)
.setPreviousMemberEpoch(11)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 2, 3, 4),
mkTopicAssignment(topicId2, 5, 6, 7)))
@ -469,19 +588,28 @@ public class CurrentAssignmentBuilderTest {
@Test
public void testUnreleasedPartitionsToUnreleasedPartitions() {
String topic1 = "topic1";
String topic2 = "topic2";
Uuid topicId1 = Uuid.randomUuid();
Uuid topicId2 = Uuid.randomUuid();
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(topicId1, topic1, 10)
.addTopic(topicId2, topic2, 10)
.buildCoordinatorMetadataImage();
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
.setState(MemberState.UNRELEASED_PARTITIONS)
.setMemberEpoch(11)
.setPreviousMemberEpoch(11)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 2, 3),
mkTopicAssignment(topicId2, 5, 6)))
.build();
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(11, new Assignment(mkAssignment(
mkTopicAssignment(topicId1, 2, 3, 4),
mkTopicAssignment(topicId2, 5, 6, 7))))
@ -493,19 +621,28 @@ public class CurrentAssignmentBuilderTest {
@Test
public void testUnreleasedPartitionsToUnrevokedPartitions() {
String topic1 = "topic1";
String topic2 = "topic2";
Uuid topicId1 = Uuid.randomUuid();
Uuid topicId2 = Uuid.randomUuid();
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(topicId1, topic1, 10)
.addTopic(topicId2, topic2, 10)
.buildCoordinatorMetadataImage();
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
.setState(MemberState.UNRELEASED_PARTITIONS)
.setMemberEpoch(11)
.setPreviousMemberEpoch(11)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 2, 3),
mkTopicAssignment(topicId2, 5, 6)))
.build();
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(12, new Assignment(mkAssignment(
mkTopicAssignment(topicId1, 3),
mkTopicAssignment(topicId2, 6))))
@ -517,6 +654,7 @@ public class CurrentAssignmentBuilderTest {
.setState(MemberState.UNREVOKED_PARTITIONS)
.setMemberEpoch(11)
.setPreviousMemberEpoch(11)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 3),
mkTopicAssignment(topicId2, 6)))
@ -530,13 +668,21 @@ public class CurrentAssignmentBuilderTest {
@Test
public void testUnknownState() {
String topic1 = "topic1";
String topic2 = "topic2";
Uuid topicId1 = Uuid.randomUuid();
Uuid topicId2 = Uuid.randomUuid();
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(topicId1, topic1, 10)
.addTopic(topicId2, topic2, 10)
.buildCoordinatorMetadataImage();
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
.setState(MemberState.UNKNOWN)
.setMemberEpoch(11)
.setPreviousMemberEpoch(11)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 3),
mkTopicAssignment(topicId2, 6)))
@ -548,6 +694,7 @@ public class CurrentAssignmentBuilderTest {
// When the member is in an unknown state, the member is first to force
// a reset of the client side member state.
assertThrows(FencedMemberEpochException.class, () -> new CurrentAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(12, new Assignment(mkAssignment(
mkTopicAssignment(topicId1, 3),
mkTopicAssignment(topicId2, 6))))
@ -556,6 +703,7 @@ public class CurrentAssignmentBuilderTest {
// Then the member rejoins with no owned partitions.
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(12, new Assignment(mkAssignment(
mkTopicAssignment(topicId1, 3),
mkTopicAssignment(topicId2, 6))))
@ -568,6 +716,7 @@ public class CurrentAssignmentBuilderTest {
.setState(MemberState.STABLE)
.setMemberEpoch(12)
.setPreviousMemberEpoch(11)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 3),
mkTopicAssignment(topicId2, 6)))
@ -575,4 +724,355 @@ public class CurrentAssignmentBuilderTest {
updatedMember
);
}
@ParameterizedTest
@CsvSource({
"10, 11, 11, false", // When advancing to a new target assignment, the assignment should
"10, 11, 11, true", // always take the subscription into account.
"10, 10, 10, true",
})
public void testStableToStableWithAssignmentTopicsNoLongerInSubscription(
int memberEpoch,
int targetAssignmentEpoch,
int expectedMemberEpoch,
boolean hasSubscriptionChanged
) {
String topic1 = "topic1";
String topic2 = "topic2";
Uuid topicId1 = Uuid.randomUuid();
Uuid topicId2 = Uuid.randomUuid();
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(topicId1, topic1, 10)
.addTopic(topicId2, topic2, 10)
.buildCoordinatorMetadataImage();
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
.setState(MemberState.STABLE)
.setMemberEpoch(memberEpoch)
.setPreviousMemberEpoch(memberEpoch)
.setSubscribedTopicNames(List.of(topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 1, 2, 3),
mkTopicAssignment(topicId2, 4, 5, 6)))
.build();
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(targetAssignmentEpoch, new Assignment(mkAssignment(
mkTopicAssignment(topicId1, 1, 2, 3),
mkTopicAssignment(topicId2, 4, 5, 6))))
.withHasSubscriptionChanged(hasSubscriptionChanged)
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
.withOwnedTopicPartitions(Arrays.asList(
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
.setTopicId(topicId2)
.setPartitions(Arrays.asList(4, 5, 6))))
.build();
assertEquals(
new ConsumerGroupMember.Builder("member")
.setState(MemberState.STABLE)
.setMemberEpoch(expectedMemberEpoch)
.setPreviousMemberEpoch(memberEpoch)
.setSubscribedTopicNames(List.of(topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId2, 4, 5, 6)))
.build(),
updatedMember
);
}
@ParameterizedTest
@CsvSource({
"10, 11, 10, false", // When advancing to a new target assignment, the assignment should always
"10, 11, 10, true", // take the subscription into account.
"10, 10, 10, true"
})
public void testStableToUnrevokedPartitionsWithAssignmentTopicsNoLongerInSubscription(
int memberEpoch,
int targetAssignmentEpoch,
int expectedMemberEpoch,
boolean hasSubscriptionChanged
) {
String topic1 = "topic1";
String topic2 = "topic2";
Uuid topicId1 = Uuid.randomUuid();
Uuid topicId2 = Uuid.randomUuid();
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(topicId1, topic1, 10)
.addTopic(topicId2, topic2, 10)
.buildCoordinatorMetadataImage();
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
.setState(MemberState.STABLE)
.setMemberEpoch(memberEpoch)
.setPreviousMemberEpoch(memberEpoch)
.setSubscribedTopicNames(List.of(topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 1, 2, 3),
mkTopicAssignment(topicId2, 4, 5, 6)))
.build();
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(targetAssignmentEpoch, new Assignment(mkAssignment(
mkTopicAssignment(topicId1, 1, 2, 3),
mkTopicAssignment(topicId2, 4, 5, 6))))
.withHasSubscriptionChanged(hasSubscriptionChanged)
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
.withOwnedTopicPartitions(Arrays.asList(
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
.setTopicId(topicId1)
.setPartitions(Arrays.asList(1, 2, 3)),
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
.setTopicId(topicId2)
.setPartitions(Arrays.asList(4, 5, 6))))
.build();
assertEquals(
new ConsumerGroupMember.Builder("member")
.setState(MemberState.UNREVOKED_PARTITIONS)
.setMemberEpoch(expectedMemberEpoch)
.setPreviousMemberEpoch(memberEpoch)
.setSubscribedTopicNames(List.of(topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId2, 4, 5, 6)))
.setPartitionsPendingRevocation(mkAssignment(
mkTopicAssignment(topicId1, 1, 2, 3)))
.build(),
updatedMember
);
}
@Test
public void testRemainsInUnrevokedPartitionsWithAssignmentTopicsNoLongerInSubscription() {
String topic1 = "topic1";
String topic2 = "topic2";
Uuid topicId1 = Uuid.randomUuid();
Uuid topicId2 = Uuid.randomUuid();
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(topicId1, topic1, 10)
.addTopic(topicId2, topic2, 10)
.buildCoordinatorMetadataImage();
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
.setState(MemberState.UNREVOKED_PARTITIONS)
.setMemberEpoch(10)
.setPreviousMemberEpoch(10)
.setSubscribedTopicNames(List.of(topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 2, 3),
mkTopicAssignment(topicId2, 5, 6)))
.setPartitionsPendingRevocation(mkAssignment(
mkTopicAssignment(topicId1, 1),
mkTopicAssignment(topicId2, 4)))
.build();
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(12, new Assignment(mkAssignment(
mkTopicAssignment(topicId1, 1, 3, 4),
mkTopicAssignment(topicId2, 6, 7))))
.withHasSubscriptionChanged(true)
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
.withOwnedTopicPartitions(Arrays.asList(
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
.setTopicId(topicId1)
.setPartitions(Arrays.asList(1, 2, 3)),
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
.setTopicId(topicId2)
.setPartitions(Arrays.asList(4, 5, 6))))
.build();
assertEquals(
new ConsumerGroupMember.Builder("member")
.setState(MemberState.UNREVOKED_PARTITIONS)
.setMemberEpoch(10)
.setPreviousMemberEpoch(10)
.setSubscribedTopicNames(List.of(topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId2, 5, 6)))
.setPartitionsPendingRevocation(mkAssignment(
mkTopicAssignment(topicId1, 1, 2, 3),
mkTopicAssignment(topicId2, 4)))
.build(),
updatedMember
);
}
@Test
public void testSubscribedTopicNameAndUnresolvedRegularExpression() {
String fooTopic = "foo";
String barTopic = "bar";
Uuid fooTopicId = Uuid.randomUuid();
Uuid barTopicId = Uuid.randomUuid();
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopic, 10)
.addTopic(barTopicId, barTopic, 10)
.buildCoordinatorMetadataImage();
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
.setState(MemberState.STABLE)
.setMemberEpoch(10)
.setPreviousMemberEpoch(10)
.setSubscribedTopicNames(List.of(fooTopic))
.setSubscribedTopicRegex("bar*")
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(fooTopicId, 1, 2, 3),
mkTopicAssignment(barTopicId, 4, 5, 6)))
.build();
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(10, new Assignment(mkAssignment(
mkTopicAssignment(fooTopicId, 1, 2, 3),
mkTopicAssignment(barTopicId, 4, 5, 6))))
.withHasSubscriptionChanged(true)
.withResolvedRegularExpressions(Map.of())
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
.withOwnedTopicPartitions(Arrays.asList(
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
.setTopicId(fooTopicId)
.setPartitions(Arrays.asList(1, 2, 3)),
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
.setTopicId(barTopicId)
.setPartitions(Arrays.asList(4, 5, 6))))
.build();
assertEquals(
new ConsumerGroupMember.Builder("member")
.setState(MemberState.UNREVOKED_PARTITIONS)
.setMemberEpoch(10)
.setPreviousMemberEpoch(10)
.setSubscribedTopicNames(List.of(fooTopic))
.setSubscribedTopicRegex("bar*")
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(fooTopicId, 1, 2, 3)))
.setPartitionsPendingRevocation(mkAssignment(
mkTopicAssignment(barTopicId, 4, 5, 6)))
.build(),
updatedMember
);
}
@Test
public void testUnresolvedRegularExpression() {
String fooTopic = "foo";
String barTopic = "bar";
Uuid fooTopicId = Uuid.randomUuid();
Uuid barTopicId = Uuid.randomUuid();
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopic, 10)
.addTopic(barTopicId, barTopic, 10)
.buildCoordinatorMetadataImage();
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
.setState(MemberState.STABLE)
.setMemberEpoch(10)
.setPreviousMemberEpoch(10)
.setSubscribedTopicNames(List.of())
.setSubscribedTopicRegex("bar*")
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(fooTopicId, 1, 2, 3),
mkTopicAssignment(barTopicId, 4, 5, 6)))
.build();
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(10, new Assignment(mkAssignment(
mkTopicAssignment(fooTopicId, 1, 2, 3),
mkTopicAssignment(barTopicId, 4, 5, 6))))
.withHasSubscriptionChanged(true)
.withResolvedRegularExpressions(Map.of())
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
.withOwnedTopicPartitions(Arrays.asList(
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
.setTopicId(fooTopicId)
.setPartitions(Arrays.asList(1, 2, 3)),
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
.setTopicId(barTopicId)
.setPartitions(Arrays.asList(4, 5, 6))))
.build();
assertEquals(
new ConsumerGroupMember.Builder("member")
.setState(MemberState.UNREVOKED_PARTITIONS)
.setMemberEpoch(10)
.setPreviousMemberEpoch(10)
.setSubscribedTopicNames(List.of())
.setSubscribedTopicRegex("bar*")
.setAssignedPartitions(mkAssignment())
.setPartitionsPendingRevocation(mkAssignment(
mkTopicAssignment(fooTopicId, 1, 2, 3),
mkTopicAssignment(barTopicId, 4, 5, 6)))
.build(),
updatedMember
);
}
@Test
public void testSubscribedTopicNameAndResolvedRegularExpression() {
String fooTopic = "foo";
String barTopic = "bar";
Uuid fooTopicId = Uuid.randomUuid();
Uuid barTopicId = Uuid.randomUuid();
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopic, 10)
.addTopic(barTopicId, barTopic, 10)
.buildCoordinatorMetadataImage();
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
.setState(MemberState.STABLE)
.setMemberEpoch(10)
.setPreviousMemberEpoch(10)
.setSubscribedTopicNames(List.of(fooTopic))
.setSubscribedTopicRegex("bar*")
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(fooTopicId, 1, 2, 3),
mkTopicAssignment(barTopicId, 4, 5, 6)))
.build();
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(10, new Assignment(mkAssignment(
mkTopicAssignment(fooTopicId, 1, 2, 3),
mkTopicAssignment(barTopicId, 4, 5, 6))))
.withHasSubscriptionChanged(true)
.withResolvedRegularExpressions(Map.of(
"bar*", new ResolvedRegularExpression(
Set.of("bar"),
12345L,
0L
)
))
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
.withOwnedTopicPartitions(Arrays.asList(
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
.setTopicId(fooTopicId)
.setPartitions(Arrays.asList(1, 2, 3)),
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
.setTopicId(barTopicId)
.setPartitions(Arrays.asList(4, 5, 6))))
.build();
assertEquals(
new ConsumerGroupMember.Builder("member")
.setState(MemberState.STABLE)
.setMemberEpoch(10)
.setPreviousMemberEpoch(10)
.setSubscribedTopicNames(List.of(fooTopic))
.setSubscribedTopicRegex("bar*")
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(fooTopicId, 1, 2, 3),
mkTopicAssignment(barTopicId, 4, 5, 6)))
.build(),
updatedMember
);
}
}

View File

@ -17,10 +17,16 @@
package org.apache.kafka.coordinator.group.modern.share;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.common.runtime.MetadataImageBuilder;
import org.apache.kafka.coordinator.group.modern.Assignment;
import org.apache.kafka.coordinator.group.modern.MemberState;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.CsvSource;
import java.util.List;
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
@ -30,19 +36,28 @@ public class ShareGroupAssignmentBuilderTest {
@Test
public void testStableToStable() {
String topic1 = "topic1";
String topic2 = "topic2";
Uuid topicId1 = Uuid.randomUuid();
Uuid topicId2 = Uuid.randomUuid();
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(topicId1, topic1, 10)
.addTopic(topicId2, topic2, 10)
.buildCoordinatorMetadataImage();
ShareGroupMember member = new ShareGroupMember.Builder("member")
.setState(MemberState.STABLE)
.setMemberEpoch(10)
.setPreviousMemberEpoch(10)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 1, 2, 3),
mkTopicAssignment(topicId2, 4, 5, 6)))
.build();
ShareGroupMember updatedMember = new ShareGroupAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(11, new Assignment(mkAssignment(
mkTopicAssignment(topicId1, 1, 2, 3),
mkTopicAssignment(topicId2, 4, 5, 6))))
@ -53,6 +68,7 @@ public class ShareGroupAssignmentBuilderTest {
.setState(MemberState.STABLE)
.setMemberEpoch(11)
.setPreviousMemberEpoch(10)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 1, 2, 3),
mkTopicAssignment(topicId2, 4, 5, 6)))
@ -63,19 +79,28 @@ public class ShareGroupAssignmentBuilderTest {
@Test
public void testStableToStableWithNewPartitions() {
String topic1 = "topic1";
String topic2 = "topic2";
Uuid topicId1 = Uuid.randomUuid();
Uuid topicId2 = Uuid.randomUuid();
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(topicId1, topic1, 10)
.addTopic(topicId2, topic2, 10)
.buildCoordinatorMetadataImage();
ShareGroupMember member = new ShareGroupMember.Builder("member")
.setState(MemberState.STABLE)
.setMemberEpoch(10)
.setPreviousMemberEpoch(10)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 1, 2, 3),
mkTopicAssignment(topicId2, 4, 5, 6)))
.build();
ShareGroupMember updatedMember = new ShareGroupAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(11, new Assignment(mkAssignment(
mkTopicAssignment(topicId1, 1, 2, 3, 4),
mkTopicAssignment(topicId2, 4, 5, 6, 7))))
@ -86,6 +111,7 @@ public class ShareGroupAssignmentBuilderTest {
.setState(MemberState.STABLE)
.setMemberEpoch(11)
.setPreviousMemberEpoch(10)
.setSubscribedTopicNames(List.of(topic1, topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 1, 2, 3, 4),
mkTopicAssignment(topicId2, 4, 5, 6, 7)))
@ -93,4 +119,56 @@ public class ShareGroupAssignmentBuilderTest {
updatedMember
);
}
@ParameterizedTest
@CsvSource({
"10, 11, false", // When advancing to a new target assignment, the assignment should always
"10, 11, true", // take the subscription into account.
"10, 10, true"
})
public void testStableToStableWithAssignmentTopicsNoLongerInSubscription(
int memberEpoch,
int targetAssignmentEpoch,
boolean hasSubscriptionChanged
) {
String topic1 = "topic1";
String topic2 = "topic2";
Uuid topicId1 = Uuid.randomUuid();
Uuid topicId2 = Uuid.randomUuid();
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(topicId1, topic1, 10)
.addTopic(topicId2, topic2, 10)
.buildCoordinatorMetadataImage();
ShareGroupMember member = new ShareGroupMember.Builder("member")
.setState(MemberState.STABLE)
.setMemberEpoch(memberEpoch)
.setPreviousMemberEpoch(memberEpoch)
.setSubscribedTopicNames(List.of(topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId1, 1, 2, 3),
mkTopicAssignment(topicId2, 4, 5, 6)))
.build();
ShareGroupMember updatedMember = new ShareGroupAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(targetAssignmentEpoch, new Assignment(mkAssignment(
mkTopicAssignment(topicId1, 1, 2, 3),
mkTopicAssignment(topicId2, 4, 5, 6))))
.withHasSubscriptionChanged(hasSubscriptionChanged)
.build();
assertEquals(
new ShareGroupMember.Builder("member")
.setState(MemberState.STABLE)
.setMemberEpoch(targetAssignmentEpoch)
.setPreviousMemberEpoch(memberEpoch)
.setSubscribedTopicNames(List.of(topic2))
.setAssignedPartitions(mkAssignment(
mkTopicAssignment(topicId2, 4, 5, 6)))
.build(),
updatedMember
);
}
}

View File

@ -0,0 +1,171 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.jmh.assignor;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.group.modern.Assignment;
import org.apache.kafka.coordinator.group.modern.MemberState;
import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroupMember;
import org.apache.kafka.coordinator.group.modern.consumer.CurrentAssignmentBuilder;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
import org.openjdk.jmh.annotations.Level;
import org.openjdk.jmh.annotations.Measurement;
import org.openjdk.jmh.annotations.Mode;
import org.openjdk.jmh.annotations.OutputTimeUnit;
import org.openjdk.jmh.annotations.Param;
import org.openjdk.jmh.annotations.Scope;
import org.openjdk.jmh.annotations.Setup;
import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.annotations.Threads;
import org.openjdk.jmh.annotations.Warmup;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
@State(Scope.Benchmark)
@Fork(value = 1)
@Warmup(iterations = 5)
@Measurement(iterations = 5)
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public class CurrentAssignmentBuilderBenchmark {
@Param({"5", "50"})
private int partitionsPerTopic;
@Param({"10", "100", "1000"})
private int topicCount;
private List<String> topicNames;
private List<Uuid> topicIds;
private CoordinatorMetadataImage metadataImage;
private ConsumerGroupMember member;
private ConsumerGroupMember memberWithUnsubscribedTopics;
private Assignment targetAssignment;
@Setup(Level.Trial)
public void setup() {
setupTopics();
setupMember();
setupTargetAssignment();
}
private void setupTopics() {
topicNames = AssignorBenchmarkUtils.createTopicNames(topicCount);
topicIds = new ArrayList<>(topicCount);
metadataImage = AssignorBenchmarkUtils.createMetadataImage(topicNames, partitionsPerTopic);
for (String topicName : topicNames) {
Uuid topicId = metadataImage.topicMetadata(topicName).get().id();
topicIds.add(topicId);
}
}
private void setupMember() {
Map<Uuid, Set<Integer>> assignedPartitions = new HashMap<>();
for (Uuid topicId : topicIds) {
Set<Integer> partitions = IntStream.range(0, partitionsPerTopic)
.boxed()
.collect(Collectors.toSet());
assignedPartitions.put(topicId, partitions);
}
ConsumerGroupMember.Builder memberBuilder = new ConsumerGroupMember.Builder("member")
.setState(MemberState.STABLE)
.setMemberEpoch(10)
.setPreviousMemberEpoch(10)
.setSubscribedTopicNames(topicNames)
.setAssignedPartitions(assignedPartitions);
member = memberBuilder.build();
memberWithUnsubscribedTopics = memberBuilder
.setSubscribedTopicNames(topicNames.subList(0, topicNames.size() - 1))
.build();
}
private void setupTargetAssignment() {
Map<Uuid, Set<Integer>> assignedPartitions = new HashMap<>();
for (Uuid topicId : topicIds) {
Set<Integer> partitions = IntStream.range(0, partitionsPerTopic)
.boxed()
.collect(Collectors.toSet());
assignedPartitions.put(topicId, partitions);
}
targetAssignment = new Assignment(assignedPartitions);
}
@Benchmark
@Threads(1)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public ConsumerGroupMember stableToStableWithNoChange() {
return new CurrentAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(member.memberEpoch(), targetAssignment)
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
.build();
}
@Benchmark
@Threads(1)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public ConsumerGroupMember stableToStableWithNewTargetAssignment() {
return new CurrentAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(member.memberEpoch() + 1, targetAssignment)
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
.build();
}
@Benchmark
@Threads(1)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public ConsumerGroupMember stableToStableWithSubscriptionChange() {
return new CurrentAssignmentBuilder(member)
.withMetadataImage(metadataImage)
.withTargetAssignment(member.memberEpoch(), targetAssignment)
.withHasSubscriptionChanged(true)
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
.build();
}
@Benchmark
@Threads(1)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public ConsumerGroupMember stableToUnrevokedPartitionsWithSubscriptionChange() {
return new CurrentAssignmentBuilder(memberWithUnsubscribedTopics)
.withMetadataImage(metadataImage)
.withTargetAssignment(memberWithUnsubscribedTopics.memberEpoch(), targetAssignment)
.withHasSubscriptionChanged(true)
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
.build();
}
}

View File

@ -494,12 +494,10 @@ public class PartitionChangeBuilder {
private void maybeUpdateLastKnownLeader(PartitionChangeRecord record) {
if (!useLastKnownLeaderInBalancedRecovery || !eligibleLeaderReplicasEnabled) return;
if (record.isr() != null && record.isr().isEmpty() && (partition.lastKnownElr.length != 1 ||
partition.lastKnownElr[0] != partition.leader)) {
if (record.leader() == NO_LEADER && partition.lastKnownElr.length == 0) {
// Only update the last known leader when the first time the partition becomes leaderless.
record.setLastKnownElr(List.of(partition.leader));
} else if ((record.leader() >= 0 || (partition.leader != NO_LEADER && record.leader() != NO_LEADER))
&& partition.lastKnownElr.length > 0) {
} else if (record.leader() >= 0 && partition.lastKnownElr.length > 0) {
// Clear the LastKnownElr field if the partition will have or continues to have a valid leader.
record.setLastKnownElr(List.of());
}

View File

@ -822,6 +822,48 @@ public class PartitionChangeBuilderTest {
}
}
@Test
public void testEligibleLeaderReplicas_lastKnownElrShouldBePopulatedWhenNoLeader() {
PartitionRegistration partition = new PartitionRegistration.Builder()
.setReplicas(new int[] {1, 2, 3})
.setDirectories(new Uuid[] {
DirectoryId.UNASSIGNED,
DirectoryId.UNASSIGNED,
DirectoryId.UNASSIGNED
})
.setIsr(new int[] {1})
.setElr(new int[] {2})
.setLeader(1)
.setLeaderRecoveryState(LeaderRecoveryState.RECOVERED)
.setLeaderEpoch(100)
.setPartitionEpoch(200)
.build();
short version = 2; // ELR supported
Uuid topicId = Uuid.fromString("FbrrdcfiR-KC2CPSTHaJrg");
// No replica is acceptable as leader, so election yields NO_LEADER.
// We intentionally do not change target ISR so record.isr remains null.
PartitionChangeBuilder builder = new PartitionChangeBuilder(partition, topicId, 0, r -> false,
metadataVersionForPartitionChangeRecordVersion(version), 3)
.setElection(Election.PREFERRED)
.setEligibleLeaderReplicasEnabled(isElrEnabled(version))
.setDefaultDirProvider(DEFAULT_DIR_PROVIDER)
.setUseLastKnownLeaderInBalancedRecovery(true);
ApiMessageAndVersion change = builder.build().get();
PartitionChangeRecord record = (PartitionChangeRecord) change.message();
assertEquals(NO_LEADER, record.leader());
// There is no ISR update if we do not perform the leader verification on the ISR members.
assertNull(record.isr(), record.toString());
assertEquals(1, record.lastKnownElr().size(), record.toString());
assertEquals(1, record.lastKnownElr().get(0), record.toString());
partition = partition.merge((PartitionChangeRecord) builder.build().get().message());
assertArrayEquals(new int[] {1}, partition.lastKnownElr);
}
@ParameterizedTest
@MethodSource("partitionChangeRecordVersions")
public void testEligibleLeaderReplicas_IsrExpandAboveMinISR(short version) {

View File

@ -0,0 +1,54 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.logger;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Simple class that sets logIdent appropriately depending on whether the state change logger is being used in the
* context of the broker (e.g. ReplicaManager and Partition).
*/
public class StateChangeLogger {
private static final Logger LOGGER = LoggerFactory.getLogger("state.change.logger");
private final String logIdent;
public StateChangeLogger(int brokerId) {
this.logIdent = String.format("[Broker id=%d] ", brokerId);
}
public void trace(String message) {
LOGGER.info("{}{}", logIdent, message);
}
public void info(String message) {
LOGGER.info("{}{}", logIdent, message);
}
public void warn(String message) {
LOGGER.warn("{}{}", logIdent, message);
}
public void error(String message) {
LOGGER.error("{}{}", logIdent, message);
}
public void error(String message, Throwable e) {
LOGGER.error("{}{}", logIdent, message, e);
}
}

View File

@ -15,13 +15,16 @@
* limitations under the License.
*/
package kafka.server;
package org.apache.kafka.server;
import org.apache.kafka.clients.admin.AddRaftVoterOptions;
import org.apache.kafka.clients.admin.Admin;
import org.apache.kafka.clients.admin.FeatureMetadata;
import org.apache.kafka.clients.admin.QuorumInfo;
import org.apache.kafka.clients.admin.RaftVoterEndpoint;
import org.apache.kafka.clients.admin.RemoveRaftVoterOptions;
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.common.test.api.TestKitDefaults;
@ -29,10 +32,12 @@ import org.apache.kafka.raft.QuorumConfig;
import org.apache.kafka.server.common.KRaftVersion;
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.TreeMap;
@ -41,6 +46,7 @@ import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
@Tag("integration")
public class ReconfigurableQuorumIntegrationTest {
static void checkKRaftVersions(Admin admin, short finalized) throws Exception {
FeatureMetadata featureMetadata = admin.describeFeatures().featureMetadata().get();
@ -70,7 +76,7 @@ public class ReconfigurableQuorumIntegrationTest {
).build()) {
cluster.format();
cluster.startup();
try (Admin admin = Admin.create(cluster.clientProperties())) {
try (var admin = Admin.create(cluster.clientProperties())) {
TestUtils.retryOnExceptionWithTimeout(30_000, () -> {
checkKRaftVersions(admin, KRaftVersion.KRAFT_VERSION_0.featureLevel());
});
@ -88,7 +94,7 @@ public class ReconfigurableQuorumIntegrationTest {
).setStandalone(true).build()) {
cluster.format();
cluster.startup();
try (Admin admin = Admin.create(cluster.clientProperties())) {
try (var admin = Admin.create(cluster.clientProperties())) {
TestUtils.retryOnExceptionWithTimeout(30_000, () -> {
checkKRaftVersions(admin, KRaftVersion.KRAFT_VERSION_1.featureLevel());
});
@ -126,7 +132,7 @@ public class ReconfigurableQuorumIntegrationTest {
) {
cluster.format();
cluster.startup();
try (Admin admin = Admin.create(cluster.clientProperties())) {
try (var admin = Admin.create(cluster.clientProperties())) {
TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> {
Map<Integer, Uuid> voters = findVoterDirs(admin);
assertEquals(Set.of(3000, 3001, 3002), voters.keySet());
@ -161,7 +167,7 @@ public class ReconfigurableQuorumIntegrationTest {
) {
cluster.format();
cluster.startup();
try (Admin admin = Admin.create(cluster.clientProperties())) {
try (var admin = Admin.create(cluster.clientProperties())) {
TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> {
Map<Integer, Uuid> voters = findVoterDirs(admin);
assertEquals(Set.of(3000, 3001, 3002, 3003), voters.keySet());
@ -200,7 +206,7 @@ public class ReconfigurableQuorumIntegrationTest {
) {
cluster.format();
cluster.startup();
try (Admin admin = Admin.create(cluster.clientProperties())) {
try (var admin = Admin.create(cluster.clientProperties())) {
TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> {
Map<Integer, Uuid> voters = findVoterDirs(admin);
assertEquals(Set.of(3000, 3001, 3002), voters.keySet());
@ -238,7 +244,7 @@ public class ReconfigurableQuorumIntegrationTest {
) {
cluster.format();
cluster.startup();
try (Admin admin = Admin.create(cluster.clientProperties())) {
try (var admin = Admin.create(cluster.clientProperties())) {
TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> {
Map<Integer, Uuid> voters = findVoterDirs(admin);
assertEquals(Set.of(3000, 3001, 3002), voters.keySet());
@ -249,4 +255,95 @@ public class ReconfigurableQuorumIntegrationTest {
}
}
}
}
@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 (var admin = Admin.create(cluster.clientProperties())) {
TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> {
Map<Integer, Uuid> voters = findVoterDirs(admin);
assertEquals(Set.of(3000, 3001, 3002), voters.keySet());
for (int replicaId : new int[] {3000, 3001, 3002}) {
assertNotEquals(Uuid.ZERO_UUID, voters.get(replicaId));
}
});
Uuid dirId = cluster.nodes().controllerNodes().get(3000).metadataDirectoryId();
admin.removeRaftVoter(
3000,
dirId,
new RemoveRaftVoterOptions().setClusterId(Optional.of("test-cluster"))
).all().get();
TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> {
Map<Integer, Uuid> voters = findVoterDirs(admin);
assertEquals(Set.of(3001, 3002), voters.keySet());
for (int replicaId : new int[] {3001, 3002}) {
assertNotEquals(Uuid.ZERO_UUID, voters.get(replicaId));
}
});
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 (var 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

@ -72,9 +72,10 @@ import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.MethodSource;
import org.junit.jupiter.params.provider.ValueSource;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -169,6 +170,15 @@ public class EosIntegrationTest {
private String stateTmpDir;
private static java.util.stream.Stream<Arguments> groupProtocolAndProcessingThreadsParameters() {
return java.util.stream.Stream.of(
Arguments.of("classic", true),
Arguments.of("classic", false),
Arguments.of("streams", true),
Arguments.of("streams", false)
);
}
@BeforeEach
public void createTopics() throws Exception {
applicationId = "appId-" + TEST_NUMBER.getAndIncrement();
@ -181,16 +191,19 @@ public class EosIntegrationTest {
CLUSTER.createTopic(MULTI_PARTITION_INPUT_TOPIC, NUM_TOPIC_PARTITIONS, 1);
CLUSTER.createTopic(MULTI_PARTITION_THROUGH_TOPIC, NUM_TOPIC_PARTITIONS, 1);
CLUSTER.createTopic(MULTI_PARTITION_OUTPUT_TOPIC, NUM_TOPIC_PARTITIONS, 1);
CLUSTER.setGroupStandbyReplicas(applicationId, 1);
}
@Test
public void shouldBeAbleToRunWithEosEnabled() throws Exception {
runSimpleCopyTest(1, SINGLE_PARTITION_INPUT_TOPIC, null, SINGLE_PARTITION_OUTPUT_TOPIC, false);
@ParameterizedTest
@ValueSource(strings = {"classic", "streams"})
public void shouldBeAbleToRunWithEosEnabled(final String groupProtocol) throws Exception {
runSimpleCopyTest(1, SINGLE_PARTITION_INPUT_TOPIC, null, SINGLE_PARTITION_OUTPUT_TOPIC, false, groupProtocol);
}
@Test
public void shouldCommitCorrectOffsetIfInputTopicIsTransactional() throws Exception {
runSimpleCopyTest(1, SINGLE_PARTITION_INPUT_TOPIC, null, SINGLE_PARTITION_OUTPUT_TOPIC, true);
@ParameterizedTest
@ValueSource(strings = {"classic", "streams"})
public void shouldCommitCorrectOffsetIfInputTopicIsTransactional(final String groupProtocol) throws Exception {
runSimpleCopyTest(1, SINGLE_PARTITION_INPUT_TOPIC, null, SINGLE_PARTITION_OUTPUT_TOPIC, true, groupProtocol);
try (final Admin adminClient = Admin.create(mkMap(mkEntry(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers())));
final Consumer<byte[], byte[]> consumer = new KafkaConsumer<>(mkMap(
@ -215,36 +228,42 @@ public class EosIntegrationTest {
}
}
@Test
public void shouldBeAbleToRestartAfterClose() throws Exception {
runSimpleCopyTest(2, SINGLE_PARTITION_INPUT_TOPIC, null, SINGLE_PARTITION_OUTPUT_TOPIC, false);
@ParameterizedTest
@ValueSource(strings = {"classic", "streams"})
public void shouldBeAbleToRestartAfterClose(final String groupProtocol) throws Exception {
runSimpleCopyTest(2, SINGLE_PARTITION_INPUT_TOPIC, null, SINGLE_PARTITION_OUTPUT_TOPIC, false, groupProtocol);
}
@Test
public void shouldBeAbleToCommitToMultiplePartitions() throws Exception {
runSimpleCopyTest(1, SINGLE_PARTITION_INPUT_TOPIC, null, MULTI_PARTITION_OUTPUT_TOPIC, false);
@ParameterizedTest
@ValueSource(strings = {"classic", "streams"})
public void shouldBeAbleToCommitToMultiplePartitions(final String groupProtocol) throws Exception {
runSimpleCopyTest(1, SINGLE_PARTITION_INPUT_TOPIC, null, MULTI_PARTITION_OUTPUT_TOPIC, false, groupProtocol);
}
@Test
public void shouldBeAbleToCommitMultiplePartitionOffsets() throws Exception {
runSimpleCopyTest(1, MULTI_PARTITION_INPUT_TOPIC, null, SINGLE_PARTITION_OUTPUT_TOPIC, false);
@ParameterizedTest
@ValueSource(strings = {"classic", "streams"})
public void shouldBeAbleToCommitMultiplePartitionOffsets(final String groupProtocol) throws Exception {
runSimpleCopyTest(1, MULTI_PARTITION_INPUT_TOPIC, null, SINGLE_PARTITION_OUTPUT_TOPIC, false, groupProtocol);
}
@Test
public void shouldBeAbleToRunWithTwoSubtopologies() throws Exception {
runSimpleCopyTest(1, SINGLE_PARTITION_INPUT_TOPIC, SINGLE_PARTITION_THROUGH_TOPIC, SINGLE_PARTITION_OUTPUT_TOPIC, false);
@ParameterizedTest
@ValueSource(strings = {"classic", "streams"})
public void shouldBeAbleToRunWithTwoSubtopologies(final String groupProtocol) throws Exception {
runSimpleCopyTest(1, SINGLE_PARTITION_INPUT_TOPIC, SINGLE_PARTITION_THROUGH_TOPIC, SINGLE_PARTITION_OUTPUT_TOPIC, false, groupProtocol);
}
@Test
public void shouldBeAbleToRunWithTwoSubtopologiesAndMultiplePartitions() throws Exception {
runSimpleCopyTest(1, MULTI_PARTITION_INPUT_TOPIC, MULTI_PARTITION_THROUGH_TOPIC, MULTI_PARTITION_OUTPUT_TOPIC, false);
@ParameterizedTest
@ValueSource(strings = {"classic", "streams"})
public void shouldBeAbleToRunWithTwoSubtopologiesAndMultiplePartitions(final String groupProtocol) throws Exception {
runSimpleCopyTest(1, MULTI_PARTITION_INPUT_TOPIC, MULTI_PARTITION_THROUGH_TOPIC, MULTI_PARTITION_OUTPUT_TOPIC, false, groupProtocol);
}
private void runSimpleCopyTest(final int numberOfRestarts,
final String inputTopic,
final String throughTopic,
final String outputTopic,
final boolean inputTopicTransactional) throws Exception {
final boolean inputTopicTransactional,
final String groupProtocol) throws Exception {
final StreamsBuilder builder = new StreamsBuilder();
final KStream<Long, Long> input = builder.stream(inputTopic);
KStream<Long, Long> output = input;
@ -263,6 +282,7 @@ public class EosIntegrationTest {
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG), "earliest");
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), MAX_POLL_INTERVAL_MS - 1);
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG), MAX_POLL_INTERVAL_MS);
properties.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol);
for (int i = 0; i < numberOfRestarts; ++i) {
final Properties config = StreamsTestUtils.getStreamsConfig(
@ -326,8 +346,9 @@ public class EosIntegrationTest {
return recordsPerKey;
}
@Test
public void shouldBeAbleToPerformMultipleTransactions() throws Exception {
@ParameterizedTest
@ValueSource(strings = {"classic", "streams"})
public void shouldBeAbleToPerformMultipleTransactions(final String groupProtocol) throws Exception {
final StreamsBuilder builder = new StreamsBuilder();
builder.stream(SINGLE_PARTITION_INPUT_TOPIC).to(SINGLE_PARTITION_OUTPUT_TOPIC);
@ -337,6 +358,7 @@ public class EosIntegrationTest {
properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
properties.put(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1000");
properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
properties.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol);
final Properties config = StreamsTestUtils.getStreamsConfig(
applicationId,
@ -374,8 +396,8 @@ public class EosIntegrationTest {
}
@ParameterizedTest
@ValueSource(booleans = {true, false})
public void shouldNotViolateEosIfOneTaskFails(final boolean processingThreadsEnabled) throws Exception {
@MethodSource("groupProtocolAndProcessingThreadsParameters")
public void shouldNotViolateEosIfOneTaskFails(final String groupProtocol, final boolean processingThreadsEnabled) throws Exception {
// this test writes 10 + 5 + 5 records per partition (running with 2 partitions)
// the app is supposed to copy all 40 records into the output topic
@ -386,7 +408,7 @@ public class EosIntegrationTest {
// -> the failure only kills one thread
// after fail over, we should read 40 committed records (even if 50 record got written)
try (final KafkaStreams streams = getKafkaStreams("dummy", false, "appDir", 2, processingThreadsEnabled)) {
try (final KafkaStreams streams = getKafkaStreams("dummy", false, "appDir", 2, groupProtocol, processingThreadsEnabled)) {
startApplicationAndWaitUntilRunning(streams);
final List<KeyValue<Long, Long>> committedDataBeforeFailure = prepareData(0L, 10L, 0L, 1L);
@ -476,8 +498,8 @@ public class EosIntegrationTest {
}
@ParameterizedTest
@ValueSource(booleans = {true, false})
public void shouldNotViolateEosIfOneTaskFailsWithState(final boolean processingThreadsEnabled) throws Exception {
@MethodSource("groupProtocolAndProcessingThreadsParameters")
public void shouldNotViolateEosIfOneTaskFailsWithState(final String groupProtocol, final boolean processingThreadsEnabled) throws Exception {
// this test updates a store with 10 + 5 + 5 records per partition (running with 2 partitions)
// the app is supposed to emit all 40 update records into the output topic
@ -493,7 +515,7 @@ public class EosIntegrationTest {
// We need more processing time under "with state" situation, so increasing the max.poll.interval.ms
// to avoid unexpected rebalance during test, which will cause unexpected fail over triggered
try (final KafkaStreams streams = getKafkaStreams("dummy", true, "appDir", 2, processingThreadsEnabled)) {
try (final KafkaStreams streams = getKafkaStreams("dummy", true, "appDir", 2, groupProtocol, processingThreadsEnabled)) {
startApplicationAndWaitUntilRunning(streams);
final List<KeyValue<Long, Long>> committedDataBeforeFailure = prepareData(0L, 10L, 0L, 1L);
@ -594,8 +616,8 @@ public class EosIntegrationTest {
}
@ParameterizedTest
@ValueSource(booleans = {true, false})
public void shouldNotViolateEosIfOneTaskGetsFencedUsingIsolatedAppInstances(final boolean processingThreadsEnabled) throws Exception {
@MethodSource("groupProtocolAndProcessingThreadsParameters")
public void shouldNotViolateEosIfOneTaskGetsFencedUsingIsolatedAppInstances(final String groupProtocol, final boolean processingThreadsEnabled) throws Exception {
// this test writes 10 + 5 + 5 + 10 records per partition (running with 2 partitions)
// the app is supposed to copy all 60 records into the output topic
//
@ -607,10 +629,9 @@ public class EosIntegrationTest {
//
// afterward, the "stalling" thread resumes, and another rebalance should get triggered
// we write the remaining 20 records and verify to read 60 result records
try (
final KafkaStreams streams1 = getKafkaStreams("streams1", false, "appDir1", 1, processingThreadsEnabled);
final KafkaStreams streams2 = getKafkaStreams("streams2", false, "appDir2", 1, processingThreadsEnabled)
final KafkaStreams streams1 = getKafkaStreams("streams1", false, "appDir1", 1, groupProtocol, processingThreadsEnabled);
final KafkaStreams streams2 = getKafkaStreams("streams2", false, "appDir2", 1, groupProtocol, processingThreadsEnabled)
) {
startApplicationAndWaitUntilRunning(streams1);
startApplicationAndWaitUntilRunning(streams2);
@ -667,13 +688,10 @@ public class EosIntegrationTest {
"Expected a host to start stalling"
);
final String observedStallingHost = stallingHost.get();
final KafkaStreams stallingInstance;
final KafkaStreams remainingInstance;
if ("streams1".equals(observedStallingHost)) {
stallingInstance = streams1;
remainingInstance = streams2;
} else if ("streams2".equals(observedStallingHost)) {
stallingInstance = streams2;
remainingInstance = streams1;
} else {
throw new IllegalArgumentException("unexpected host name: " + observedStallingHost);
@ -683,8 +701,7 @@ public class EosIntegrationTest {
// the assignment is. We only really care that the remaining instance only sees one host
// that owns both partitions.
waitForCondition(
() -> stallingInstance.metadataForAllStreamsClients().size() == 2
&& remainingInstance.metadataForAllStreamsClients().size() == 1
() -> remainingInstance.metadataForAllStreamsClients().size() == 1
&& remainingInstance.metadataForAllStreamsClients().iterator().next().topicPartitions().size() == 2,
MAX_WAIT_TIME_MS,
() -> "Should have rebalanced.\n" +
@ -755,12 +772,12 @@ public class EosIntegrationTest {
}
@ParameterizedTest
@ValueSource(booleans = {true, false})
public void shouldWriteLatestOffsetsToCheckpointOnShutdown(final boolean processingThreadsEnabled) throws Exception {
@MethodSource("groupProtocolAndProcessingThreadsParameters")
public void shouldWriteLatestOffsetsToCheckpointOnShutdown(final String groupProtocol, final boolean processingThreadsEnabled) throws Exception {
final List<KeyValue<Long, Long>> writtenData = prepareData(0L, 10, 0L, 1L);
final List<KeyValue<Long, Long>> expectedResult = computeExpectedResult(writtenData);
try (final KafkaStreams streams = getKafkaStreams("streams", true, "appDir", 1, processingThreadsEnabled)) {
try (final KafkaStreams streams = getKafkaStreams("streams", true, "appDir", 1, groupProtocol, processingThreadsEnabled)) {
writeInputData(writtenData);
startApplicationAndWaitUntilRunning(streams);
@ -787,9 +804,9 @@ public class EosIntegrationTest {
}
@ParameterizedTest
@ValueSource(booleans = {true, false})
@MethodSource("groupProtocolAndProcessingThreadsParameters")
public void shouldCheckpointRestoredOffsetsWhenClosingCleanDuringRestoring(
final boolean processingThreadsEnabled) throws Exception {
final String groupProtocol, final boolean processingThreadsEnabled) throws Exception {
final Properties streamsConfiguration = new Properties();
streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId);
@ -801,6 +818,7 @@ public class EosIntegrationTest {
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(applicationId).getPath());
streamsConfiguration.put(InternalConfig.PROCESSING_THREADS_ENABLED, processingThreadsEnabled);
streamsConfiguration.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol);
streamsConfiguration.put(StreamsConfig.restoreConsumerPrefix(ConsumerConfig.MAX_POLL_RECORDS_CONFIG), 100);
final String stateStoreName = "stateStore";
@ -934,8 +952,13 @@ public class EosIntegrationTest {
static final AtomicReference<TaskId> TASK_WITH_DATA = new AtomicReference<>();
static final AtomicBoolean DID_REVOKE_IDLE_TASK = new AtomicBoolean(false);
@Test
public void shouldNotCommitActiveTasksWithPendingInputIfRevokedTaskDidNotMakeProgress() throws Exception {
@ParameterizedTest
@ValueSource(strings = {"classic", "streams"})
public void shouldNotCommitActiveTasksWithPendingInputIfRevokedTaskDidNotMakeProgress(final String groupProtocol) throws Exception {
// Reset static variables to ensure test isolation
TASK_WITH_DATA.set(null);
DID_REVOKE_IDLE_TASK.set(false);
final AtomicBoolean requestCommit = new AtomicBoolean(false);
final StreamsBuilder builder = new StreamsBuilder();
@ -970,6 +993,7 @@ public class EosIntegrationTest {
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG), MAX_POLL_INTERVAL_MS);
properties.put(StreamsConfig.producerPrefix(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG), Integer.MAX_VALUE);
properties.put(StreamsConfig.TASK_ASSIGNOR_CLASS_CONFIG, TestTaskAssignor.class.getName());
properties.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol);
final Properties config = StreamsTestUtils.getStreamsConfig(
applicationId,
@ -1003,9 +1027,9 @@ public class EosIntegrationTest {
// add second thread, to trigger rebalance
// expect idle task to get revoked -- this should not trigger a TX commit
streams.addStreamThread();
waitForCondition(DID_REVOKE_IDLE_TASK::get, "Idle Task was not revoked as expected.");
if (groupProtocol.equals("classic")) {
waitForCondition(DID_REVOKE_IDLE_TASK::get, "Idle Task was not revoked as expected.");
}
// best-effort sanity check (might pass and not detect issue in slow environments)
try {
readResult(SINGLE_PARTITION_OUTPUT_TOPIC, 1, "consumer", 10_000L);
@ -1104,6 +1128,7 @@ public class EosIntegrationTest {
final boolean withState,
final String appDir,
final int numberOfStreamsThreads,
final String groupProtocol,
final boolean processingThreadsEnabled) {
commitRequested = new AtomicInteger(0);
errorInjected = new AtomicBoolean(false);
@ -1212,6 +1237,7 @@ public class EosIntegrationTest {
properties.put(StreamsConfig.STATE_DIR_CONFIG, stateTmpDir + appDir);
properties.put(StreamsConfig.APPLICATION_SERVER_CONFIG, dummyHostName + ":2142");
properties.put(InternalConfig.PROCESSING_THREADS_ENABLED, processingThreadsEnabled);
properties.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol);
final Properties config = StreamsTestUtils.getStreamsConfig(
applicationId,

View File

@ -28,8 +28,8 @@ import org.apache.kafka.common.serialization.StringSerializer;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig;
import org.apache.kafka.streams.CloseOptions;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.KafkaStreams.CloseOptions;
import org.apache.kafka.streams.KeyValue;
import org.apache.kafka.streams.StreamsBuilder;
import org.apache.kafka.streams.StreamsConfig;
@ -159,7 +159,7 @@ public class KafkaStreamsCloseOptionsIntegrationTest {
IntegrationTestUtils.startApplicationAndWaitUntilRunning(streams);
IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived(resultConsumerConfig, OUTPUT_TOPIC, 10);
streams.close(new CloseOptions().leaveGroup(true).timeout(Duration.ofSeconds(30)));
streams.close(CloseOptions.groupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP).withTimeout(Duration.ofSeconds(30)));
waitForEmptyConsumerGroup(adminClient, streamsConfig.getProperty(StreamsConfig.APPLICATION_ID_CONFIG), 0);
}

View File

@ -0,0 +1,98 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams;
import java.time.Duration;
import java.util.Objects;
import java.util.Optional;
public class CloseOptions {
/**
* Enum to specify the group membership operation upon closing the Kafka Streams application.
*
* <ul>
* <li><b>{@code LEAVE_GROUP}</b>: means the consumer leave the group.</li>
* <li><b>{@code REMAIN_IN_GROUP}</b>: means the consumer will remain in the group.</li>
* </ul>
*/
public enum GroupMembershipOperation {
LEAVE_GROUP,
REMAIN_IN_GROUP
}
/**
* Specifies the group membership operation upon shutdown.
* By default, {@code GroupMembershipOperation.REMAIN_IN_GROUP} will be applied, which follows the KafkaStreams default behavior.
*/
protected GroupMembershipOperation operation = GroupMembershipOperation.REMAIN_IN_GROUP;
/**
* Specifies the maximum amount of time to wait for the close process to complete.
* This allows users to define a custom timeout for gracefully stopping the KafkaStreams.
*/
protected Optional<Duration> timeout = Optional.of(Duration.ofMillis(Long.MAX_VALUE));
private CloseOptions() {
}
protected CloseOptions(final CloseOptions closeOptions) {
this.operation = closeOptions.operation;
this.timeout = closeOptions.timeout;
}
/**
* Static method to create a {@code CloseOptions} with a custom timeout.
*
* @param timeout the maximum time to wait for the KafkaStreams to close.
* @return a new {@code CloseOptions} instance with the specified timeout.
*/
public static CloseOptions timeout(final Duration timeout) {
return new CloseOptions().withTimeout(timeout);
}
/**
* Static method to create a {@code CloseOptions} with a specified group membership operation.
*
* @param operation the group membership operation to apply. Must be one of {@code LEAVE_GROUP}, {@code REMAIN_IN_GROUP}.
* @return a new {@code CloseOptions} instance with the specified group membership operation.
*/
public static CloseOptions groupMembershipOperation(final GroupMembershipOperation operation) {
return new CloseOptions().withGroupMembershipOperation(operation);
}
/**
* Fluent method to set the timeout for the close process.
*
* @param timeout the maximum time to wait for the KafkaStreams to close. If {@code null}, the default timeout will be used.
* @return this {@code CloseOptions} instance.
*/
public CloseOptions withTimeout(final Duration timeout) {
this.timeout = Optional.ofNullable(timeout);
return this;
}
/**
* Fluent method to set the group membership operation upon shutdown.
*
* @param operation the group membership operation to apply. Must be one of {@code LEAVE_GROUP}, {@code REMAIN_IN_GROUP}.
* @return this {@code CloseOptions} instance.
*/
public CloseOptions withGroupMembershipOperation(final GroupMembershipOperation operation) {
this.operation = Objects.requireNonNull(operation, "operation should not be null");
return this;
}
}

View File

@ -49,6 +49,7 @@ import org.apache.kafka.streams.errors.StreamsStoppedException;
import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler;
import org.apache.kafka.streams.errors.UnknownStateStoreException;
import org.apache.kafka.streams.internals.ClientInstanceIdsImpl;
import org.apache.kafka.streams.internals.CloseOptionsInternal;
import org.apache.kafka.streams.internals.metrics.ClientMetrics;
import org.apache.kafka.streams.internals.metrics.StreamsClientMetricsDelegatingReporter;
import org.apache.kafka.streams.processor.StandbyUpdateListener;
@ -488,7 +489,7 @@ public class KafkaStreams implements AutoCloseable {
closeToError();
}
final StreamThread deadThread = (StreamThread) Thread.currentThread();
deadThread.shutdown(false);
deadThread.shutdown(org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP);
addStreamThread();
if (throwable instanceof RuntimeException) {
throw (RuntimeException) throwable;
@ -765,7 +766,7 @@ public class KafkaStreams implements AutoCloseable {
@Override
public void onUpdateStart(final TopicPartition topicPartition,
final String storeName,
final String storeName,
final long startingOffset) {
if (userStandbyListener != null) {
try {
@ -1136,7 +1137,7 @@ public class KafkaStreams implements AutoCloseable {
return Optional.of(streamThread.getName());
} else {
log.warn("Terminating the new thread because the Kafka Streams client is in state {}", state);
streamThread.shutdown(true);
streamThread.shutdown(org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
threads.remove(streamThread);
final long cacheSizePerThread = cacheSizePerThread(numLiveStreamThreads());
log.info("Resizing thread cache due to terminating added thread, new cache size per thread is {}", cacheSizePerThread);
@ -1200,7 +1201,7 @@ public class KafkaStreams implements AutoCloseable {
final boolean callingThreadIsNotCurrentStreamThread = !streamThread.getName().equals(Thread.currentThread().getName());
if (streamThread.isThreadAlive() && (callingThreadIsNotCurrentStreamThread || numLiveStreamThreads() == 1)) {
log.info("Removing StreamThread {}", streamThread.getName());
streamThread.shutdown(true);
streamThread.shutdown(org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
if (callingThreadIsNotCurrentStreamThread) {
final long remainingTimeMs = timeoutMs - (time.milliseconds() - startMs);
if (remainingTimeMs <= 0 || !streamThread.waitOnThreadState(StreamThread.State.DEAD, remainingTimeMs)) {
@ -1418,15 +1419,18 @@ public class KafkaStreams implements AutoCloseable {
/**
* Class that handles options passed in case of {@code KafkaStreams} instance scale down
*/
@Deprecated(since = "4.2")
public static class CloseOptions {
private Duration timeout = Duration.ofMillis(Long.MAX_VALUE);
private boolean leaveGroup = false;
@Deprecated(since = "4.2")
public CloseOptions timeout(final Duration timeout) {
this.timeout = timeout;
return this;
}
@Deprecated(since = "4.2")
public CloseOptions leaveGroup(final boolean leaveGroup) {
this.leaveGroup = leaveGroup;
return this;
@ -1438,10 +1442,14 @@ public class KafkaStreams implements AutoCloseable {
* This will block until all threads have stopped.
*/
public void close() {
close(Optional.empty(), false);
close(Optional.empty(), org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP);
}
private Thread shutdownHelper(final boolean error, final long timeoutMs, final boolean leaveGroup) {
private Thread shutdownHelper(
final boolean error,
final long timeoutMs,
final org.apache.kafka.streams.CloseOptions.GroupMembershipOperation operation
) {
stateDirCleaner.shutdownNow();
if (rocksDBMetricsRecordingService != null) {
rocksDBMetricsRecordingService.shutdownNow();
@ -1453,7 +1461,9 @@ public class KafkaStreams implements AutoCloseable {
return new Thread(() -> {
// notify all the threads to stop; avoid deadlocks by stopping any
// further state reports from the thread since we're shutting down
int numStreamThreads = processStreamThread(streamThread -> streamThread.shutdown(leaveGroup));
int numStreamThreads = processStreamThread(
streamThread -> streamThread.shutdown(operation)
);
log.info("Shutting down {} stream threads", numStreamThreads);
@ -1513,7 +1523,7 @@ public class KafkaStreams implements AutoCloseable {
}, clientId + "-CloseThread");
}
private boolean close(final Optional<Long> timeout, final boolean leaveGroup) {
private boolean close(final Optional<Long> timeout, final org.apache.kafka.streams.CloseOptions.GroupMembershipOperation operation) {
final long timeoutMs;
if (timeout.isPresent()) {
timeoutMs = timeout.get();
@ -1544,7 +1554,7 @@ public class KafkaStreams implements AutoCloseable {
+ "PENDING_SHUTDOWN, PENDING_ERROR, ERROR, or NOT_RUNNING");
}
final Thread shutdownThread = shutdownHelper(false, timeoutMs, leaveGroup);
final Thread shutdownThread = shutdownHelper(false, timeoutMs, operation);
shutdownThread.setDaemon(true);
shutdownThread.start();
@ -1562,7 +1572,7 @@ public class KafkaStreams implements AutoCloseable {
if (!setState(State.PENDING_ERROR)) {
log.info("Skipping shutdown since we are already in {}", state());
} else {
final Thread shutdownThread = shutdownHelper(true, -1, false);
final Thread shutdownThread = shutdownHelper(true, -1, org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP);
shutdownThread.setDaemon(true);
shutdownThread.start();
@ -1588,12 +1598,13 @@ public class KafkaStreams implements AutoCloseable {
throw new IllegalArgumentException("Timeout can't be negative.");
}
return close(Optional.of(timeoutMs), false);
return close(Optional.of(timeoutMs), org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP);
}
/**
* Shutdown this {@code KafkaStreams} by signaling all the threads to stop, and then wait up to the timeout for the
* threads to join.
* This method is deprecated and replaced by {@link #close(org.apache.kafka.streams.CloseOptions)}.
* @param options contains timeout to specify how long to wait for the threads to shut down, and a flag leaveGroup to
* trigger consumer leave call
* @return {@code true} if all threads were successfully stopped&mdash;{@code false} if the timeout was reached
@ -1601,15 +1612,36 @@ public class KafkaStreams implements AutoCloseable {
* Note that this method must not be called in the {@link StateListener#onChange(KafkaStreams.State, KafkaStreams.State)} callback of {@link StateListener}.
* @throws IllegalArgumentException if {@code timeout} can't be represented as {@code long milliseconds}
*/
@Deprecated(since = "4.2")
public synchronized boolean close(final CloseOptions options) throws IllegalArgumentException {
final org.apache.kafka.streams.CloseOptions closeOptions = org.apache.kafka.streams.CloseOptions.timeout(options.timeout)
.withGroupMembershipOperation(options.leaveGroup ?
org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.LEAVE_GROUP :
org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP);
return close(closeOptions);
}
/**
* Shutdown this {@code KafkaStreams} by signaling all the threads to stop, and then wait up to the timeout for the
* threads to join.
* @param options contains timeout to specify how long to wait for the threads to shut down,
* and a {@link org.apache.kafka.streams.CloseOptions.GroupMembershipOperation}
* to trigger consumer leave call or remain in the group
* @return {@code true} if all threads were successfully stopped&mdash;{@code false} if the timeout was reached
* before all threads stopped
* Note that this method must not be called in the {@link StateListener#onChange(KafkaStreams.State, KafkaStreams.State)} callback of {@link StateListener}.
* @throws IllegalArgumentException if {@code timeout} can't be represented as {@code long milliseconds}
*/
public synchronized boolean close(final org.apache.kafka.streams.CloseOptions options) throws IllegalArgumentException {
Objects.requireNonNull(options, "options cannot be null");
final String msgPrefix = prepareMillisCheckFailMsgPrefix(options.timeout, "timeout");
final long timeoutMs = validateMillisecondDuration(options.timeout, msgPrefix);
final CloseOptionsInternal optionsInternal = new CloseOptionsInternal(options);
final String msgPrefix = prepareMillisCheckFailMsgPrefix(optionsInternal.timeout(), "timeout");
final long timeoutMs = validateMillisecondDuration(optionsInternal.timeout().get(), msgPrefix);
if (timeoutMs < 0) {
throw new IllegalArgumentException("Timeout can't be negative.");
}
return close(Optional.of(timeoutMs), options.leaveGroup);
return close(Optional.of(timeoutMs), optionsInternal.operation());
}
/**

View File

@ -0,0 +1,37 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.internals;
import org.apache.kafka.streams.CloseOptions;
import java.time.Duration;
import java.util.Optional;
public class CloseOptionsInternal extends CloseOptions {
public CloseOptionsInternal(final CloseOptions options) {
super(options);
}
public GroupMembershipOperation operation() {
return operation;
}
public Optional<Duration> timeout() {
return timeout;
}
}

View File

@ -91,9 +91,9 @@ import java.util.Optional;
import java.util.Queue;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.BiConsumer;
import java.util.stream.Collectors;
@ -367,7 +367,8 @@ public class StreamThread extends Thread implements ProcessingThread {
// These are used to signal from outside the stream thread, but the variables themselves are internal to the thread
private final AtomicLong cacheResizeSize = new AtomicLong(-1L);
private final AtomicBoolean leaveGroupRequested = new AtomicBoolean(false);
private final AtomicReference<org.apache.kafka.streams.CloseOptions.GroupMembershipOperation> leaveGroupRequested =
new AtomicReference<>(org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP);
private final AtomicLong lastShutdownWarningTimestamp = new AtomicLong(0L);
private final boolean eosEnabled;
private final boolean stateUpdaterEnabled;
@ -898,7 +899,7 @@ public class StreamThread extends Thread implements ProcessingThread {
cleanRun = runLoop();
} catch (final Throwable e) {
failedStreamThreadSensor.record();
leaveGroupRequested.set(true);
leaveGroupRequested.set(org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
streamsUncaughtExceptionHandler.accept(e, false);
// Note: the above call currently rethrows the exception, so nothing below this line will be executed
} finally {
@ -1547,7 +1548,7 @@ public class StreamThread extends Thread implements ProcessingThread {
if (streamsRebalanceData.isPresent()) {
boolean hasMissingSourceTopics = false;
String missingTopicsDetail = null;
for (final StreamsGroupHeartbeatResponseData.Status status : streamsRebalanceData.get().statuses()) {
if (status.statusCode() == StreamsGroupHeartbeatResponse.Status.SHUTDOWN_APPLICATION.code()) {
shutdownErrorHook.run();
@ -1560,7 +1561,7 @@ public class StreamThread extends Thread implements ProcessingThread {
throw new TopologyException(errorMsg);
}
}
if (hasMissingSourceTopics) {
handleMissingSourceTopicsWithTimeout(missingTopicsDetail);
} else {
@ -1589,25 +1590,25 @@ public class StreamThread extends Thread implements ProcessingThread {
// Start timeout tracking on first encounter with missing topics
if (topicsReadyTimer == null) {
topicsReadyTimer = time.timer(maxPollTimeMs);
log.info("Missing source topics detected: {}. Will wait up to {}ms before failing.",
log.info("Missing source topics detected: {}. Will wait up to {}ms before failing.",
missingTopicsDetail, maxPollTimeMs);
} else {
topicsReadyTimer.update();
}
if (topicsReadyTimer.isExpired()) {
final long elapsedTime = topicsReadyTimer.elapsedMs();
final String errorMsg = String.format("Missing source topics: %s. Timeout exceeded after %dms.",
final String errorMsg = String.format("Missing source topics: %s. Timeout exceeded after %dms.",
missingTopicsDetail, elapsedTime);
log.error(errorMsg);
throw new MissingSourceTopicException(errorMsg);
} else {
log.debug("Missing source topics: {}. Elapsed time: {}ms, timeout in: {}ms",
log.debug("Missing source topics: {}. Elapsed time: {}ms, timeout in: {}ms",
missingTopicsDetail, topicsReadyTimer.elapsedMs(), topicsReadyTimer.remainingMs());
}
}
static Map<TopicPartition, PartitionInfo> getTopicPartitionInfo(final Map<HostInfo, Set<TopicPartition>> partitionsByHost) {
final Map<TopicPartition, PartitionInfo> topicToPartitionInfo = new HashMap<>();
@ -1879,12 +1880,12 @@ public class StreamThread extends Thread implements ProcessingThread {
* Note that there is nothing to prevent this function from being called multiple times
* (e.g., in testing), hence the state is set only the first time
*
* @param leaveGroup this flag will control whether the consumer will leave the group on close or not
* @param operation the group membership operation to apply on shutdown. Must be one of LEAVE_GROUP or REMAIN_IN_GROUP.
*/
public void shutdown(final boolean leaveGroup) {
public void shutdown(final org.apache.kafka.streams.CloseOptions.GroupMembershipOperation operation) {
log.info("Informed to shut down");
final State oldState = setState(State.PENDING_SHUTDOWN);
leaveGroupRequested.set(leaveGroup);
leaveGroupRequested.set(operation);
if (oldState == State.CREATED) {
// The thread may not have been started. Take responsibility for shutting down
completeShutdown(true);
@ -1917,7 +1918,8 @@ public class StreamThread extends Thread implements ProcessingThread {
log.error("Failed to close changelog reader due to the following error:", e);
}
try {
final GroupMembershipOperation membershipOperation = leaveGroupRequested.get() ? LEAVE_GROUP : REMAIN_IN_GROUP;
final GroupMembershipOperation membershipOperation =
leaveGroupRequested.get() == org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.LEAVE_GROUP ? LEAVE_GROUP : REMAIN_IN_GROUP;
mainConsumer.close(CloseOptions.groupMembershipOperation(membershipOperation));
} catch (final Throwable e) {
log.error("Failed to close consumer due to the following error:", e);

View File

@ -19,7 +19,6 @@ package org.apache.kafka.streams;
import org.apache.kafka.clients.admin.Admin;
import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo;
import org.apache.kafka.clients.admin.MockAdminClient;
import org.apache.kafka.clients.consumer.CloseOptions;
import org.apache.kafka.clients.producer.MockProducer;
import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.KafkaFuture;
@ -310,8 +309,12 @@ public class KafkaStreamsTest {
private void prepareConsumer(final StreamThread thread, final AtomicReference<StreamThread.State> state) {
doAnswer(invocation -> {
supplier.consumer.close(CloseOptions.groupMembershipOperation(CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP));
supplier.restoreConsumer.close(CloseOptions.groupMembershipOperation(CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP));
supplier.consumer.close(
org.apache.kafka.clients.consumer.CloseOptions.groupMembershipOperation(org.apache.kafka.clients.consumer.CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP)
);
supplier.restoreConsumer.close(
org.apache.kafka.clients.consumer.CloseOptions.groupMembershipOperation(org.apache.kafka.clients.consumer.CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP)
);
for (final MockProducer<byte[], byte[]> producer : supplier.producers) {
producer.close();
}
@ -320,7 +323,7 @@ public class KafkaStreamsTest {
threadStateListenerCapture.getValue().onChange(thread, StreamThread.State.PENDING_SHUTDOWN, StreamThread.State.RUNNING);
threadStateListenerCapture.getValue().onChange(thread, StreamThread.State.DEAD, StreamThread.State.PENDING_SHUTDOWN);
return null;
}).when(thread).shutdown(false);
}).when(thread).shutdown(CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP);
}
private void prepareThreadLock(final StreamThread thread) {
@ -571,7 +574,7 @@ public class KafkaStreamsTest {
for (int i = 0; i < NUM_THREADS; i++) {
final StreamThread tmpThread = streams.threads.get(i);
tmpThread.shutdown(false);
tmpThread.shutdown(CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP);
waitForCondition(() -> tmpThread.state() == StreamThread.State.DEAD,
"Thread never stopped.");
streams.threads.get(i).join();
@ -790,7 +793,7 @@ public class KafkaStreamsTest {
prepareThreadLock(streamThreadTwo);
try (final KafkaStreams streams = new KafkaStreams(getBuilderWithSource().build(), props, supplier, time)) {
streams.start();
streamThreadOne.shutdown(true);
streamThreadOne.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
final Set<ThreadMetadata> threads = streams.metadataForLocalThreads();
assertThat(threads.size(), equalTo(1));
assertThat(threads, hasItem(streamThreadTwo.threadMetadata()));
@ -1016,9 +1019,8 @@ public class KafkaStreamsTest {
() -> streams.state() == KafkaStreams.State.RUNNING,
"Streams never started.");
final KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
closeOptions.timeout(Duration.ZERO);
closeOptions.leaveGroup(true);
final CloseOptions closeOptions = CloseOptions.timeout(Duration.ZERO)
.withGroupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
streams.close(closeOptions);
assertThat(streams.state() == State.PENDING_SHUTDOWN, equalTo(true));
@ -1041,8 +1043,7 @@ public class KafkaStreamsTest {
() -> streams.state() == KafkaStreams.State.RUNNING,
"Streams never started.");
final KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
closeOptions.timeout(Duration.ZERO);
final CloseOptions closeOptions = CloseOptions.timeout(Duration.ZERO);
streams.close(closeOptions);
assertThat(streams.state() == State.PENDING_SHUTDOWN, equalTo(true));
@ -1229,8 +1230,7 @@ public class KafkaStreamsTest {
prepareStreamThread(streamThreadTwo, 2);
prepareTerminableThread(streamThreadOne);
final KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
closeOptions.timeout(Duration.ofMillis(10L));
final CloseOptions closeOptions = CloseOptions.timeout(Duration.ofMillis(10L));
try (final KafkaStreams streams = new KafkaStreamsWithTerminableThread(getBuilderWithSource().build(), props, supplier)) {
assertFalse(streams.close(closeOptions));
}
@ -1243,8 +1243,7 @@ public class KafkaStreamsTest {
prepareStreamThread(streamThreadTwo, 2);
prepareTerminableThread(streamThreadOne);
final KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
closeOptions.timeout(Duration.ofMillis(-1L));
final CloseOptions closeOptions = CloseOptions.timeout(Duration.ofMillis(-1L));
try (final KafkaStreams streams = new KafkaStreamsWithTerminableThread(getBuilderWithSource().build(), props, supplier, time)) {
assertThrows(IllegalArgumentException.class, () -> streams.close(closeOptions));
}
@ -1257,8 +1256,7 @@ public class KafkaStreamsTest {
prepareStreamThread(streamThreadTwo, 2);
prepareTerminableThread(streamThreadOne);
final KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
closeOptions.timeout(Duration.ZERO);
final CloseOptions closeOptions = CloseOptions.timeout(Duration.ZERO);
try (final KafkaStreams streams = new KafkaStreamsWithTerminableThread(getBuilderWithSource().build(), props, supplier)) {
assertFalse(streams.close(closeOptions));
}
@ -1275,9 +1273,8 @@ public class KafkaStreamsTest {
when(mockClientSupplier.getAdmin(any())).thenReturn(adminClient);
final KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
closeOptions.timeout(Duration.ofMillis(10L));
closeOptions.leaveGroup(true);
final CloseOptions closeOptions = CloseOptions.timeout(Duration.ofMillis(10L))
.withGroupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
try (final KafkaStreams streams = new KafkaStreamsWithTerminableThread(getBuilderWithSource().build(), props, mockClientSupplier)) {
assertFalse(streams.close(closeOptions));
}
@ -1293,9 +1290,8 @@ public class KafkaStreamsTest {
final MockClientSupplier mockClientSupplier = spy(MockClientSupplier.class);
when(mockClientSupplier.getAdmin(any())).thenReturn(adminClient);
final KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
closeOptions.timeout(Duration.ofMillis(-1L));
closeOptions.leaveGroup(true);
final CloseOptions closeOptions = CloseOptions.timeout(Duration.ofMillis(-1L))
.withGroupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
try (final KafkaStreams streams = new KafkaStreamsWithTerminableThread(getBuilderWithSource().build(), props, mockClientSupplier, time)) {
assertThrows(IllegalArgumentException.class, () -> streams.close(closeOptions));
}
@ -1312,9 +1308,8 @@ public class KafkaStreamsTest {
when(mockClientSupplier.getAdmin(any())).thenReturn(adminClient);
final KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
closeOptions.timeout(Duration.ZERO);
closeOptions.leaveGroup(true);
final CloseOptions closeOptions = CloseOptions.timeout(Duration.ZERO)
.withGroupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
try (final KafkaStreams streams = new KafkaStreamsWithTerminableThread(getBuilderWithSource().build(), props, mockClientSupplier)) {
assertFalse(streams.close(closeOptions));
}
@ -1720,7 +1715,7 @@ public class KafkaStreamsTest {
producerFuture.complete(producerInstanceId);
final Uuid adminInstanceId = Uuid.randomUuid();
adminClient.setClientInstanceId(adminInstanceId);
final Map<String, KafkaFuture<Uuid>> expectedClientIds = Map.of("main-consumer", consumerFuture, "some-thread-producer", producerFuture);
when(streamThreadOne.clientInstanceIds(any())).thenReturn(expectedClientIds);

View File

@ -60,6 +60,7 @@ import org.apache.kafka.common.utils.LogCaptureAppender;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.streams.CloseOptions;
import org.apache.kafka.streams.GroupProtocol;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.StreamsConfig.InternalConfig;
@ -247,7 +248,7 @@ public class StreamThreadTest {
if (thread.state() != State.CREATED) {
thread.taskManager().shutdown(false);
}
thread.shutdown(true);
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
thread = null;
}
final Set<Thread> t = Collections.unmodifiableSet(Thread.getAllStackTraces().keySet());
@ -409,12 +410,12 @@ public class StreamThreadTest {
assertEquals(4, stateListener.numChanges);
assertEquals(StreamThread.State.PARTITIONS_ASSIGNED, stateListener.oldState);
thread.shutdown(true);
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
assertSame(StreamThread.State.PENDING_SHUTDOWN, thread.state());
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldChangeStateAtStartClose(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
thread = createStreamThread(CLIENT_ID, new MockTime(1), stateUpdaterEnabled, processingThreadsEnabled);
@ -427,18 +428,18 @@ public class StreamThreadTest {
10 * 1000,
"Thread never started.");
thread.shutdown(true);
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
TestUtils.waitForCondition(
() -> thread.state() == StreamThread.State.DEAD,
10 * 1000,
"Thread never shut down.");
thread.shutdown(true);
assertEquals(thread.state(), StreamThread.State.DEAD);
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
assertEquals(State.DEAD, thread.state());
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldCreateMetricsAtStartup(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
thread = createStreamThread(CLIENT_ID, new MockTime(1), stateUpdaterEnabled, processingThreadsEnabled);
final String defaultGroupName = "stream-thread-metrics";
@ -538,7 +539,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotCommitBeforeTheCommitInterval(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final long commitInterval = 1000L;
final Properties props = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
@ -565,7 +566,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotPurgeBeforeThePurgeInterval(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final long commitInterval = 1000L;
final long purgeInterval = 2000L;
@ -593,7 +594,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldAlsoPurgeWhenNothingGetsCommitted(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final long purgeInterval = 1000L;
final Properties props = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
@ -658,7 +659,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotProcessWhenPartitionRevoked(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
assumeFalse(processingThreadsEnabled);
@ -682,7 +683,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldProcessWhenRunning(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
assumeFalse(processingThreadsEnabled);
final Properties props = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
@ -707,7 +708,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldProcessWhenPartitionAssigned(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
assumeTrue(stateUpdaterEnabled);
assumeFalse(processingThreadsEnabled);
@ -732,7 +733,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldProcessWhenStarting(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
assumeTrue(stateUpdaterEnabled);
assumeFalse(processingThreadsEnabled);
@ -757,7 +758,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldEnforceRebalanceWhenScheduledAndNotCurrentlyRebalancing(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws InterruptedException {
final Time mockTime = new MockTime(1);
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
@ -812,7 +813,7 @@ public class StreamThreadTest {
10 * 1000,
"Thread never started.");
thread.shutdown(true);
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
TestUtils.waitForCondition(
() -> thread.state() == StreamThread.State.DEAD,
10 * 1000,
@ -822,7 +823,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotEnforceRebalanceWhenCurrentlyRebalancing(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws InterruptedException {
final Time mockTime = new MockTime(1);
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
@ -880,7 +881,7 @@ public class StreamThreadTest {
() -> { }
);
thread.shutdown(true);
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
// Validate that the scheduled rebalance wasn't reset then set to MAX_VALUE so we
// don't trigger one before we can shut down, since the rebalance must be ended
@ -918,7 +919,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldRespectNumIterationsInMainLoopWithoutProcessingThreads(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
// With processing threads, there is no guarantee how many iterations will be performed
assumeFalse(processingThreadsEnabled);
@ -1047,7 +1048,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotCauseExceptionIfNothingCommitted(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final long commitInterval = 1000L;
final Properties props = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
@ -1076,7 +1077,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldCommitAfterCommitInterval(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final long commitInterval = 100L;
final long commitLatency = 10L;
@ -1137,7 +1138,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldPurgeAfterPurgeInterval(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final long commitInterval = 100L;
final long purgeInterval = 200L;
@ -1170,7 +1171,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldRecordCommitLatency(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class);
when(consumer.groupMetadata()).thenReturn(consumerGroupMetadata);
@ -1279,7 +1280,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldInjectSharedProducerForAllTasksUsingClientSupplierOnCreateIfEosDisabled(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
internalTopologyBuilder.addSource(null, "source1", null, null, null, topic1);
internalStreamsBuilder.buildAndOptimizeTopology();
@ -1319,7 +1320,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldInjectProducerPerThreadUsingClientSupplierOnCreateIfEosV2Enabled(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
internalTopologyBuilder.addSource(null, "source1", null, null, null, topic1);
@ -1357,7 +1358,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldOnlyCompleteShutdownAfterRebalanceNotInProgress(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws InterruptedException {
// The state updater is disabled for this test because this test relies on the fact the mainConsumer.resume()
// is not called. This is not true when the state updater is enabled which leads to
@ -1390,7 +1391,7 @@ public class StreamThreadTest {
10 * 1000,
"Thread never started.");
thread.shutdown(true);
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
// even if thread is no longer running, it should still be polling
// as long as the rebalance is still ongoing
@ -1411,7 +1412,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldShutdownTaskManagerOnClose(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class);
when(consumer.groupMetadata()).thenReturn(consumerGroupMetadata);
@ -1426,7 +1427,7 @@ public class StreamThreadTest {
thread.setStateListener(
(t, newState, oldState) -> {
if (oldState == StreamThread.State.CREATED && newState == StreamThread.State.STARTING) {
thread.shutdown(true);
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
}
});
thread.run();
@ -1435,7 +1436,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotReturnDataAfterTaskMigrated(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final TaskManager taskManager = mock(TaskManager.class);
final InternalTopologyBuilder internalTopologyBuilder = mock(InternalTopologyBuilder.class);
@ -1512,7 +1513,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldShutdownTaskManagerOnCloseWithoutStart(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class);
when(consumer.groupMetadata()).thenReturn(consumerGroupMetadata);
@ -1524,13 +1525,13 @@ public class StreamThreadTest {
topologyMetadata.buildAndRewriteTopology();
thread = buildStreamThread(consumer, taskManager, config, topologyMetadata)
.updateThreadMetadata(adminClientId(CLIENT_ID));
thread.shutdown(true);
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
verify(taskManager).shutdown(true);
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldOnlyShutdownOnce(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class);
when(consumer.groupMetadata()).thenReturn(consumerGroupMetadata);
@ -1542,7 +1543,7 @@ public class StreamThreadTest {
topologyMetadata.buildAndRewriteTopology();
thread = buildStreamThread(consumer, taskManager, config, topologyMetadata)
.updateThreadMetadata(adminClientId(CLIENT_ID));
thread.shutdown(true);
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
// Execute the run method. Verification of the mock will check that shutdown was only done once
thread.run();
@ -1550,7 +1551,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotThrowWhenStandbyTasksAssignedAndNoStateStoresForTopology(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
internalTopologyBuilder.addSource(null, "name", null, null, null, "topic");
internalTopologyBuilder.addSink("out", "output", null, null, null, "name");
@ -1572,7 +1573,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotCloseTaskAndRemoveFromTaskManagerIfProducerWasFencedWhileProcessing(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
internalTopologyBuilder.addSource(null, "source", null, null, null, topic1);
internalTopologyBuilder.addSink("sink", "dummyTopic", null, null, null, "source");
@ -1688,18 +1689,18 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotCloseTaskAndRemoveFromTaskManagerIfProducerGotFencedInCommitTransactionWhenSuspendingTasks(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
testThrowingDurringCommitTransactionException(new ProducerFencedException("Producer is fenced"), stateUpdaterEnabled, processingThreadsEnabled);
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotCloseTaskAndRemoveFromTaskManagerIfInvalidPidMappingOccurredInCommitTransactionWhenSuspendingTasks(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
testThrowingDurringCommitTransactionException(new InvalidPidMappingException("PidMapping is invalid"), stateUpdaterEnabled, processingThreadsEnabled);
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldReinitializeRevivedTasksInAnyState(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
thread = createStreamThread(CLIENT_ID, config, new MockTime(1));
@ -1873,19 +1874,19 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotCloseTaskAndRemoveFromTaskManagerIfProducerGotFencedInCommitTransactionWhenCommitting(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
testNotCloseTaskAndRemoveFromTaskManagerInCommitTransactionWhenCommitting(new ProducerFencedException("Producer is fenced"), stateUpdaterEnabled, processingThreadsEnabled);
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotCloseTaskAndRemoveFromTaskManagerIfPidMappingIsInvalidInCommitTransactionWhenCommitting(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
testNotCloseTaskAndRemoveFromTaskManagerInCommitTransactionWhenCommitting(new InvalidPidMappingException("PID Mapping is invalid"), stateUpdaterEnabled, processingThreadsEnabled);
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotCloseTaskProducerWhenSuspending(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
final StreamsConfig config = new StreamsConfig(configProps(true, stateUpdaterEnabled, processingThreadsEnabled));
thread = createStreamThread(CLIENT_ID, config);
@ -1933,7 +1934,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldReturnActiveTaskMetadataWhileRunningState(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
internalTopologyBuilder.addSource(null, "source", null, null, null, topic1);
@ -2011,7 +2012,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldReturnStandbyTaskMetadataWhileRunningState(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
internalStreamsBuilder.stream(Collections.singleton(topic1), consumed)
@ -2059,7 +2060,7 @@ public class StreamThreadTest {
@SuppressWarnings("unchecked")
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldUpdateStandbyTask(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
// Updating standby tasks on the stream thread only happens when the state updater is disabled
assumeFalse(stateUpdaterEnabled);
@ -2183,7 +2184,7 @@ public class StreamThreadTest {
@SuppressWarnings("unchecked")
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotUpdateStandbyTaskWhenPaused(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
// Updating standby tasks on the stream thread only happens when the state updater is disabled
assumeFalse(stateUpdaterEnabled);
@ -2243,7 +2244,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldCreateStandbyTask(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
setupInternalTopologyWithoutState(config);
@ -2253,7 +2254,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotCreateStandbyTaskWithoutStateStores(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
setupInternalTopologyWithoutState(config);
@ -2262,7 +2263,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotCreateStandbyTaskIfStateStoresHaveLoggingDisabled(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
setupInternalTopologyWithoutState(config);
@ -2275,7 +2276,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
@SuppressWarnings("deprecation")
public void shouldPunctuateActiveTask(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
assumeFalse(processingThreadsEnabled);
@ -2426,7 +2427,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldAlwaysUpdateTasksMetadataAfterChangingState(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
thread = createStreamThread(CLIENT_ID, config);
@ -2442,7 +2443,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldRecoverFromInvalidOffsetExceptionOnRestoreAndFinishRestore(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
internalStreamsBuilder.stream(Collections.singleton("topic"), consumed)
.groupByKey()
@ -2632,7 +2633,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldThrowTaskMigratedExceptionHandlingTaskLost(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
final Set<TopicPartition> assignedPartitions = Collections.singleton(t1p1);
@ -2660,7 +2661,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldThrowTaskMigratedExceptionHandlingRevocation(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
final Set<TopicPartition> assignedPartitions = Collections.singleton(t1p1);
@ -2688,7 +2689,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
@SuppressWarnings("unchecked")
public void shouldCatchHandleCorruptionOnTaskCorruptedExceptionPath(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
@ -2749,7 +2750,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
@SuppressWarnings("unchecked")
public void shouldCatchTimeoutExceptionFromHandleCorruptionAndInvokeExceptionHandler(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
@ -2815,7 +2816,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
@SuppressWarnings("unchecked")
public void shouldCatchTaskMigratedExceptionOnOnTaskCorruptedExceptionPath(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
@ -2881,7 +2882,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
@SuppressWarnings("unchecked")
public void shouldEnforceRebalanceWhenTaskCorruptedExceptionIsThrownForAnActiveTask(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(true, stateUpdaterEnabled, processingThreadsEnabled));
@ -2946,7 +2947,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
@SuppressWarnings("unchecked")
public void shouldNotEnforceRebalanceWhenTaskCorruptedExceptionIsThrownForAnInactiveTask(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(true, stateUpdaterEnabled, processingThreadsEnabled));
@ -3009,7 +3010,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotCommitNonRunningNonRestoringTasks(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
final TaskManager taskManager = mock(TaskManager.class);
@ -3048,7 +3049,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldLogAndRecordSkippedRecordsForInvalidTimestamps(
final boolean stateUpdaterEnabled,
final boolean processingThreadsEnabled
@ -3155,7 +3156,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldTransmitTaskManagerMetrics(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
final ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class);
@ -3182,7 +3183,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldConstructAdminMetrics(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
final Node broker1 = new Node(0, "dummyHost-1", 1234);
@ -3239,13 +3240,13 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotRecordFailedStreamThread(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
runAndVerifyFailedStreamThreadRecording(false, stateUpdaterEnabled, processingThreadsEnabled);
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldRecordFailedStreamThread(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
runAndVerifyFailedStreamThreadRecording(true, stateUpdaterEnabled, processingThreadsEnabled);
}
@ -3308,7 +3309,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldCheckStateUpdater(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
assumeTrue(stateUpdaterEnabled);
final Properties streamsConfigProps = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
@ -3326,7 +3327,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldCheckStateUpdaterInBetweenProcessCalls(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
assumeTrue(stateUpdaterEnabled);
assumeFalse(processingThreadsEnabled);
@ -3344,7 +3345,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldUpdateLagsAfterPolling(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final Properties streamsConfigProps = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
thread = setUpThread(streamsConfigProps);
@ -3362,7 +3363,7 @@ public class StreamThreadTest {
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldResumePollingForPartitionsWithAvailableSpaceBeforePolling(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final Properties streamsConfigProps = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
thread = setUpThread(streamsConfigProps);
@ -3377,7 +3378,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldRespectPollTimeInPartitionsAssignedStateWithStateUpdater(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
assumeTrue(stateUpdaterEnabled);
final Properties streamsConfigProps = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
@ -3393,7 +3394,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotBlockWhenPollingInPartitionsAssignedStateWithoutStateUpdater(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
assumeFalse(stateUpdaterEnabled);
final Properties streamsConfigProps = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
@ -3407,13 +3408,13 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldGetMainAndRestoreConsumerInstanceId(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
getClientInstanceId(false, stateUpdaterEnabled, processingThreadsEnabled);
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldGetMainAndRestoreConsumerInstanceIdWithInternalTimeout(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
getClientInstanceId(true, stateUpdaterEnabled, processingThreadsEnabled);
}
@ -3460,7 +3461,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldReturnErrorIfMainConsumerInstanceIdNotInitialized(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
thread = createStreamThread("clientId", stateUpdaterEnabled, processingThreadsEnabled);
thread.setState(State.STARTING);
@ -3477,7 +3478,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldReturnErrorIfRestoreConsumerInstanceIdNotInitialized(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
thread = createStreamThread("clientId", stateUpdaterEnabled, processingThreadsEnabled);
thread.setState(State.STARTING);
@ -3494,7 +3495,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldReturnErrorIfProducerInstanceIdNotInitialized(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
thread = createStreamThread("clientId", stateUpdaterEnabled, processingThreadsEnabled);
thread.setState(State.STARTING);
@ -3511,7 +3512,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldReturnNullIfMainConsumerTelemetryDisabled(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
clientSupplier.consumer.disableTelemetry();
thread = createStreamThread("clientId", stateUpdaterEnabled, processingThreadsEnabled);
@ -3528,7 +3529,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldReturnNullIfRestoreConsumerTelemetryDisabled(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
clientSupplier.restoreConsumer.disableTelemetry();
@ -3546,7 +3547,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldReturnNullIfProducerTelemetryDisabled(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
final MockProducer<byte[], byte[]> producer = new MockProducer<>();
producer.disableTelemetry();
@ -3566,7 +3567,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldTimeOutOnMainConsumerInstanceId(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
clientSupplier.consumer.setClientInstanceId(Uuid.randomUuid());
clientSupplier.consumer.injectTimeoutException(-1);
@ -3591,7 +3592,7 @@ public class StreamThreadTest {
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldTimeOutOnRestoreConsumerInstanceId(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
clientSupplier.restoreConsumer.setClientInstanceId(Uuid.randomUuid());
clientSupplier.restoreConsumer.injectTimeoutException(-1);
@ -3616,7 +3617,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldTimeOutOnProducerInstanceId(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final MockProducer<byte[], byte[]> producer = new MockProducer<>();
producer.setClientInstanceId(Uuid.randomUuid());
@ -3964,13 +3965,13 @@ public class StreamThreadTest {
.setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_SOURCE_TOPICS.code())
.setStatusDetail("Missing source topics")
));
// First call should not throw exception (within timeout)
thread.runOnceWithoutProcessingThreads();
// Advance time beyond max.poll.interval.ms (default is 300000ms) to trigger timeout
mockTime.sleep(300001);
final MissingSourceTopicException exception = assertThrows(MissingSourceTopicException.class, () -> thread.runOnceWithoutProcessingThreads());
assertTrue(exception.getMessage().contains("Missing source topics"));
assertTrue(exception.getMessage().contains("Timeout exceeded"));
@ -4032,7 +4033,7 @@ public class StreamThreadTest {
));
// Should immediately throw TopologyException (no timeout like MISSING_SOURCE_TOPICS)
final TopologyException exception = assertThrows(TopologyException.class,
final TopologyException exception = assertThrows(TopologyException.class,
() -> thread.runOnceWithoutProcessingThreads());
assertTrue(exception.getMessage().contains("Topics are incorrectly partitioned"));
}
@ -4151,13 +4152,13 @@ public class StreamThreadTest {
.setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_SOURCE_TOPICS.code())
.setStatusDetail("Missing source topics")
));
// First call should not throw exception (within timeout)
thread.runOnceWithProcessingThreads();
// Advance time beyond max.poll.interval.ms (default is 300000ms) to trigger timeout
mockTime.sleep(300001);
final MissingSourceTopicException exception = assertThrows(MissingSourceTopicException.class, () -> thread.runOnceWithProcessingThreads());
assertTrue(exception.getMessage().contains("Missing source topics"));
assertTrue(exception.getMessage().contains("Timeout exceeded"));
@ -4219,35 +4220,35 @@ public class StreamThreadTest {
.setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_SOURCE_TOPICS.code())
.setStatusDetail("Missing source topics")
));
// First call should not throw exception (within timeout)
thread.runOnceWithoutProcessingThreads();
// Advance time but not beyond timeout
mockTime.sleep(150000); // Half of max.poll.interval.ms
// Should still not throw exception
thread.runOnceWithoutProcessingThreads();
// Clear the missing source topics (simulate recovery)
streamsRebalanceData.setStatuses(List.of());
// Should complete without exception (recovery successful)
assertDoesNotThrow(() -> thread.runOnceWithoutProcessingThreads());
// Set missing topics again - should reset the timeout
streamsRebalanceData.setStatuses(List.of(
new StreamsGroupHeartbeatResponseData.Status()
.setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_SOURCE_TOPICS.code())
.setStatusDetail("Different missing topics")
));
// Advance time by 250 seconds to test if timer was reset
// Total time from beginning: 150000 + 250000 = 400000ms (400s)
// If timer was NOT reset: elapsed time = 400s > 300s should throw
// If timer WAS reset: elapsed time = 250s < 300s should NOT throw
mockTime.sleep(250000); // Advance by 250 seconds
// Should not throw because timer was reset - only 250s elapsed from reset point
assertDoesNotThrow(() -> thread.runOnceWithoutProcessingThreads());
}
@ -4427,7 +4428,7 @@ public class StreamThreadTest {
null
);
}
private void runOnce(final boolean processingThreadsEnabled) {
if (processingThreadsEnabled) {
thread.runOnceWithProcessingThreads();

View File

@ -27,6 +27,7 @@ import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.serialization.StringSerializer;
import org.apache.kafka.common.utils.Exit;
import org.apache.kafka.streams.CloseOptions;
import org.apache.kafka.streams.GroupProtocol;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.KeyValueTimestamp;
@ -341,9 +342,8 @@ public class DeleteStreamsGroupOffsetTest {
private void stopKSApp(String appId, String topic, KafkaStreams streams, StreamsGroupCommand.StreamsGroupService service) throws InterruptedException {
if (streams != null) {
KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
closeOptions.timeout(Duration.ofSeconds(30));
closeOptions.leaveGroup(true);
CloseOptions closeOptions = CloseOptions.timeout(Duration.ofSeconds(30))
.withGroupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
streams.close(closeOptions);
streams.cleanUp();

View File

@ -31,6 +31,7 @@ import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.serialization.StringSerializer;
import org.apache.kafka.common.utils.Exit;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.streams.CloseOptions;
import org.apache.kafka.streams.GroupProtocol;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.KeyValueTimestamp;
@ -512,9 +513,8 @@ public class DeleteStreamsGroupTest {
private void stopKSApp(String appId, KafkaStreams streams, StreamsGroupCommand.StreamsGroupService service) throws InterruptedException {
if (streams != null) {
KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
closeOptions.timeout(Duration.ofSeconds(30));
closeOptions.leaveGroup(true);
CloseOptions closeOptions = CloseOptions.timeout(Duration.ofSeconds(30))
.withGroupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
streams.close(closeOptions);
streams.cleanUp();

View File

@ -23,7 +23,7 @@
// and not the version installed on the machine running the task.
// Read more about the wrapper here: https://docs.gradle.org/current/userguide/gradle_wrapper.html
wrapper {
gradleVersion = project.gradleVersion
gradleVersion = versions.gradle
}
// Custom task to inject support for downloading the gradle wrapper jar if it doesn't exist.
@ -35,14 +35,12 @@ task bootstrapWrapper() {
def wrapperBasePath = "\$APP_HOME/gradle/wrapper"
def wrapperJarPath = wrapperBasePath + "/gradle-wrapper.jar"
// Add a trailing zero to the version if needed.
def fullVersion = project.gradleVersion.count(".") == 1 ? "${project.gradleVersion}.0" : versions.gradle
// Leverages the wrapper jar checked into the gradle project on github because the jar isn't
// available elsewhere. Using raw.githubusercontent.com instead of github.com because
// github.com servers deprecated TLSv1/TLSv1.1 support some time ago, so older versions
// of curl (built against OpenSSL library that doesn't support TLSv1.2) would fail to
// fetch the jar.
def wrapperBaseUrl = "https://raw.githubusercontent.com/gradle/gradle/v$fullVersion/gradle/wrapper"
def wrapperBaseUrl = "https://raw.githubusercontent.com/gradle/gradle/v$versions.gradle/gradle/wrapper"
def wrapperJarUrl = wrapperBaseUrl + "/gradle-wrapper.jar"
def bootstrapString = """
@ -59,13 +57,15 @@ task bootstrapWrapper() {
done
""".stripIndent()
String putBootstrapStringAbove = "# Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script."
def wrapperScript = wrapper.scriptFile
def wrapperLines = wrapperScript.readLines()
wrapperScript.withPrintWriter { out ->
def bootstrapWritten = false
wrapperLines.each { line ->
// Print the wrapper bootstrap before the first usage of the wrapper jar.
if (!bootstrapWritten && line.contains("gradle-wrapper.jar")) {
if (!bootstrapWritten && line.contains(putBootstrapStringAbove)) {
out.println(bootstrapString)
bootstrapWritten = true
}