mirror of https://github.com/apache/kafka.git
Compare commits
26 Commits
bffcbb828c
...
6969ca94d2
Author | SHA1 | Date |
---|---|---|
|
6969ca94d2 | |
|
4a5aa37169 | |
|
2938c4242e | |
|
ebae768bd8 | |
|
fa2496bb91 | |
|
c6bbbbe24d | |
|
f5a87b3703 | |
|
2da02d9fcf | |
|
162db130f6 | |
|
f68a149a18 | |
|
24cad50840 | |
|
71a7d85955 | |
|
611f4128b4 | |
|
f9703f7e8b | |
|
cf1f13b0c2 | |
|
ff404d01ec | |
|
69975c02ae | |
|
4fa33f2d0f | |
|
f15d2ef460 | |
|
fe3df3c168 | |
|
5817251104 | |
|
d1ab4b41b9 | |
|
f91dc5a239 | |
|
2b68e44a35 | |
|
9922d9dacd | |
|
39fb8a0f95 |
|
@ -42,7 +42,7 @@ runs:
|
||||||
distribution: temurin
|
distribution: temurin
|
||||||
java-version: ${{ inputs.java-version }}
|
java-version: ${{ inputs.java-version }}
|
||||||
- name: Setup Gradle
|
- name: Setup Gradle
|
||||||
uses: gradle/actions/setup-gradle@94baf225fe0a508e581a564467443d0e2379123b # v4.3.0
|
uses: gradle/actions/setup-gradle@748248ddd2a24f49513d8f472f81c3a07d4d50e1 # v4.4.4
|
||||||
env:
|
env:
|
||||||
GRADLE_BUILD_ACTION_CACHE_DEBUG_ENABLED: true
|
GRADLE_BUILD_ACTION_CACHE_DEBUG_ENABLED: true
|
||||||
with:
|
with:
|
||||||
|
|
|
@ -38,7 +38,7 @@ run-name: Build Scans for ${{ github.event.workflow_run.display_title}}
|
||||||
jobs:
|
jobs:
|
||||||
upload-build-scan:
|
upload-build-scan:
|
||||||
# Skip this workflow if the CI run was skipped or cancelled
|
# 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
|
runs-on: ubuntu-latest
|
||||||
strategy:
|
strategy:
|
||||||
fail-fast: false
|
fail-fast: false
|
||||||
|
|
23
build.gradle
23
build.gradle
|
@ -29,22 +29,21 @@ buildscript {
|
||||||
}
|
}
|
||||||
|
|
||||||
plugins {
|
plugins {
|
||||||
id 'com.github.ben-manes.versions' version '0.48.0'
|
id 'com.github.ben-manes.versions' version '0.52.0'
|
||||||
id 'idea'
|
id 'idea'
|
||||||
id 'jacoco'
|
id 'jacoco'
|
||||||
id 'java-library'
|
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 'org.nosphere.apache.rat' version "0.8.1"
|
||||||
id "io.swagger.core.v3.swagger-gradle-plugin" version "${swaggerVersion}"
|
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 'org.scoverage' version '8.0.3' apply false
|
||||||
id 'com.gradleup.shadow' version '8.3.6' apply false
|
id 'com.gradleup.shadow' version '8.3.9' apply false
|
||||||
id 'com.diffplug.spotless' version "6.25.0"
|
id 'com.diffplug.spotless' version "7.2.1"
|
||||||
}
|
}
|
||||||
|
|
||||||
ext {
|
ext {
|
||||||
gradleVersion = versions.gradle
|
|
||||||
minClientJavaVersion = 11
|
minClientJavaVersion = 11
|
||||||
minNonClientJavaVersion = 17
|
minNonClientJavaVersion = 17
|
||||||
modulesNeedingJava11 = [":clients", ":generator", ":streams", ":streams:test-utils", ":streams:examples", ":streams-scala", ":test-common:test-common-util"]
|
modulesNeedingJava11 = [":clients", ":generator", ":streams", ":streams:test-utils", ":streams:examples", ":streams-scala", ":test-common:test-common-util"]
|
||||||
|
@ -297,7 +296,7 @@ if (repo != null) {
|
||||||
} else {
|
} else {
|
||||||
rat.enabled = false
|
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")
|
println("Build properties: ignoreFailures=$userIgnoreFailures, maxParallelForks=$maxTestForks, maxScalacThreads=$maxScalacThreads, maxTestRetries=$userMaxTestRetries")
|
||||||
|
|
||||||
subprojects {
|
subprojects {
|
||||||
|
@ -328,6 +327,16 @@ subprojects {
|
||||||
tasks.register('uploadArchives').configure { dependsOn(publish) }
|
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.
|
// apply the eclipse plugin only to subprojects that hold code. 'connect' is just a folder.
|
||||||
if (!project.name.equals('connect')) {
|
if (!project.name.equals('connect')) {
|
||||||
apply plugin: 'eclipse'
|
apply plugin: 'eclipse'
|
||||||
|
|
|
@ -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);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -39,6 +39,8 @@ import java.util.Queue;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.CompletableFuture;
|
||||||
import java.util.concurrent.ConcurrentLinkedDeque;
|
import java.util.concurrent.ConcurrentLinkedDeque;
|
||||||
|
import java.util.concurrent.CountDownLatch;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.function.Consumer;
|
import java.util.function.Consumer;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
@ -72,6 +74,8 @@ public class MockClient implements KafkaClient {
|
||||||
private int correlation;
|
private int correlation;
|
||||||
private Runnable wakeupHook;
|
private Runnable wakeupHook;
|
||||||
private boolean advanceTimeDuringPoll;
|
private boolean advanceTimeDuringPoll;
|
||||||
|
private boolean shouldUpdateWithCurrentMetadata = true;
|
||||||
|
private CountDownLatch isMetadataUpdateNeeded = new CountDownLatch(1);
|
||||||
private final Time time;
|
private final Time time;
|
||||||
private final MockMetadataUpdater metadataUpdater;
|
private final MockMetadataUpdater metadataUpdater;
|
||||||
private final Map<String, ConnectionState> connections = new HashMap<>();
|
private final Map<String, ConnectionState> connections = new HashMap<>();
|
||||||
|
@ -192,6 +196,10 @@ public class MockClient implements KafkaClient {
|
||||||
this.disconnectFuture = disconnectFuture;
|
this.disconnectFuture = disconnectFuture;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void setShouldUpdateWithCurrentMetadata(boolean shouldUpdateWithCurrentMetadata) {
|
||||||
|
this.shouldUpdateWithCurrentMetadata = shouldUpdateWithCurrentMetadata;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void disconnect(String node) {
|
public void disconnect(String node) {
|
||||||
disconnect(node, false);
|
disconnect(node, false);
|
||||||
|
@ -329,8 +337,10 @@ public class MockClient implements KafkaClient {
|
||||||
MetadataUpdate metadataUpdate = metadataUpdates.poll();
|
MetadataUpdate metadataUpdate = metadataUpdates.poll();
|
||||||
if (metadataUpdate != null) {
|
if (metadataUpdate != null) {
|
||||||
metadataUpdater.update(time, metadataUpdate);
|
metadataUpdater.update(time, metadataUpdate);
|
||||||
} else {
|
} else if (shouldUpdateWithCurrentMetadata) {
|
||||||
metadataUpdater.updateWithCurrentMetadata(time);
|
metadataUpdater.updateWithCurrentMetadata(time);
|
||||||
|
} else {
|
||||||
|
isMetadataUpdateNeeded.countDown();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -350,6 +360,14 @@ public class MockClient implements KafkaClient {
|
||||||
return copy;
|
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) {
|
private long elapsedTimeMs(long currentTimeMs, long startTimeMs) {
|
||||||
return Math.max(0, currentTimeMs - startTimeMs);
|
return Math.max(0, currentTimeMs - startTimeMs);
|
||||||
}
|
}
|
||||||
|
@ -523,6 +541,10 @@ public class MockClient implements KafkaClient {
|
||||||
metadataUpdates.add(new MetadataUpdate(updateResponse, expectMatchMetadataTopics));
|
metadataUpdates.add(new MetadataUpdate(updateResponse, expectMatchMetadataTopics));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public int preparedMetadataUpdatesCount() {
|
||||||
|
return metadataUpdates.size();
|
||||||
|
}
|
||||||
|
|
||||||
public void updateMetadata(MetadataResponse updateResponse) {
|
public void updateMetadata(MetadataResponse updateResponse) {
|
||||||
metadataUpdater.update(time, new MetadataUpdate(updateResponse, false));
|
metadataUpdater.update(time, new MetadataUpdate(updateResponse, false));
|
||||||
}
|
}
|
||||||
|
|
|
@ -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.AutoOffsetResetStrategy;
|
||||||
import org.apache.kafka.clients.consumer.internals.SubscriptionState;
|
import org.apache.kafka.clients.consumer.internals.SubscriptionState;
|
||||||
import org.apache.kafka.common.TopicPartition;
|
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.metrics.Metrics;
|
||||||
import org.apache.kafka.common.utils.LogContext;
|
import org.apache.kafka.common.utils.LogContext;
|
||||||
import org.apache.kafka.common.utils.MockTime;
|
import org.apache.kafka.common.utils.MockTime;
|
||||||
import org.apache.kafka.common.utils.Time;
|
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 org.junit.jupiter.api.Test;
|
||||||
|
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
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.assertNotNull;
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||||
import static org.mockito.Mockito.mock;
|
import static org.mockito.Mockito.mock;
|
||||||
|
|
||||||
|
|
||||||
class ConsumerRebalanceMetricsManagerTest {
|
class ConsumerRebalanceMetricsManagerTest {
|
||||||
|
|
||||||
private final Time time = new MockTime();
|
private Time time;
|
||||||
private final Metrics metrics = new Metrics(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
|
@Test
|
||||||
public void testAssignedPartitionCountMetric() {
|
public void testAssignedPartitionCountMetric() {
|
||||||
SubscriptionState subscriptionState = new SubscriptionState(mock(LogContext.class), AutoOffsetResetStrategy.EARLIEST);
|
assertNotNull(metrics.metric(metricsManager.assignedPartitionsCount), "Metric assigned-partitions has not been registered as expected");
|
||||||
ConsumerRebalanceMetricsManager consumerRebalanceMetricsManager = new ConsumerRebalanceMetricsManager(metrics, subscriptionState);
|
|
||||||
|
|
||||||
assertNotNull(metrics.metric(consumerRebalanceMetricsManager.assignedPartitionsCount), "Metric assigned-partitions has not been registered as expected");
|
|
||||||
|
|
||||||
// Check for manually assigned partitions
|
// Check for manually assigned partitions
|
||||||
subscriptionState.assignFromUser(Set.of(new TopicPartition("topic", 0), new TopicPartition("topic", 1)));
|
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());
|
subscriptionState.assignFromUser(Set.of());
|
||||||
assertEquals(0.0d, metrics.metric(consumerRebalanceMetricsManager.assignedPartitionsCount).metricValue());
|
assertEquals(0.0d, metrics.metric(metricsManager.assignedPartitionsCount).metricValue());
|
||||||
|
|
||||||
subscriptionState.unsubscribe();
|
subscriptionState.unsubscribe();
|
||||||
assertEquals(0.0d, metrics.metric(consumerRebalanceMetricsManager.assignedPartitionsCount).metricValue());
|
assertEquals(0.0d, metrics.metric(metricsManager.assignedPartitionsCount).metricValue());
|
||||||
|
|
||||||
// Check for automatically assigned partitions
|
// Check for automatically assigned partitions
|
||||||
subscriptionState.subscribe(Set.of("topic"), Optional.empty());
|
subscriptionState.subscribe(Set.of("topic"), Optional.empty());
|
||||||
subscriptionState.assignFromSubscribed(Set.of(new TopicPartition("topic", 0)));
|
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");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -128,13 +128,11 @@ import java.util.Map;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.Properties;
|
import java.util.Properties;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.Exchanger;
|
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
import java.util.concurrent.Executors;
|
import java.util.concurrent.Executors;
|
||||||
import java.util.concurrent.Future;
|
import java.util.concurrent.Future;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
import java.util.concurrent.atomic.AtomicReference;
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
@ -961,10 +959,9 @@ public class KafkaProducerTest {
|
||||||
@Test
|
@Test
|
||||||
public void testTopicRefreshInMetadata() throws InterruptedException {
|
public void testTopicRefreshInMetadata() throws InterruptedException {
|
||||||
Map<String, Object> configs = new HashMap<>();
|
Map<String, Object> configs = new HashMap<>();
|
||||||
|
final String maxBlockMs = "600000";
|
||||||
configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999");
|
configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999");
|
||||||
configs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, "600000");
|
configs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, maxBlockMs);
|
||||||
// test under normal producer for simplicity
|
|
||||||
configs.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, false);
|
|
||||||
long refreshBackoffMs = 500L;
|
long refreshBackoffMs = 500L;
|
||||||
long refreshBackoffMaxMs = 5000L;
|
long refreshBackoffMaxMs = 5000L;
|
||||||
long metadataExpireMs = 60000L;
|
long metadataExpireMs = 60000L;
|
||||||
|
@ -972,35 +969,35 @@ public class KafkaProducerTest {
|
||||||
final Time time = new MockTime();
|
final Time time = new MockTime();
|
||||||
final ProducerMetadata metadata = new ProducerMetadata(refreshBackoffMs, refreshBackoffMaxMs, metadataExpireMs, metadataIdleMs,
|
final ProducerMetadata metadata = new ProducerMetadata(refreshBackoffMs, refreshBackoffMaxMs, metadataExpireMs, metadataIdleMs,
|
||||||
new LogContext(), new ClusterResourceListeners(), time);
|
new LogContext(), new ClusterResourceListeners(), time);
|
||||||
|
final String warmupTopic = "warmup-topic";
|
||||||
final String topic = "topic";
|
final String topic = "topic";
|
||||||
|
MockClient client = new MockClient(time, metadata);
|
||||||
|
client.setShouldUpdateWithCurrentMetadata(false);
|
||||||
|
client.advanceTimeDuringPoll(true);
|
||||||
try (KafkaProducer<String, String> producer = kafkaProducer(configs,
|
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);
|
MetadataResponse updateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap(warmupTopic, 1));
|
||||||
Thread t = new Thread(() -> {
|
client.updateMetadata(updateResponse);
|
||||||
long startTimeMs = System.currentTimeMillis();
|
|
||||||
while (running.get()) {
|
final int preparedUpdatesCount = 100;
|
||||||
while (!metadata.updateRequested() && System.currentTimeMillis() - startTimeMs < 100)
|
updateResponse = RequestTestUtils.metadataUpdateWith("kafka-cluster", 1,
|
||||||
Thread.yield();
|
singletonMap(topic, Errors.UNKNOWN_TOPIC_OR_PARTITION), emptyMap());
|
||||||
MetadataResponse updateResponse = RequestTestUtils.metadataUpdateWith("kafka-cluster", 1,
|
for (int i = 0; i < preparedUpdatesCount; i++) {
|
||||||
singletonMap(topic, Errors.UNKNOWN_TOPIC_OR_PARTITION), emptyMap());
|
client.prepareMetadataUpdate(updateResponse);
|
||||||
metadata.updateWithCurrentRequestVersion(updateResponse, false, time.milliseconds());
|
}
|
||||||
time.sleep(60 * 1000L);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
t.start();
|
|
||||||
Throwable throwable = assertThrows(TimeoutException.class, () -> producer.partitionsFor(topic));
|
Throwable throwable = assertThrows(TimeoutException.class, () -> producer.partitionsFor(topic));
|
||||||
assertInstanceOf(UnknownTopicOrPartitionException.class, throwable.getCause());
|
assertInstanceOf(UnknownTopicOrPartitionException.class, throwable.getCause());
|
||||||
running.set(false);
|
assertTrue(preparedUpdatesCount > client.preparedMetadataUpdatesCount());
|
||||||
t.join();
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testTopicNotExistingInMetadata() throws InterruptedException {
|
public void testTopicNotExistingInMetadata() {
|
||||||
Map<String, Object> configs = new HashMap<>();
|
Map<String, Object> configs = new HashMap<>();
|
||||||
|
final String maxBlockMs = "30000";
|
||||||
configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999");
|
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 refreshBackoffMs = 500L;
|
||||||
long refreshBackoffMaxMs = 5000L;
|
long refreshBackoffMaxMs = 5000L;
|
||||||
long metadataExpireMs = 60000L;
|
long metadataExpireMs = 60000L;
|
||||||
|
@ -1008,39 +1005,33 @@ public class KafkaProducerTest {
|
||||||
final Time time = new MockTime();
|
final Time time = new MockTime();
|
||||||
final ProducerMetadata metadata = new ProducerMetadata(refreshBackoffMs, refreshBackoffMaxMs, metadataExpireMs, metadataIdleMs,
|
final ProducerMetadata metadata = new ProducerMetadata(refreshBackoffMs, refreshBackoffMaxMs, metadataExpireMs, metadataIdleMs,
|
||||||
new LogContext(), new ClusterResourceListeners(), time);
|
new LogContext(), new ClusterResourceListeners(), time);
|
||||||
|
final String warmupTopic = "warmup-topic";
|
||||||
final String topic = "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(),
|
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));
|
Throwable throwable = assertThrows(TimeoutException.class, () -> producer.partitionsFor(topic));
|
||||||
assertInstanceOf(UnknownTopicOrPartitionException.class, throwable.getCause());
|
assertInstanceOf(UnknownTopicOrPartitionException.class, throwable.getCause());
|
||||||
t.join();
|
assertEquals(0, client.preparedMetadataUpdatesCount());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testTopicExpiryInMetadata() throws InterruptedException {
|
public void testTopicExpiryInMetadata() throws InterruptedException {
|
||||||
Map<String, Object> configs = new HashMap<>();
|
Map<String, Object> configs = new HashMap<>();
|
||||||
|
final String maxBlockMs = "300000";
|
||||||
configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999");
|
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 refreshBackoffMs = 500L;
|
||||||
long refreshBackoffMaxMs = 5000L;
|
long refreshBackoffMaxMs = 5000L;
|
||||||
long metadataExpireMs = 60000L;
|
long metadataExpireMs = 60000L;
|
||||||
|
@ -1049,39 +1040,28 @@ public class KafkaProducerTest {
|
||||||
final ProducerMetadata metadata = new ProducerMetadata(refreshBackoffMs, refreshBackoffMaxMs, metadataExpireMs, metadataIdleMs,
|
final ProducerMetadata metadata = new ProducerMetadata(refreshBackoffMs, refreshBackoffMaxMs, metadataExpireMs, metadataIdleMs,
|
||||||
new LogContext(), new ClusterResourceListeners(), time);
|
new LogContext(), new ClusterResourceListeners(), time);
|
||||||
final String topic = "topic";
|
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(),
|
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(() -> {
|
updateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap(topic, 1));
|
||||||
try {
|
client.prepareMetadataUpdate(updateResponse);
|
||||||
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
|
|
||||||
assertNotNull(producer.partitionsFor(topic));
|
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));
|
assertThrows(TimeoutException.class, () -> producer.partitionsFor(topic));
|
||||||
t.join();
|
assertTrue(client.awaitMetadataUpdateRequest(0));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -233,13 +233,13 @@ public class MirrorMaker {
|
||||||
private void addHerder(SourceAndTarget sourceAndTarget) {
|
private void addHerder(SourceAndTarget sourceAndTarget) {
|
||||||
log.info("creating herder for {}", sourceAndTarget.toString());
|
log.info("creating herder for {}", sourceAndTarget.toString());
|
||||||
Map<String, String> workerProps = config.workerConfig(sourceAndTarget);
|
Map<String, String> workerProps = config.workerConfig(sourceAndTarget);
|
||||||
|
DistributedConfig distributedConfig = new DistributedConfig(workerProps);
|
||||||
String encodedSource = encodePath(sourceAndTarget.source());
|
String encodedSource = encodePath(sourceAndTarget.source());
|
||||||
String encodedTarget = encodePath(sourceAndTarget.target());
|
String encodedTarget = encodePath(sourceAndTarget.target());
|
||||||
List<String> restNamespace = List.of(encodedSource, encodedTarget);
|
List<String> restNamespace = List.of(encodedSource, encodedTarget);
|
||||||
String workerId = generateWorkerId(sourceAndTarget);
|
String workerId = generateWorkerId(sourceAndTarget);
|
||||||
Plugins plugins = new Plugins(workerProps);
|
Plugins plugins = new Plugins(workerProps);
|
||||||
plugins.compareAndSwapWithDelegatingLoader();
|
plugins.compareAndSwapWithDelegatingLoader();
|
||||||
DistributedConfig distributedConfig = new DistributedConfig(workerProps);
|
|
||||||
String kafkaClusterId = distributedConfig.kafkaClusterId();
|
String kafkaClusterId = distributedConfig.kafkaClusterId();
|
||||||
String clientIdBase = ConnectUtils.clientIdBase(distributedConfig);
|
String clientIdBase = ConnectUtils.clientIdBase(distributedConfig);
|
||||||
// Create the admin client to be shared by all backing stores for this herder
|
// Create the admin client to be shared by all backing stores for this herder
|
||||||
|
|
|
@ -114,14 +114,15 @@ public abstract class AbstractConnectCli<H extends Herder, T extends WorkerConfi
|
||||||
log.info("Kafka Connect worker initializing ...");
|
log.info("Kafka Connect worker initializing ...");
|
||||||
long initStart = time.hiResClockMs();
|
long initStart = time.hiResClockMs();
|
||||||
|
|
||||||
|
T config = createConfig(workerProps);
|
||||||
|
log.debug("Kafka cluster ID: {}", config.kafkaClusterId());
|
||||||
|
|
||||||
WorkerInfo initInfo = new WorkerInfo();
|
WorkerInfo initInfo = new WorkerInfo();
|
||||||
initInfo.logAll();
|
initInfo.logAll();
|
||||||
|
|
||||||
log.info("Scanning for plugin classes. This might take a moment ...");
|
log.info("Scanning for plugin classes. This might take a moment ...");
|
||||||
Plugins plugins = new Plugins(workerProps);
|
Plugins plugins = new Plugins(workerProps);
|
||||||
plugins.compareAndSwapWithDelegatingLoader();
|
plugins.compareAndSwapWithDelegatingLoader();
|
||||||
T config = createConfig(workerProps);
|
|
||||||
log.debug("Kafka cluster ID: {}", config.kafkaClusterId());
|
|
||||||
|
|
||||||
RestClient restClient = new RestClient(config);
|
RestClient restClient = new RestClient(config);
|
||||||
|
|
||||||
|
|
|
@ -20,7 +20,6 @@ import java.lang.{Long => JLong}
|
||||||
import java.util.concurrent.locks.ReentrantReadWriteLock
|
import java.util.concurrent.locks.ReentrantReadWriteLock
|
||||||
import java.util.Optional
|
import java.util.Optional
|
||||||
import java.util.concurrent.{CompletableFuture, ConcurrentHashMap, CopyOnWriteArrayList}
|
import java.util.concurrent.{CompletableFuture, ConcurrentHashMap, CopyOnWriteArrayList}
|
||||||
import kafka.controller.StateChangeLogger
|
|
||||||
import kafka.log._
|
import kafka.log._
|
||||||
import kafka.server._
|
import kafka.server._
|
||||||
import kafka.server.share.DelayedShareFetch
|
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._
|
||||||
import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET}
|
import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET}
|
||||||
import org.apache.kafka.common.utils.Time
|
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.metadata.{LeaderAndIsr, LeaderRecoveryState, MetadataCache, PartitionRegistration}
|
||||||
import org.apache.kafka.server.common.RequestLocal
|
import org.apache.kafka.server.common.RequestLocal
|
||||||
import org.apache.kafka.server.log.remote.TopicPartitionLog
|
import org.apache.kafka.server.log.remote.TopicPartitionLog
|
||||||
|
@ -322,7 +322,7 @@ class Partition(val topicPartition: TopicPartition,
|
||||||
def topic: String = topicPartition.topic
|
def topic: String = topicPartition.topic
|
||||||
def partitionId: Int = topicPartition.partition
|
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]
|
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
|
// The read lock is only required when multiple reads are executed and needs to be in a consistent manner
|
||||||
private val leaderIsrUpdateLock = new ReentrantReadWriteLock
|
private val leaderIsrUpdateLock = new ReentrantReadWriteLock
|
||||||
|
|
|
@ -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] "
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
|
@ -18,7 +18,6 @@ package kafka.server
|
||||||
|
|
||||||
import com.yammer.metrics.core.Meter
|
import com.yammer.metrics.core.Meter
|
||||||
import kafka.cluster.{Partition, PartitionListener}
|
import kafka.cluster.{Partition, PartitionListener}
|
||||||
import kafka.controller.StateChangeLogger
|
|
||||||
import kafka.log.LogManager
|
import kafka.log.LogManager
|
||||||
import kafka.server.HostedPartition.Online
|
import kafka.server.HostedPartition.Online
|
||||||
import kafka.server.QuotaFactory.QuotaManagers
|
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.common.utils.{Exit, Time, Utils}
|
||||||
import org.apache.kafka.coordinator.transaction.{AddPartitionsToTxnConfig, TransactionLogConfig}
|
import org.apache.kafka.coordinator.transaction.{AddPartitionsToTxnConfig, TransactionLogConfig}
|
||||||
import org.apache.kafka.image.{LocalReplicaChanges, MetadataImage, TopicsDelta}
|
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.LeaderConstants.NO_LEADER
|
||||||
import org.apache.kafka.metadata.MetadataCache
|
import org.apache.kafka.metadata.MetadataCache
|
||||||
import org.apache.kafka.server.common.{DirectoryEventHandler, RequestLocal, StopPartition}
|
import org.apache.kafka.server.common.{DirectoryEventHandler, RequestLocal, StopPartition}
|
||||||
|
@ -272,7 +272,7 @@ class ReplicaManager(val config: KafkaConfig,
|
||||||
@volatile private var isInControlledShutdown = false
|
@volatile private var isInControlledShutdown = false
|
||||||
|
|
||||||
this.logIdent = s"[ReplicaManager broker=$localBrokerId] "
|
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 = _
|
private var logDirFailureHandler: LogDirFailureHandler = _
|
||||||
|
|
||||||
|
@ -789,9 +789,9 @@ class ReplicaManager(val config: KafkaConfig,
|
||||||
hasCustomErrorMessage = customException.isDefined
|
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
|
// directly use entriesPerPartition instead of creating a new filtered collection
|
||||||
val entriesWithoutErrorsPerPartition =
|
val entriesWithoutErrorsPerPartition =
|
||||||
if (errorResults.nonEmpty) entriesPerPartition.filter { case (key, _) => !errorResults.contains(key) }
|
if (errorResults.nonEmpty) entriesPerPartition.filter { case (key, _) => !errorResults.contains(key) }
|
||||||
else entriesPerPartition
|
else entriesPerPartition
|
||||||
|
|
||||||
|
@ -1637,13 +1637,13 @@ class ReplicaManager(val config: KafkaConfig,
|
||||||
remoteFetchPartitionStatus: Seq[(TopicIdPartition, FetchPartitionStatus)]): Unit = {
|
remoteFetchPartitionStatus: Seq[(TopicIdPartition, FetchPartitionStatus)]): Unit = {
|
||||||
val remoteFetchTasks = new util.HashMap[TopicIdPartition, Future[Void]]
|
val remoteFetchTasks = new util.HashMap[TopicIdPartition, Future[Void]]
|
||||||
val remoteFetchResults = new util.HashMap[TopicIdPartition, CompletableFuture[RemoteLogReadResult]]
|
val remoteFetchResults = new util.HashMap[TopicIdPartition, CompletableFuture[RemoteLogReadResult]]
|
||||||
|
|
||||||
remoteFetchInfos.forEach { (topicIdPartition, remoteFetchInfo) =>
|
remoteFetchInfos.forEach { (topicIdPartition, remoteFetchInfo) =>
|
||||||
val (task, result) = processRemoteFetch(remoteFetchInfo)
|
val (task, result) = processRemoteFetch(remoteFetchInfo)
|
||||||
remoteFetchTasks.put(topicIdPartition, task)
|
remoteFetchTasks.put(topicIdPartition, task)
|
||||||
remoteFetchResults.put(topicIdPartition, result)
|
remoteFetchResults.put(topicIdPartition, result)
|
||||||
}
|
}
|
||||||
|
|
||||||
val remoteFetchMaxWaitMs = config.remoteLogManagerConfig.remoteFetchMaxWaitMs().toLong
|
val remoteFetchMaxWaitMs = config.remoteLogManagerConfig.remoteFetchMaxWaitMs().toLong
|
||||||
val remoteFetch = new DelayedRemoteFetch(remoteFetchTasks, remoteFetchResults, remoteFetchInfos, remoteFetchMaxWaitMs,
|
val remoteFetch = new DelayedRemoteFetch(remoteFetchTasks, remoteFetchResults, remoteFetchInfos, remoteFetchMaxWaitMs,
|
||||||
remoteFetchPartitionStatus, params, logReadResults, this, responseCallback)
|
remoteFetchPartitionStatus, params, logReadResults, this, responseCallback)
|
||||||
|
|
|
@ -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.LeaveGroupResponseData.MemberResponse
|
||||||
import org.apache.kafka.common.message.SyncGroupRequestData.SyncGroupRequestAssignment
|
import org.apache.kafka.common.message.SyncGroupRequestData.SyncGroupRequestAssignment
|
||||||
import org.apache.kafka.common.message.WriteTxnMarkersRequestData.{WritableTxnMarker, WritableTxnMarkerTopic}
|
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.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.serialization.StringSerializer
|
||||||
import org.apache.kafka.common.test.ClusterInstance
|
import org.apache.kafka.common.test.ClusterInstance
|
||||||
import org.apache.kafka.common.utils.ProducerIdAndEpoch
|
import org.apache.kafka.common.utils.ProducerIdAndEpoch
|
||||||
|
@ -768,6 +768,21 @@ class GroupCoordinatorBaseRequestTest(cluster: ClusterInstance) {
|
||||||
shareGroupDescribeResponse.data.groups.asScala.toList
|
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(
|
protected def heartbeat(
|
||||||
groupId: String,
|
groupId: String,
|
||||||
generationId: Int,
|
generationId: Int,
|
||||||
|
@ -855,6 +870,41 @@ class GroupCoordinatorBaseRequestTest(cluster: ClusterInstance) {
|
||||||
shareGroupHeartbeatResponse.data
|
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(
|
protected def leaveGroupWithNewProtocol(
|
||||||
groupId: String,
|
groupId: String,
|
||||||
memberId: String
|
memberId: String
|
||||||
|
|
|
@ -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
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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>
|
<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>
|
<h4>Early Access of the Streams Rebalance Protocol</h4>
|
||||||
|
|
||||||
<p>
|
<p>
|
||||||
|
|
|
@ -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,
|
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).
|
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>
|
||||||
|
<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>
|
</ul>
|
||||||
|
|
||||||
<h4><a id="upgrade_4_1_0" href="#upgrade_4_1_0">Upgrading to 4.1.0</a></h4>
|
<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_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>
|
<h5><a id="upgrade_410_notable" href="#upgrade_410_notable">Notable changes in 4.1.0</a></h5>
|
||||||
<ul>
|
<ul>
|
||||||
|
|
|
@ -60,7 +60,7 @@ versions += [
|
||||||
commonsLang: "3.18.0",
|
commonsLang: "3.18.0",
|
||||||
commonsValidator: "1.10.0",
|
commonsValidator: "1.10.0",
|
||||||
classgraph: "4.8.179",
|
classgraph: "4.8.179",
|
||||||
gradle: "8.14.3",
|
gradle: "9.1.0",
|
||||||
grgit: "4.1.1",
|
grgit: "4.1.1",
|
||||||
httpclient: "4.5.14",
|
httpclient: "4.5.14",
|
||||||
jackson: "2.19.0",
|
jackson: "2.19.0",
|
||||||
|
@ -125,7 +125,7 @@ versions += [
|
||||||
snappy: "1.1.10.7",
|
snappy: "1.1.10.7",
|
||||||
spotbugs: "4.9.4",
|
spotbugs: "4.9.4",
|
||||||
mockOAuth2Server: "2.2.1",
|
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
|
// 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
|
// Also make sure the compression levels in org.apache.kafka.common.record.CompressionType are still valid
|
||||||
zstd: "1.5.6-10",
|
zstd: "1.5.6-10",
|
||||||
|
|
|
@ -1,7 +1,7 @@
|
||||||
distributionBase=GRADLE_USER_HOME
|
distributionBase=GRADLE_USER_HOME
|
||||||
distributionPath=wrapper/dists
|
distributionPath=wrapper/dists
|
||||||
distributionSha256Sum=bd71102213493060956ec229d946beee57158dbd89d0e62b91bca0fa2c5f3531
|
distributionSha256Sum=a17ddd85a26b6a7f5ddb71ff8b05fc5104c0202c6e64782429790c933686c806
|
||||||
distributionUrl=https\://services.gradle.org/distributions/gradle-8.14.3-bin.zip
|
distributionUrl=https\://services.gradle.org/distributions/gradle-9.1.0-bin.zip
|
||||||
networkTimeout=10000
|
networkTimeout=10000
|
||||||
validateDistributionUrl=true
|
validateDistributionUrl=true
|
||||||
zipStoreBase=GRADLE_USER_HOME
|
zipStoreBase=GRADLE_USER_HOME
|
||||||
|
|
|
@ -1,7 +1,7 @@
|
||||||
#!/bin/sh
|
#!/bin/sh
|
||||||
|
|
||||||
#
|
#
|
||||||
# Copyright © 2015-2021 the original authors.
|
# Copyright © 2015 the original authors.
|
||||||
#
|
#
|
||||||
# Licensed under the Apache License, Version 2.0 (the "License");
|
# Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
# you may not use this file except in compliance with 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
|
# See the License for the specific language governing permissions and
|
||||||
# limitations under the License.
|
# limitations under the License.
|
||||||
#
|
#
|
||||||
|
# SPDX-License-Identifier: Apache-2.0
|
||||||
|
#
|
||||||
|
|
||||||
##############################################################################
|
##############################################################################
|
||||||
#
|
#
|
||||||
|
@ -55,7 +57,7 @@
|
||||||
# Darwin, MinGW, and NonStop.
|
# Darwin, MinGW, and NonStop.
|
||||||
#
|
#
|
||||||
# (3) This script is generated from the Groovy template
|
# (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.
|
# within the Gradle project.
|
||||||
#
|
#
|
||||||
# You can find Gradle at https://github.com/gradle/gradle/.
|
# You can find Gradle at https://github.com/gradle/gradle/.
|
||||||
|
@ -84,7 +86,7 @@ done
|
||||||
# shellcheck disable=SC2034
|
# shellcheck disable=SC2034
|
||||||
APP_BASE_NAME=${0##*/}
|
APP_BASE_NAME=${0##*/}
|
||||||
# Discard cd standard output in case $CDPATH is set (https://github.com/gradle/gradle/issues/25036)
|
# 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.
|
# Use the maximum available, or set MAX_FD != -1 to use that value.
|
||||||
MAX_FD=maximum
|
MAX_FD=maximum
|
||||||
|
@ -113,20 +115,6 @@ case "$( uname )" in #(
|
||||||
esac
|
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.
|
# Determine the Java command to use to start the JVM.
|
||||||
if [ -n "$JAVA_HOME" ] ; then
|
if [ -n "$JAVA_HOME" ] ; then
|
||||||
|
@ -183,7 +171,6 @@ fi
|
||||||
# For Cygwin or MSYS, switch paths to Windows format before running java
|
# For Cygwin or MSYS, switch paths to Windows format before running java
|
||||||
if "$cygwin" || "$msys" ; then
|
if "$cygwin" || "$msys" ; then
|
||||||
APP_HOME=$( cygpath --path --mixed "$APP_HOME" )
|
APP_HOME=$( cygpath --path --mixed "$APP_HOME" )
|
||||||
CLASSPATH=$( cygpath --path --mixed "$CLASSPATH" )
|
|
||||||
|
|
||||||
JAVACMD=$( cygpath --unix "$JAVACMD" )
|
JAVACMD=$( cygpath --unix "$JAVACMD" )
|
||||||
|
|
||||||
|
@ -212,19 +199,31 @@ if "$cygwin" || "$msys" ; then
|
||||||
fi
|
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.
|
# 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"'
|
DEFAULT_JVM_OPTS='"-Xmx64m" "-Xms64m"'
|
||||||
|
|
||||||
# Collect all arguments for the java command:
|
# 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.
|
# 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
|
# * 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.
|
# treated as '${Hostname}' itself on the command line.
|
||||||
|
|
||||||
set -- \
|
set -- \
|
||||||
"-Dorg.gradle.appname=$APP_BASE_NAME" \
|
"-Dorg.gradle.appname=$APP_BASE_NAME" \
|
||||||
-classpath "$CLASSPATH" \
|
-jar "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" \
|
||||||
org.gradle.wrapper.GradleWrapperMain \
|
|
||||||
"$@"
|
"$@"
|
||||||
|
|
||||||
# Stop when "xargs" is not available.
|
# Stop when "xargs" is not available.
|
||||||
|
|
|
@ -2249,18 +2249,13 @@ public class GroupMetadataManager {
|
||||||
.setClassicMemberMetadata(null)
|
.setClassicMemberMetadata(null)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
// If the group is newly created, we must ensure that it moves away from
|
boolean subscribedTopicNamesChanged = hasMemberSubscriptionChanged(
|
||||||
// epoch 0 and that it is fully initialized.
|
|
||||||
boolean bumpGroupEpoch = group.groupEpoch() == 0;
|
|
||||||
|
|
||||||
bumpGroupEpoch |= hasMemberSubscriptionChanged(
|
|
||||||
groupId,
|
groupId,
|
||||||
member,
|
member,
|
||||||
updatedMember,
|
updatedMember,
|
||||||
records
|
records
|
||||||
);
|
);
|
||||||
|
UpdateRegularExpressionsResult updateRegularExpressionsResult = maybeUpdateRegularExpressions(
|
||||||
bumpGroupEpoch |= maybeUpdateRegularExpressions(
|
|
||||||
context,
|
context,
|
||||||
group,
|
group,
|
||||||
member,
|
member,
|
||||||
|
@ -2268,9 +2263,24 @@ public class GroupMetadataManager {
|
||||||
records
|
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();
|
int groupEpoch = group.groupEpoch();
|
||||||
SubscriptionType subscriptionType = group.subscriptionType();
|
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)) {
|
if (bumpGroupEpoch || group.hasMetadataExpired(currentTimeMs)) {
|
||||||
// The subscription metadata is updated in two cases:
|
// The subscription metadata is updated in two cases:
|
||||||
// 1) The member has updated its subscriptions;
|
// 1) The member has updated its subscriptions;
|
||||||
|
@ -2315,6 +2325,9 @@ public class GroupMetadataManager {
|
||||||
group::currentPartitionEpoch,
|
group::currentPartitionEpoch,
|
||||||
targetAssignmentEpoch,
|
targetAssignmentEpoch,
|
||||||
targetAssignment,
|
targetAssignment,
|
||||||
|
group.resolvedRegularExpressions(),
|
||||||
|
// Force consistency with the subscription when the subscription has changed.
|
||||||
|
hasSubscriptionChanged,
|
||||||
ownedTopicPartitions,
|
ownedTopicPartitions,
|
||||||
records
|
records
|
||||||
);
|
);
|
||||||
|
@ -2468,6 +2481,8 @@ public class GroupMetadataManager {
|
||||||
group::currentPartitionEpoch,
|
group::currentPartitionEpoch,
|
||||||
group.assignmentEpoch(),
|
group.assignmentEpoch(),
|
||||||
group.targetAssignment(updatedMember.memberId(), updatedMember.instanceId()),
|
group.targetAssignment(updatedMember.memberId(), updatedMember.instanceId()),
|
||||||
|
group.resolvedRegularExpressions(),
|
||||||
|
bumpGroupEpoch,
|
||||||
toTopicPartitions(subscription.ownedPartitions(), metadataImage),
|
toTopicPartitions(subscription.ownedPartitions(), metadataImage),
|
||||||
records
|
records
|
||||||
);
|
);
|
||||||
|
@ -2511,6 +2526,9 @@ public class GroupMetadataManager {
|
||||||
group::currentPartitionEpoch,
|
group::currentPartitionEpoch,
|
||||||
targetAssignmentEpoch,
|
targetAssignmentEpoch,
|
||||||
targetAssignment,
|
targetAssignment,
|
||||||
|
group.resolvedRegularExpressions(),
|
||||||
|
// Force consistency with the subscription when the subscription has changed.
|
||||||
|
bumpGroupEpoch,
|
||||||
toTopicPartitions(subscription.ownedPartitions(), metadataImage),
|
toTopicPartitions(subscription.ownedPartitions(), metadataImage),
|
||||||
records
|
records
|
||||||
);
|
);
|
||||||
|
@ -2669,6 +2687,8 @@ public class GroupMetadataManager {
|
||||||
updatedMember,
|
updatedMember,
|
||||||
targetAssignmentEpoch,
|
targetAssignmentEpoch,
|
||||||
targetAssignment,
|
targetAssignment,
|
||||||
|
// Force consistency with the subscription when the subscription has changed.
|
||||||
|
bumpGroupEpoch,
|
||||||
records
|
records
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -3108,6 +3128,16 @@ public class GroupMetadataManager {
|
||||||
return value != null && !value.isEmpty();
|
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
|
* 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
|
* 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 member The old member.
|
||||||
* @param updatedMember The new member.
|
* @param updatedMember The new member.
|
||||||
* @param records The records accumulator.
|
* @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,
|
AuthorizableRequestContext context,
|
||||||
ConsumerGroup group,
|
ConsumerGroup group,
|
||||||
ConsumerGroupMember member,
|
ConsumerGroupMember member,
|
||||||
|
@ -3134,14 +3164,17 @@ public class GroupMetadataManager {
|
||||||
String oldSubscribedTopicRegex = member.subscribedTopicRegex();
|
String oldSubscribedTopicRegex = member.subscribedTopicRegex();
|
||||||
String newSubscribedTopicRegex = updatedMember.subscribedTopicRegex();
|
String newSubscribedTopicRegex = updatedMember.subscribedTopicRegex();
|
||||||
|
|
||||||
boolean bumpGroupEpoch = false;
|
|
||||||
boolean requireRefresh = false;
|
boolean requireRefresh = false;
|
||||||
|
UpdateRegularExpressionsResult updateRegularExpressionsResult = UpdateRegularExpressionsResult.NO_CHANGE;
|
||||||
|
|
||||||
// Check whether the member has changed its subscribed regex.
|
// 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: {}.",
|
log.debug("[GroupId {}] Member {} updated its subscribed regex to: {}.",
|
||||||
groupId, memberId, newSubscribedTopicRegex);
|
groupId, memberId, newSubscribedTopicRegex);
|
||||||
|
|
||||||
|
updateRegularExpressionsResult = UpdateRegularExpressionsResult.REGEX_UPDATED;
|
||||||
|
|
||||||
if (isNotEmpty(oldSubscribedTopicRegex) && group.numSubscribedMembers(oldSubscribedTopicRegex) == 1) {
|
if (isNotEmpty(oldSubscribedTopicRegex) && group.numSubscribedMembers(oldSubscribedTopicRegex) == 1) {
|
||||||
// If the member was the last one subscribed to the regex, we delete the
|
// If the member was the last one subscribed to the regex, we delete the
|
||||||
// resolved regular expression.
|
// resolved regular expression.
|
||||||
|
@ -3160,7 +3193,9 @@ public class GroupMetadataManager {
|
||||||
} else {
|
} else {
|
||||||
// If the new regex is already resolved, we trigger a rebalance
|
// If the new regex is already resolved, we trigger a rebalance
|
||||||
// by bumping the group epoch.
|
// 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
|
// 0. The group is subscribed to regular expressions. We also take the one
|
||||||
// that the current may have just introduced.
|
// that the current may have just introduced.
|
||||||
if (!requireRefresh && group.subscribedRegularExpressions().isEmpty()) {
|
if (!requireRefresh && group.subscribedRegularExpressions().isEmpty()) {
|
||||||
return bumpGroupEpoch;
|
return updateRegularExpressionsResult;
|
||||||
}
|
}
|
||||||
|
|
||||||
// 1. There is no ongoing refresh for the group.
|
// 1. There is no ongoing refresh for the group.
|
||||||
String key = group.groupId() + "-regex";
|
String key = group.groupId() + "-regex";
|
||||||
if (executor.isScheduled(key)) {
|
if (executor.isScheduled(key)) {
|
||||||
return bumpGroupEpoch;
|
return updateRegularExpressionsResult;
|
||||||
}
|
}
|
||||||
|
|
||||||
// 2. The last refresh is older than 10s. If the group does not have any regular
|
// 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.
|
// expressions but the current member just brought a new one, we should continue.
|
||||||
long lastRefreshTimeMs = group.lastResolvedRegularExpressionRefreshTimeMs();
|
long lastRefreshTimeMs = group.lastResolvedRegularExpressionRefreshTimeMs();
|
||||||
if (currentTimeMs <= lastRefreshTimeMs + REGEX_BATCH_REFRESH_MIN_INTERVAL_MS) {
|
if (currentTimeMs <= lastRefreshTimeMs + REGEX_BATCH_REFRESH_MIN_INTERVAL_MS) {
|
||||||
return bumpGroupEpoch;
|
return updateRegularExpressionsResult;
|
||||||
}
|
}
|
||||||
|
|
||||||
// 3.1 The group has unresolved regular expressions.
|
// 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.
|
* Reconciles the current assignment of the member towards the target assignment if needed.
|
||||||
*
|
*
|
||||||
* @param groupId The group id.
|
* @param groupId The group id.
|
||||||
* @param member The member to reconcile.
|
* @param member The member to reconcile.
|
||||||
* @param currentPartitionEpoch The function returning the current epoch of
|
* @param currentPartitionEpoch The function returning the current epoch of
|
||||||
* a given partition.
|
* a given partition.
|
||||||
* @param targetAssignmentEpoch The target assignment epoch.
|
* @param targetAssignmentEpoch The target assignment epoch.
|
||||||
* @param targetAssignment The target assignment.
|
* @param targetAssignment The target assignment.
|
||||||
* @param ownedTopicPartitions The list of partitions owned by the member. This
|
* @param resolvedRegularExpressions The resolved regular expressions.
|
||||||
* is reported in the ConsumerGroupHeartbeat API and
|
* @param hasSubscriptionChanged Whether the member has changed its subscription on the current heartbeat.
|
||||||
* it could be null if not provided.
|
* @param ownedTopicPartitions The list of partitions owned by the member. This
|
||||||
* @param records The list to accumulate any new records.
|
* 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
|
* @return The received member if no changes have been made; or a new
|
||||||
* member containing the new assignment.
|
* member containing the new assignment.
|
||||||
*/
|
*/
|
||||||
|
@ -3511,15 +3548,20 @@ public class GroupMetadataManager {
|
||||||
BiFunction<Uuid, Integer, Integer> currentPartitionEpoch,
|
BiFunction<Uuid, Integer, Integer> currentPartitionEpoch,
|
||||||
int targetAssignmentEpoch,
|
int targetAssignmentEpoch,
|
||||||
Assignment targetAssignment,
|
Assignment targetAssignment,
|
||||||
|
Map<String, ResolvedRegularExpression> resolvedRegularExpressions,
|
||||||
|
boolean hasSubscriptionChanged,
|
||||||
List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions,
|
List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions,
|
||||||
List<CoordinatorRecord> records
|
List<CoordinatorRecord> records
|
||||||
) {
|
) {
|
||||||
if (member.isReconciledTo(targetAssignmentEpoch)) {
|
if (!hasSubscriptionChanged && member.isReconciledTo(targetAssignmentEpoch)) {
|
||||||
return member;
|
return member;
|
||||||
}
|
}
|
||||||
|
|
||||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||||
|
.withMetadataImage(metadataImage)
|
||||||
.withTargetAssignment(targetAssignmentEpoch, targetAssignment)
|
.withTargetAssignment(targetAssignmentEpoch, targetAssignment)
|
||||||
|
.withHasSubscriptionChanged(hasSubscriptionChanged)
|
||||||
|
.withResolvedRegularExpressions(resolvedRegularExpressions)
|
||||||
.withCurrentPartitionEpoch(currentPartitionEpoch)
|
.withCurrentPartitionEpoch(currentPartitionEpoch)
|
||||||
.withOwnedTopicPartitions(ownedTopicPartitions)
|
.withOwnedTopicPartitions(ownedTopicPartitions)
|
||||||
.build();
|
.build();
|
||||||
|
@ -3556,11 +3598,12 @@ public class GroupMetadataManager {
|
||||||
/**
|
/**
|
||||||
* Reconciles the current assignment of the member towards the target assignment if needed.
|
* Reconciles the current assignment of the member towards the target assignment if needed.
|
||||||
*
|
*
|
||||||
* @param groupId The group id.
|
* @param groupId The group id.
|
||||||
* @param member The member to reconcile.
|
* @param member The member to reconcile.
|
||||||
* @param targetAssignmentEpoch The target assignment epoch.
|
* @param targetAssignmentEpoch The target assignment epoch.
|
||||||
* @param targetAssignment The target assignment.
|
* @param targetAssignment The target assignment.
|
||||||
* @param records The list to accumulate any new records.
|
* @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
|
* @return The received member if no changes have been made; or a new
|
||||||
* member containing the new assignment.
|
* member containing the new assignment.
|
||||||
*/
|
*/
|
||||||
|
@ -3569,14 +3612,17 @@ public class GroupMetadataManager {
|
||||||
ShareGroupMember member,
|
ShareGroupMember member,
|
||||||
int targetAssignmentEpoch,
|
int targetAssignmentEpoch,
|
||||||
Assignment targetAssignment,
|
Assignment targetAssignment,
|
||||||
|
boolean hasSubscriptionChanged,
|
||||||
List<CoordinatorRecord> records
|
List<CoordinatorRecord> records
|
||||||
) {
|
) {
|
||||||
if (member.isReconciledTo(targetAssignmentEpoch)) {
|
if (!hasSubscriptionChanged && member.isReconciledTo(targetAssignmentEpoch)) {
|
||||||
return member;
|
return member;
|
||||||
}
|
}
|
||||||
|
|
||||||
ShareGroupMember updatedMember = new ShareGroupAssignmentBuilder(member)
|
ShareGroupMember updatedMember = new ShareGroupAssignmentBuilder(member)
|
||||||
|
.withMetadataImage(metadataImage)
|
||||||
.withTargetAssignment(targetAssignmentEpoch, targetAssignment)
|
.withTargetAssignment(targetAssignmentEpoch, targetAssignment)
|
||||||
|
.withHasSubscriptionChanged(hasSubscriptionChanged)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
if (!updatedMember.equals(member)) {
|
if (!updatedMember.equals(member)) {
|
||||||
|
|
|
@ -19,8 +19,11 @@ package org.apache.kafka.coordinator.group.modern.consumer;
|
||||||
import org.apache.kafka.common.Uuid;
|
import org.apache.kafka.common.Uuid;
|
||||||
import org.apache.kafka.common.errors.FencedMemberEpochException;
|
import org.apache.kafka.common.errors.FencedMemberEpochException;
|
||||||
import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
|
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.Assignment;
|
||||||
import org.apache.kafka.coordinator.group.modern.MemberState;
|
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.HashMap;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
|
@ -41,6 +44,11 @@ public class CurrentAssignmentBuilder {
|
||||||
*/
|
*/
|
||||||
private final ConsumerGroupMember member;
|
private final ConsumerGroupMember member;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The metadata image.
|
||||||
|
*/
|
||||||
|
private CoordinatorMetadataImage metadataImage = CoordinatorMetadataImage.EMPTY;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The target assignment epoch.
|
* The target assignment epoch.
|
||||||
*/
|
*/
|
||||||
|
@ -51,6 +59,16 @@ public class CurrentAssignmentBuilder {
|
||||||
*/
|
*/
|
||||||
private Assignment targetAssignment;
|
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
|
* 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.
|
* 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);
|
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
|
* Sets the target assignment epoch and the target assignment that the
|
||||||
* consumer group member must be reconciled to.
|
* consumer group member must be reconciled to.
|
||||||
|
@ -90,6 +121,32 @@ public class CurrentAssignmentBuilder {
|
||||||
return this;
|
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
|
* Sets a BiFunction which allows to retrieve the current epoch of a
|
||||||
* partition. This is used by the state machine to determine if a
|
* partition. This is used by the state machine to determine if a
|
||||||
|
@ -132,12 +189,15 @@ public class CurrentAssignmentBuilder {
|
||||||
case STABLE:
|
case STABLE:
|
||||||
// When the member is in the STABLE state, we verify if a newer
|
// When the member is in the STABLE state, we verify if a newer
|
||||||
// epoch (or target assignment) is available. If it is, we can
|
// 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) {
|
if (member.memberEpoch() != targetAssignmentEpoch) {
|
||||||
return computeNextAssignment(
|
return computeNextAssignment(
|
||||||
member.memberEpoch(),
|
member.memberEpoch(),
|
||||||
member.assignedPartitions()
|
member.assignedPartitions()
|
||||||
);
|
);
|
||||||
|
} else if (hasSubscriptionChanged) {
|
||||||
|
return updateCurrentAssignment(member.assignedPartitions());
|
||||||
} else {
|
} else {
|
||||||
return member;
|
return member;
|
||||||
}
|
}
|
||||||
|
@ -147,18 +207,27 @@ public class CurrentAssignmentBuilder {
|
||||||
// until the member has revoked the necessary partitions. They are
|
// until the member has revoked the necessary partitions. They are
|
||||||
// considered revoked when they are not anymore reported in the
|
// considered revoked when they are not anymore reported in the
|
||||||
// owned partitions set in the ConsumerGroupHeartbeat API.
|
// 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
|
// If the member provides its owned partitions. We verify if it still
|
||||||
// owns any of the revoked partitions. If it does, we cannot progress.
|
// owns any of the revoked partitions. If it does, we cannot progress.
|
||||||
if (ownsRevokedPartitions(member.partitionsPendingRevocation())) {
|
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
|
// When the member has revoked all the pending partitions, it can
|
||||||
// transition to the next epoch (current + 1) and we can reconcile
|
// transition to the next epoch (current + 1) and we can reconcile
|
||||||
// its state towards the latest target assignment.
|
// its state towards the latest target assignment.
|
||||||
return computeNextAssignment(
|
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()
|
member.assignedPartitions()
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -215,6 +284,71 @@ public class CurrentAssignmentBuilder {
|
||||||
return false;
|
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.
|
* Computes the next assignment.
|
||||||
*
|
*
|
||||||
|
@ -227,6 +361,8 @@ public class CurrentAssignmentBuilder {
|
||||||
int memberEpoch,
|
int memberEpoch,
|
||||||
Map<Uuid, Set<Integer>> memberAssignedPartitions
|
Map<Uuid, Set<Integer>> memberAssignedPartitions
|
||||||
) {
|
) {
|
||||||
|
Set<Uuid> subscribedTopicIds = subscribedTopicIds();
|
||||||
|
|
||||||
boolean hasUnreleasedPartitions = false;
|
boolean hasUnreleasedPartitions = false;
|
||||||
Map<Uuid, Set<Integer>> newAssignedPartitions = new HashMap<>();
|
Map<Uuid, Set<Integer>> newAssignedPartitions = new HashMap<>();
|
||||||
Map<Uuid, Set<Integer>> newPartitionsPendingRevocation = new HashMap<>();
|
Map<Uuid, Set<Integer>> newPartitionsPendingRevocation = new HashMap<>();
|
||||||
|
@ -241,6 +377,11 @@ public class CurrentAssignmentBuilder {
|
||||||
Set<Integer> currentAssignedPartitions = memberAssignedPartitions
|
Set<Integer> currentAssignedPartitions = memberAssignedPartitions
|
||||||
.getOrDefault(topicId, Set.of());
|
.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
|
// New Assigned Partitions = Previous Assigned Partitions ∩ Target
|
||||||
Set<Integer> assignedPartitions = new HashSet<>(currentAssignedPartitions);
|
Set<Integer> assignedPartitions = new HashSet<>(currentAssignedPartitions);
|
||||||
assignedPartitions.retainAll(target);
|
assignedPartitions.retainAll(target);
|
||||||
|
@ -317,4 +458,28 @@ public class CurrentAssignmentBuilder {
|
||||||
.build();
|
.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);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -16,10 +16,16 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.coordinator.group.modern.share;
|
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.Assignment;
|
||||||
import org.apache.kafka.coordinator.group.modern.MemberState;
|
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.Objects;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The ShareGroupAssignmentBuilder class encapsulates the reconciliation engine of the
|
* The ShareGroupAssignmentBuilder class encapsulates the reconciliation engine of the
|
||||||
|
@ -32,6 +38,11 @@ public class ShareGroupAssignmentBuilder {
|
||||||
*/
|
*/
|
||||||
private final ShareGroupMember member;
|
private final ShareGroupMember member;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The metadata image.
|
||||||
|
*/
|
||||||
|
private CoordinatorMetadataImage metadataImage = CoordinatorMetadataImage.EMPTY;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The target assignment epoch.
|
* The target assignment epoch.
|
||||||
*/
|
*/
|
||||||
|
@ -42,6 +53,11 @@ public class ShareGroupAssignmentBuilder {
|
||||||
*/
|
*/
|
||||||
private Assignment targetAssignment;
|
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
|
* Constructs the ShareGroupAssignmentBuilder based on the current state of the
|
||||||
* provided share group member.
|
* provided share group member.
|
||||||
|
@ -52,6 +68,19 @@ public class ShareGroupAssignmentBuilder {
|
||||||
this.member = Objects.requireNonNull(member);
|
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
|
* Sets the target assignment epoch and the target assignment that the
|
||||||
* share group member must be reconciled to.
|
* share group member must be reconciled to.
|
||||||
|
@ -69,6 +98,19 @@ public class ShareGroupAssignmentBuilder {
|
||||||
return this;
|
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
|
* Builds the next state for the member or keep the current one if it
|
||||||
* is not possible to move forward with the current state.
|
* is not possible to move forward with the current state.
|
||||||
|
@ -83,11 +125,38 @@ public class ShareGroupAssignmentBuilder {
|
||||||
// when the member is updated.
|
// when the member is updated.
|
||||||
return new ShareGroupMember.Builder(member)
|
return new ShareGroupMember.Builder(member)
|
||||||
.setState(MemberState.STABLE)
|
.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)
|
.updateMemberEpoch(targetAssignmentEpoch)
|
||||||
.build();
|
.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;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -74,6 +74,7 @@ public class ShareGroupMember extends ModernGroupMember {
|
||||||
this.memberId = Objects.requireNonNull(newMemberId);
|
this.memberId = Objects.requireNonNull(newMemberId);
|
||||||
this.memberEpoch = member.memberEpoch;
|
this.memberEpoch = member.memberEpoch;
|
||||||
this.previousMemberEpoch = member.previousMemberEpoch;
|
this.previousMemberEpoch = member.previousMemberEpoch;
|
||||||
|
this.state = member.state;
|
||||||
this.rackId = member.rackId;
|
this.rackId = member.rackId;
|
||||||
this.clientId = member.clientId;
|
this.clientId = member.clientId;
|
||||||
this.clientHost = member.clientHost;
|
this.clientHost = member.clientHost;
|
||||||
|
|
|
@ -20604,7 +20604,7 @@ public class GroupMetadataManagerTest {
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
// Member 1 updates its new regular expression.
|
// Member 1 updates its new regular expression.
|
||||||
CoordinatorResult<ConsumerGroupHeartbeatResponseData, CoordinatorRecord> result = context.consumerGroupHeartbeat(
|
CoordinatorResult<ConsumerGroupHeartbeatResponseData, CoordinatorRecord> result1 = context.consumerGroupHeartbeat(
|
||||||
new ConsumerGroupHeartbeatRequestData()
|
new ConsumerGroupHeartbeatRequestData()
|
||||||
.setGroupId(groupId)
|
.setGroupId(groupId)
|
||||||
.setMemberId(memberId1)
|
.setMemberId(memberId1)
|
||||||
|
@ -20620,19 +20620,15 @@ public class GroupMetadataManagerTest {
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setHeartbeatIntervalMs(5000)
|
.setHeartbeatIntervalMs(5000)
|
||||||
.setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()
|
.setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()
|
||||||
.setTopicPartitions(List.of(
|
.setTopicPartitions(List.of())
|
||||||
new ConsumerGroupHeartbeatResponseData.TopicPartitions()
|
|
||||||
.setTopicId(fooTopicId)
|
|
||||||
.setPartitions(List.of(0, 1, 2, 3, 4, 5))
|
|
||||||
))
|
|
||||||
),
|
),
|
||||||
result.response()
|
result1.response()
|
||||||
);
|
);
|
||||||
|
|
||||||
ConsumerGroupMember expectedMember1 = new ConsumerGroupMember.Builder(memberId1)
|
ConsumerGroupMember expectedMember1 = new ConsumerGroupMember.Builder(memberId1)
|
||||||
.setState(MemberState.STABLE)
|
.setState(MemberState.STABLE)
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setPreviousMemberEpoch(0)
|
.setPreviousMemberEpoch(10)
|
||||||
.setClientId(DEFAULT_CLIENT_ID)
|
.setClientId(DEFAULT_CLIENT_ID)
|
||||||
.setClientHost(DEFAULT_CLIENT_ADDRESS.toString())
|
.setClientHost(DEFAULT_CLIENT_ADDRESS.toString())
|
||||||
.setRebalanceTimeoutMs(5000)
|
.setRebalanceTimeoutMs(5000)
|
||||||
|
@ -20644,10 +20640,12 @@ public class GroupMetadataManagerTest {
|
||||||
// The member subscription is updated.
|
// The member subscription is updated.
|
||||||
GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember1),
|
GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember1),
|
||||||
// The previous regular expression is deleted.
|
// 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.
|
// Execute pending tasks.
|
||||||
List<MockCoordinatorExecutor.ExecutorResult<CoordinatorRecord>> tasks = context.processTasks();
|
List<MockCoordinatorExecutor.ExecutorResult<CoordinatorRecord>> tasks = context.processTasks();
|
||||||
|
@ -20675,6 +20673,65 @@ public class GroupMetadataManagerTest {
|
||||||
),
|
),
|
||||||
task.result().records()
|
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
|
@Test
|
||||||
|
@ -21077,10 +21134,7 @@ public class GroupMetadataManagerTest {
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setHeartbeatIntervalMs(5000)
|
.setHeartbeatIntervalMs(5000)
|
||||||
.setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()
|
.setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()
|
||||||
.setTopicPartitions(List.of(
|
.setTopicPartitions(List.of())),
|
||||||
new ConsumerGroupHeartbeatResponseData.TopicPartitions()
|
|
||||||
.setTopicId(fooTopicId)
|
|
||||||
.setPartitions(List.of(3, 4, 5))))),
|
|
||||||
result1.response()
|
result1.response()
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -21098,7 +21152,8 @@ public class GroupMetadataManagerTest {
|
||||||
assertRecordsEquals(
|
assertRecordsEquals(
|
||||||
List.of(
|
List.of(
|
||||||
GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember2),
|
GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember2),
|
||||||
GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionTombstone(groupId, "foo*")
|
GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionTombstone(groupId, "foo*"),
|
||||||
|
GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedMember2)
|
||||||
),
|
),
|
||||||
result1.records()
|
result1.records()
|
||||||
);
|
);
|
||||||
|
@ -21164,8 +21219,7 @@ public class GroupMetadataManagerTest {
|
||||||
.setRebalanceTimeoutMs(5000)
|
.setRebalanceTimeoutMs(5000)
|
||||||
.setSubscribedTopicRegex("foo|bar*")
|
.setSubscribedTopicRegex("foo|bar*")
|
||||||
.setServerAssignorName("range")
|
.setServerAssignorName("range")
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment())
|
||||||
mkTopicAssignment(fooTopicId, 3, 4, 5)))
|
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
assertResponseEquals(
|
assertResponseEquals(
|
||||||
|
@ -21174,10 +21228,7 @@ public class GroupMetadataManagerTest {
|
||||||
.setMemberEpoch(11)
|
.setMemberEpoch(11)
|
||||||
.setHeartbeatIntervalMs(5000)
|
.setHeartbeatIntervalMs(5000)
|
||||||
.setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()
|
.setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()
|
||||||
.setTopicPartitions(List.of(
|
.setTopicPartitions(List.of())),
|
||||||
new ConsumerGroupHeartbeatResponseData.TopicPartitions()
|
|
||||||
.setTopicId(fooTopicId)
|
|
||||||
.setPartitions(List.of(3, 4, 5))))),
|
|
||||||
result2.response()
|
result2.response()
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -21306,10 +21357,7 @@ public class GroupMetadataManagerTest {
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setHeartbeatIntervalMs(5000)
|
.setHeartbeatIntervalMs(5000)
|
||||||
.setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()
|
.setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()
|
||||||
.setTopicPartitions(List.of(
|
.setTopicPartitions(List.of())),
|
||||||
new ConsumerGroupHeartbeatResponseData.TopicPartitions()
|
|
||||||
.setTopicId(fooTopicId)
|
|
||||||
.setPartitions(List.of(3, 4, 5))))),
|
|
||||||
result1.response()
|
result1.response()
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -21327,7 +21375,8 @@ public class GroupMetadataManagerTest {
|
||||||
assertRecordsEquals(
|
assertRecordsEquals(
|
||||||
List.of(
|
List.of(
|
||||||
GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember2),
|
GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember2),
|
||||||
GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionTombstone(groupId, "foo*")
|
GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionTombstone(groupId, "foo*"),
|
||||||
|
GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedMember2)
|
||||||
),
|
),
|
||||||
result1.records()
|
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
|
@Test
|
||||||
public void testResolvedRegularExpressionsRemovedWhenMembersLeaveOrFenced() {
|
public void testResolvedRegularExpressionsRemovedWhenMembersLeaveOrFenced() {
|
||||||
String groupId = "fooup";
|
String groupId = "fooup";
|
||||||
|
|
|
@ -19,13 +19,19 @@ package org.apache.kafka.coordinator.group.modern.consumer;
|
||||||
import org.apache.kafka.common.Uuid;
|
import org.apache.kafka.common.Uuid;
|
||||||
import org.apache.kafka.common.errors.FencedMemberEpochException;
|
import org.apache.kafka.common.errors.FencedMemberEpochException;
|
||||||
import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
|
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.Assignment;
|
||||||
import org.apache.kafka.coordinator.group.modern.MemberState;
|
import org.apache.kafka.coordinator.group.modern.MemberState;
|
||||||
|
|
||||||
import org.junit.jupiter.api.Test;
|
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.Arrays;
|
||||||
import java.util.List;
|
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.mkAssignment;
|
||||||
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
|
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
|
||||||
|
@ -36,19 +42,28 @@ public class CurrentAssignmentBuilderTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testStableToStable() {
|
public void testStableToStable() {
|
||||||
|
String topic1 = "topic1";
|
||||||
|
String topic2 = "topic2";
|
||||||
Uuid topicId1 = Uuid.randomUuid();
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
Uuid topicId2 = 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")
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
.setState(MemberState.STABLE)
|
.setState(MemberState.STABLE)
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||||
|
.withMetadataImage(metadataImage)
|
||||||
.withTargetAssignment(11, new Assignment(mkAssignment(
|
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6))))
|
mkTopicAssignment(topicId2, 4, 5, 6))))
|
||||||
|
@ -60,6 +75,7 @@ public class CurrentAssignmentBuilderTest {
|
||||||
.setState(MemberState.STABLE)
|
.setState(MemberState.STABLE)
|
||||||
.setMemberEpoch(11)
|
.setMemberEpoch(11)
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||||
|
@ -70,19 +86,28 @@ public class CurrentAssignmentBuilderTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testStableToStableWithNewPartitions() {
|
public void testStableToStableWithNewPartitions() {
|
||||||
|
String topic1 = "topic1";
|
||||||
|
String topic2 = "topic2";
|
||||||
Uuid topicId1 = Uuid.randomUuid();
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
Uuid topicId2 = 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")
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
.setState(MemberState.STABLE)
|
.setState(MemberState.STABLE)
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||||
|
.withMetadataImage(metadataImage)
|
||||||
.withTargetAssignment(11, new Assignment(mkAssignment(
|
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3, 4),
|
mkTopicAssignment(topicId1, 1, 2, 3, 4),
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6, 7))))
|
mkTopicAssignment(topicId2, 4, 5, 6, 7))))
|
||||||
|
@ -94,6 +119,7 @@ public class CurrentAssignmentBuilderTest {
|
||||||
.setState(MemberState.STABLE)
|
.setState(MemberState.STABLE)
|
||||||
.setMemberEpoch(11)
|
.setMemberEpoch(11)
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3, 4),
|
mkTopicAssignment(topicId1, 1, 2, 3, 4),
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6, 7)))
|
mkTopicAssignment(topicId2, 4, 5, 6, 7)))
|
||||||
|
@ -104,19 +130,28 @@ public class CurrentAssignmentBuilderTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testStableToUnrevokedPartitions() {
|
public void testStableToUnrevokedPartitions() {
|
||||||
|
String topic1 = "topic1";
|
||||||
|
String topic2 = "topic2";
|
||||||
Uuid topicId1 = Uuid.randomUuid();
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
Uuid topicId2 = 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")
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
.setState(MemberState.STABLE)
|
.setState(MemberState.STABLE)
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||||
|
.withMetadataImage(metadataImage)
|
||||||
.withTargetAssignment(11, new Assignment(mkAssignment(
|
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 2, 3, 4),
|
mkTopicAssignment(topicId1, 2, 3, 4),
|
||||||
mkTopicAssignment(topicId2, 5, 6, 7))))
|
mkTopicAssignment(topicId2, 5, 6, 7))))
|
||||||
|
@ -128,6 +163,7 @@ public class CurrentAssignmentBuilderTest {
|
||||||
.setState(MemberState.UNREVOKED_PARTITIONS)
|
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 2, 3),
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 5, 6)))
|
mkTopicAssignment(topicId2, 5, 6)))
|
||||||
|
@ -141,19 +177,28 @@ public class CurrentAssignmentBuilderTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testStableToUnreleasedPartitions() {
|
public void testStableToUnreleasedPartitions() {
|
||||||
|
String topic1 = "topic1";
|
||||||
|
String topic2 = "topic2";
|
||||||
Uuid topicId1 = Uuid.randomUuid();
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
Uuid topicId2 = 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")
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
.setState(MemberState.STABLE)
|
.setState(MemberState.STABLE)
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||||
|
.withMetadataImage(metadataImage)
|
||||||
.withTargetAssignment(11, new Assignment(mkAssignment(
|
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3, 4),
|
mkTopicAssignment(topicId1, 1, 2, 3, 4),
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6, 7))))
|
mkTopicAssignment(topicId2, 4, 5, 6, 7))))
|
||||||
|
@ -165,6 +210,7 @@ public class CurrentAssignmentBuilderTest {
|
||||||
.setState(MemberState.UNRELEASED_PARTITIONS)
|
.setState(MemberState.UNRELEASED_PARTITIONS)
|
||||||
.setMemberEpoch(11)
|
.setMemberEpoch(11)
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||||
|
@ -175,19 +221,28 @@ public class CurrentAssignmentBuilderTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testStableToUnreleasedPartitionsWithOwnedPartitionsNotHavingRevokedPartitions() {
|
public void testStableToUnreleasedPartitionsWithOwnedPartitionsNotHavingRevokedPartitions() {
|
||||||
|
String topic1 = "topic1";
|
||||||
|
String topic2 = "topic2";
|
||||||
Uuid topicId1 = Uuid.randomUuid();
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
Uuid topicId2 = 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")
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
.setState(MemberState.STABLE)
|
.setState(MemberState.STABLE)
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||||
|
.withMetadataImage(metadataImage)
|
||||||
.withTargetAssignment(11, new Assignment(mkAssignment(
|
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 4, 5, 7))))
|
mkTopicAssignment(topicId2, 4, 5, 7))))
|
||||||
|
@ -202,6 +257,7 @@ public class CurrentAssignmentBuilderTest {
|
||||||
.setState(MemberState.UNRELEASED_PARTITIONS)
|
.setState(MemberState.UNRELEASED_PARTITIONS)
|
||||||
.setMemberEpoch(11)
|
.setMemberEpoch(11)
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 4, 5)))
|
mkTopicAssignment(topicId2, 4, 5)))
|
||||||
|
@ -212,13 +268,21 @@ public class CurrentAssignmentBuilderTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testUnrevokedPartitionsToStable() {
|
public void testUnrevokedPartitionsToStable() {
|
||||||
|
String topic1 = "topic1";
|
||||||
|
String topic2 = "topic2";
|
||||||
Uuid topicId1 = Uuid.randomUuid();
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
Uuid topicId2 = 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")
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
.setState(MemberState.UNREVOKED_PARTITIONS)
|
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 2, 3),
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 5, 6)))
|
mkTopicAssignment(topicId2, 5, 6)))
|
||||||
|
@ -228,6 +292,7 @@ public class CurrentAssignmentBuilderTest {
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||||
|
.withMetadataImage(metadataImage)
|
||||||
.withTargetAssignment(11, new Assignment(mkAssignment(
|
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 2, 3),
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 5, 6))))
|
mkTopicAssignment(topicId2, 5, 6))))
|
||||||
|
@ -246,6 +311,7 @@ public class CurrentAssignmentBuilderTest {
|
||||||
.setState(MemberState.STABLE)
|
.setState(MemberState.STABLE)
|
||||||
.setMemberEpoch(11)
|
.setMemberEpoch(11)
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 2, 3),
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 5, 6)))
|
mkTopicAssignment(topicId2, 5, 6)))
|
||||||
|
@ -256,13 +322,21 @@ public class CurrentAssignmentBuilderTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testRemainsInUnrevokedPartitions() {
|
public void testRemainsInUnrevokedPartitions() {
|
||||||
|
String topic1 = "topic1";
|
||||||
|
String topic2 = "topic2";
|
||||||
Uuid topicId1 = Uuid.randomUuid();
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
Uuid topicId2 = 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")
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
.setState(MemberState.UNREVOKED_PARTITIONS)
|
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 2, 3),
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 5, 6)))
|
mkTopicAssignment(topicId2, 5, 6)))
|
||||||
|
@ -272,6 +346,7 @@ public class CurrentAssignmentBuilderTest {
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
CurrentAssignmentBuilder currentAssignmentBuilder = new CurrentAssignmentBuilder(member)
|
CurrentAssignmentBuilder currentAssignmentBuilder = new CurrentAssignmentBuilder(member)
|
||||||
|
.withMetadataImage(metadataImage)
|
||||||
.withTargetAssignment(12, new Assignment(mkAssignment(
|
.withTargetAssignment(12, new Assignment(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 3),
|
mkTopicAssignment(topicId1, 3),
|
||||||
mkTopicAssignment(topicId2, 6))))
|
mkTopicAssignment(topicId2, 6))))
|
||||||
|
@ -311,15 +386,27 @@ public class CurrentAssignmentBuilderTest {
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@ParameterizedTest
|
||||||
public void testUnrevokedPartitionsToUnrevokedPartitions() {
|
@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 topicId1 = Uuid.randomUuid();
|
||||||
Uuid topicId2 = 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")
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
.setState(MemberState.UNREVOKED_PARTITIONS)
|
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(memberEpoch)
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(memberEpoch)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 2, 3),
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 5, 6)))
|
mkTopicAssignment(topicId2, 5, 6)))
|
||||||
|
@ -329,7 +416,8 @@ public class CurrentAssignmentBuilderTest {
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||||
.withTargetAssignment(12, new Assignment(mkAssignment(
|
.withMetadataImage(metadataImage)
|
||||||
|
.withTargetAssignment(targetAssignmentEpoch, new Assignment(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 3),
|
mkTopicAssignment(topicId1, 3),
|
||||||
mkTopicAssignment(topicId2, 6))))
|
mkTopicAssignment(topicId2, 6))))
|
||||||
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
|
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
|
||||||
|
@ -345,8 +433,9 @@ public class CurrentAssignmentBuilderTest {
|
||||||
assertEquals(
|
assertEquals(
|
||||||
new ConsumerGroupMember.Builder("member")
|
new ConsumerGroupMember.Builder("member")
|
||||||
.setState(MemberState.UNREVOKED_PARTITIONS)
|
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||||
.setMemberEpoch(11)
|
.setMemberEpoch(expectedMemberEpoch)
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(memberEpoch)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 3),
|
mkTopicAssignment(topicId1, 3),
|
||||||
mkTopicAssignment(topicId2, 6)))
|
mkTopicAssignment(topicId2, 6)))
|
||||||
|
@ -360,19 +449,28 @@ public class CurrentAssignmentBuilderTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testUnrevokedPartitionsToUnreleasedPartitions() {
|
public void testUnrevokedPartitionsToUnreleasedPartitions() {
|
||||||
|
String topic1 = "topic1";
|
||||||
|
String topic2 = "topic2";
|
||||||
Uuid topicId1 = Uuid.randomUuid();
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
Uuid topicId2 = 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")
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
.setState(MemberState.UNREVOKED_PARTITIONS)
|
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||||
.setMemberEpoch(11)
|
.setMemberEpoch(11)
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 2, 3),
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 5, 6)))
|
mkTopicAssignment(topicId2, 5, 6)))
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||||
|
.withMetadataImage(metadataImage)
|
||||||
.withTargetAssignment(11, new Assignment(mkAssignment(
|
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 2, 3, 4),
|
mkTopicAssignment(topicId1, 2, 3, 4),
|
||||||
mkTopicAssignment(topicId2, 5, 6, 7))))
|
mkTopicAssignment(topicId2, 5, 6, 7))))
|
||||||
|
@ -391,6 +489,7 @@ public class CurrentAssignmentBuilderTest {
|
||||||
.setState(MemberState.UNRELEASED_PARTITIONS)
|
.setState(MemberState.UNRELEASED_PARTITIONS)
|
||||||
.setMemberEpoch(11)
|
.setMemberEpoch(11)
|
||||||
.setPreviousMemberEpoch(11)
|
.setPreviousMemberEpoch(11)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 2, 3),
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 5, 6)))
|
mkTopicAssignment(topicId2, 5, 6)))
|
||||||
|
@ -401,19 +500,28 @@ public class CurrentAssignmentBuilderTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testUnreleasedPartitionsToStable() {
|
public void testUnreleasedPartitionsToStable() {
|
||||||
|
String topic1 = "topic1";
|
||||||
|
String topic2 = "topic2";
|
||||||
Uuid topicId1 = Uuid.randomUuid();
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
Uuid topicId2 = 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")
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
.setState(MemberState.UNRELEASED_PARTITIONS)
|
.setState(MemberState.UNRELEASED_PARTITIONS)
|
||||||
.setMemberEpoch(11)
|
.setMemberEpoch(11)
|
||||||
.setPreviousMemberEpoch(11)
|
.setPreviousMemberEpoch(11)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 2, 3),
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 5, 6)))
|
mkTopicAssignment(topicId2, 5, 6)))
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||||
|
.withMetadataImage(metadataImage)
|
||||||
.withTargetAssignment(12, new Assignment(mkAssignment(
|
.withTargetAssignment(12, new Assignment(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 2, 3),
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 5, 6))))
|
mkTopicAssignment(topicId2, 5, 6))))
|
||||||
|
@ -425,6 +533,7 @@ public class CurrentAssignmentBuilderTest {
|
||||||
.setState(MemberState.STABLE)
|
.setState(MemberState.STABLE)
|
||||||
.setMemberEpoch(12)
|
.setMemberEpoch(12)
|
||||||
.setPreviousMemberEpoch(11)
|
.setPreviousMemberEpoch(11)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 2, 3),
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 5, 6)))
|
mkTopicAssignment(topicId2, 5, 6)))
|
||||||
|
@ -435,19 +544,28 @@ public class CurrentAssignmentBuilderTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testUnreleasedPartitionsToStableWithNewPartitions() {
|
public void testUnreleasedPartitionsToStableWithNewPartitions() {
|
||||||
|
String topic1 = "topic1";
|
||||||
|
String topic2 = "topic2";
|
||||||
Uuid topicId1 = Uuid.randomUuid();
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
Uuid topicId2 = 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")
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
.setState(MemberState.UNRELEASED_PARTITIONS)
|
.setState(MemberState.UNRELEASED_PARTITIONS)
|
||||||
.setMemberEpoch(11)
|
.setMemberEpoch(11)
|
||||||
.setPreviousMemberEpoch(11)
|
.setPreviousMemberEpoch(11)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 2, 3),
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 5, 6)))
|
mkTopicAssignment(topicId2, 5, 6)))
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||||
|
.withMetadataImage(metadataImage)
|
||||||
.withTargetAssignment(11, new Assignment(mkAssignment(
|
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 2, 3, 4),
|
mkTopicAssignment(topicId1, 2, 3, 4),
|
||||||
mkTopicAssignment(topicId2, 5, 6, 7))))
|
mkTopicAssignment(topicId2, 5, 6, 7))))
|
||||||
|
@ -459,6 +577,7 @@ public class CurrentAssignmentBuilderTest {
|
||||||
.setState(MemberState.STABLE)
|
.setState(MemberState.STABLE)
|
||||||
.setMemberEpoch(11)
|
.setMemberEpoch(11)
|
||||||
.setPreviousMemberEpoch(11)
|
.setPreviousMemberEpoch(11)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 2, 3, 4),
|
mkTopicAssignment(topicId1, 2, 3, 4),
|
||||||
mkTopicAssignment(topicId2, 5, 6, 7)))
|
mkTopicAssignment(topicId2, 5, 6, 7)))
|
||||||
|
@ -469,19 +588,28 @@ public class CurrentAssignmentBuilderTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testUnreleasedPartitionsToUnreleasedPartitions() {
|
public void testUnreleasedPartitionsToUnreleasedPartitions() {
|
||||||
|
String topic1 = "topic1";
|
||||||
|
String topic2 = "topic2";
|
||||||
Uuid topicId1 = Uuid.randomUuid();
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
Uuid topicId2 = 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")
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
.setState(MemberState.UNRELEASED_PARTITIONS)
|
.setState(MemberState.UNRELEASED_PARTITIONS)
|
||||||
.setMemberEpoch(11)
|
.setMemberEpoch(11)
|
||||||
.setPreviousMemberEpoch(11)
|
.setPreviousMemberEpoch(11)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 2, 3),
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 5, 6)))
|
mkTopicAssignment(topicId2, 5, 6)))
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||||
|
.withMetadataImage(metadataImage)
|
||||||
.withTargetAssignment(11, new Assignment(mkAssignment(
|
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 2, 3, 4),
|
mkTopicAssignment(topicId1, 2, 3, 4),
|
||||||
mkTopicAssignment(topicId2, 5, 6, 7))))
|
mkTopicAssignment(topicId2, 5, 6, 7))))
|
||||||
|
@ -493,19 +621,28 @@ public class CurrentAssignmentBuilderTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testUnreleasedPartitionsToUnrevokedPartitions() {
|
public void testUnreleasedPartitionsToUnrevokedPartitions() {
|
||||||
|
String topic1 = "topic1";
|
||||||
|
String topic2 = "topic2";
|
||||||
Uuid topicId1 = Uuid.randomUuid();
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
Uuid topicId2 = 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")
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
.setState(MemberState.UNRELEASED_PARTITIONS)
|
.setState(MemberState.UNRELEASED_PARTITIONS)
|
||||||
.setMemberEpoch(11)
|
.setMemberEpoch(11)
|
||||||
.setPreviousMemberEpoch(11)
|
.setPreviousMemberEpoch(11)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 2, 3),
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 5, 6)))
|
mkTopicAssignment(topicId2, 5, 6)))
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||||
|
.withMetadataImage(metadataImage)
|
||||||
.withTargetAssignment(12, new Assignment(mkAssignment(
|
.withTargetAssignment(12, new Assignment(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 3),
|
mkTopicAssignment(topicId1, 3),
|
||||||
mkTopicAssignment(topicId2, 6))))
|
mkTopicAssignment(topicId2, 6))))
|
||||||
|
@ -517,6 +654,7 @@ public class CurrentAssignmentBuilderTest {
|
||||||
.setState(MemberState.UNREVOKED_PARTITIONS)
|
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||||
.setMemberEpoch(11)
|
.setMemberEpoch(11)
|
||||||
.setPreviousMemberEpoch(11)
|
.setPreviousMemberEpoch(11)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 3),
|
mkTopicAssignment(topicId1, 3),
|
||||||
mkTopicAssignment(topicId2, 6)))
|
mkTopicAssignment(topicId2, 6)))
|
||||||
|
@ -530,13 +668,21 @@ public class CurrentAssignmentBuilderTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testUnknownState() {
|
public void testUnknownState() {
|
||||||
|
String topic1 = "topic1";
|
||||||
|
String topic2 = "topic2";
|
||||||
Uuid topicId1 = Uuid.randomUuid();
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
Uuid topicId2 = 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")
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
.setState(MemberState.UNKNOWN)
|
.setState(MemberState.UNKNOWN)
|
||||||
.setMemberEpoch(11)
|
.setMemberEpoch(11)
|
||||||
.setPreviousMemberEpoch(11)
|
.setPreviousMemberEpoch(11)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 3),
|
mkTopicAssignment(topicId1, 3),
|
||||||
mkTopicAssignment(topicId2, 6)))
|
mkTopicAssignment(topicId2, 6)))
|
||||||
|
@ -548,6 +694,7 @@ public class CurrentAssignmentBuilderTest {
|
||||||
// When the member is in an unknown state, the member is first to force
|
// When the member is in an unknown state, the member is first to force
|
||||||
// a reset of the client side member state.
|
// a reset of the client side member state.
|
||||||
assertThrows(FencedMemberEpochException.class, () -> new CurrentAssignmentBuilder(member)
|
assertThrows(FencedMemberEpochException.class, () -> new CurrentAssignmentBuilder(member)
|
||||||
|
.withMetadataImage(metadataImage)
|
||||||
.withTargetAssignment(12, new Assignment(mkAssignment(
|
.withTargetAssignment(12, new Assignment(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 3),
|
mkTopicAssignment(topicId1, 3),
|
||||||
mkTopicAssignment(topicId2, 6))))
|
mkTopicAssignment(topicId2, 6))))
|
||||||
|
@ -556,6 +703,7 @@ public class CurrentAssignmentBuilderTest {
|
||||||
|
|
||||||
// Then the member rejoins with no owned partitions.
|
// Then the member rejoins with no owned partitions.
|
||||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||||
|
.withMetadataImage(metadataImage)
|
||||||
.withTargetAssignment(12, new Assignment(mkAssignment(
|
.withTargetAssignment(12, new Assignment(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 3),
|
mkTopicAssignment(topicId1, 3),
|
||||||
mkTopicAssignment(topicId2, 6))))
|
mkTopicAssignment(topicId2, 6))))
|
||||||
|
@ -568,6 +716,7 @@ public class CurrentAssignmentBuilderTest {
|
||||||
.setState(MemberState.STABLE)
|
.setState(MemberState.STABLE)
|
||||||
.setMemberEpoch(12)
|
.setMemberEpoch(12)
|
||||||
.setPreviousMemberEpoch(11)
|
.setPreviousMemberEpoch(11)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 3),
|
mkTopicAssignment(topicId1, 3),
|
||||||
mkTopicAssignment(topicId2, 6)))
|
mkTopicAssignment(topicId2, 6)))
|
||||||
|
@ -575,4 +724,355 @@ public class CurrentAssignmentBuilderTest {
|
||||||
updatedMember
|
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
|
||||||
|
);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -17,10 +17,16 @@
|
||||||
package org.apache.kafka.coordinator.group.modern.share;
|
package org.apache.kafka.coordinator.group.modern.share;
|
||||||
|
|
||||||
import org.apache.kafka.common.Uuid;
|
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.Assignment;
|
||||||
import org.apache.kafka.coordinator.group.modern.MemberState;
|
import org.apache.kafka.coordinator.group.modern.MemberState;
|
||||||
|
|
||||||
import org.junit.jupiter.api.Test;
|
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.mkAssignment;
|
||||||
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
|
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
|
||||||
|
@ -30,19 +36,28 @@ public class ShareGroupAssignmentBuilderTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testStableToStable() {
|
public void testStableToStable() {
|
||||||
|
String topic1 = "topic1";
|
||||||
|
String topic2 = "topic2";
|
||||||
Uuid topicId1 = Uuid.randomUuid();
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
Uuid topicId2 = 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")
|
ShareGroupMember member = new ShareGroupMember.Builder("member")
|
||||||
.setState(MemberState.STABLE)
|
.setState(MemberState.STABLE)
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
ShareGroupMember updatedMember = new ShareGroupAssignmentBuilder(member)
|
ShareGroupMember updatedMember = new ShareGroupAssignmentBuilder(member)
|
||||||
|
.withMetadataImage(metadataImage)
|
||||||
.withTargetAssignment(11, new Assignment(mkAssignment(
|
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6))))
|
mkTopicAssignment(topicId2, 4, 5, 6))))
|
||||||
|
@ -53,6 +68,7 @@ public class ShareGroupAssignmentBuilderTest {
|
||||||
.setState(MemberState.STABLE)
|
.setState(MemberState.STABLE)
|
||||||
.setMemberEpoch(11)
|
.setMemberEpoch(11)
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||||
|
@ -63,19 +79,28 @@ public class ShareGroupAssignmentBuilderTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testStableToStableWithNewPartitions() {
|
public void testStableToStableWithNewPartitions() {
|
||||||
|
String topic1 = "topic1";
|
||||||
|
String topic2 = "topic2";
|
||||||
Uuid topicId1 = Uuid.randomUuid();
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
Uuid topicId2 = 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")
|
ShareGroupMember member = new ShareGroupMember.Builder("member")
|
||||||
.setState(MemberState.STABLE)
|
.setState(MemberState.STABLE)
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
ShareGroupMember updatedMember = new ShareGroupAssignmentBuilder(member)
|
ShareGroupMember updatedMember = new ShareGroupAssignmentBuilder(member)
|
||||||
|
.withMetadataImage(metadataImage)
|
||||||
.withTargetAssignment(11, new Assignment(mkAssignment(
|
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3, 4),
|
mkTopicAssignment(topicId1, 1, 2, 3, 4),
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6, 7))))
|
mkTopicAssignment(topicId2, 4, 5, 6, 7))))
|
||||||
|
@ -86,6 +111,7 @@ public class ShareGroupAssignmentBuilderTest {
|
||||||
.setState(MemberState.STABLE)
|
.setState(MemberState.STABLE)
|
||||||
.setMemberEpoch(11)
|
.setMemberEpoch(11)
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
|
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3, 4),
|
mkTopicAssignment(topicId1, 1, 2, 3, 4),
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6, 7)))
|
mkTopicAssignment(topicId2, 4, 5, 6, 7)))
|
||||||
|
@ -93,4 +119,56 @@ public class ShareGroupAssignmentBuilderTest {
|
||||||
updatedMember
|
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
|
||||||
|
);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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();
|
||||||
|
}
|
||||||
|
}
|
|
@ -494,12 +494,10 @@ public class PartitionChangeBuilder {
|
||||||
|
|
||||||
private void maybeUpdateLastKnownLeader(PartitionChangeRecord record) {
|
private void maybeUpdateLastKnownLeader(PartitionChangeRecord record) {
|
||||||
if (!useLastKnownLeaderInBalancedRecovery || !eligibleLeaderReplicasEnabled) return;
|
if (!useLastKnownLeaderInBalancedRecovery || !eligibleLeaderReplicasEnabled) return;
|
||||||
if (record.isr() != null && record.isr().isEmpty() && (partition.lastKnownElr.length != 1 ||
|
if (record.leader() == NO_LEADER && partition.lastKnownElr.length == 0) {
|
||||||
partition.lastKnownElr[0] != partition.leader)) {
|
|
||||||
// Only update the last known leader when the first time the partition becomes leaderless.
|
// Only update the last known leader when the first time the partition becomes leaderless.
|
||||||
record.setLastKnownElr(List.of(partition.leader));
|
record.setLastKnownElr(List.of(partition.leader));
|
||||||
} else if ((record.leader() >= 0 || (partition.leader != NO_LEADER && record.leader() != NO_LEADER))
|
} else if (record.leader() >= 0 && partition.lastKnownElr.length > 0) {
|
||||||
&& partition.lastKnownElr.length > 0) {
|
|
||||||
// Clear the LastKnownElr field if the partition will have or continues to have a valid leader.
|
// Clear the LastKnownElr field if the partition will have or continues to have a valid leader.
|
||||||
record.setLastKnownElr(List.of());
|
record.setLastKnownElr(List.of());
|
||||||
}
|
}
|
||||||
|
|
|
@ -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
|
@ParameterizedTest
|
||||||
@MethodSource("partitionChangeRecordVersions")
|
@MethodSource("partitionChangeRecordVersions")
|
||||||
public void testEligibleLeaderReplicas_IsrExpandAboveMinISR(short version) {
|
public void testEligibleLeaderReplicas_IsrExpandAboveMinISR(short version) {
|
||||||
|
|
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
|
@ -15,13 +15,16 @@
|
||||||
* limitations under the License.
|
* 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.Admin;
|
||||||
import org.apache.kafka.clients.admin.FeatureMetadata;
|
import org.apache.kafka.clients.admin.FeatureMetadata;
|
||||||
import org.apache.kafka.clients.admin.QuorumInfo;
|
import org.apache.kafka.clients.admin.QuorumInfo;
|
||||||
import org.apache.kafka.clients.admin.RaftVoterEndpoint;
|
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.Uuid;
|
||||||
|
import org.apache.kafka.common.errors.InconsistentClusterIdException;
|
||||||
import org.apache.kafka.common.test.KafkaClusterTestKit;
|
import org.apache.kafka.common.test.KafkaClusterTestKit;
|
||||||
import org.apache.kafka.common.test.TestKitNodes;
|
import org.apache.kafka.common.test.TestKitNodes;
|
||||||
import org.apache.kafka.common.test.api.TestKitDefaults;
|
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.server.common.KRaftVersion;
|
||||||
import org.apache.kafka.test.TestUtils;
|
import org.apache.kafka.test.TestUtils;
|
||||||
|
|
||||||
|
import org.junit.jupiter.api.Tag;
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
|
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Optional;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.TreeMap;
|
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.assertNotEquals;
|
||||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||||
|
|
||||||
|
@Tag("integration")
|
||||||
public class ReconfigurableQuorumIntegrationTest {
|
public class ReconfigurableQuorumIntegrationTest {
|
||||||
static void checkKRaftVersions(Admin admin, short finalized) throws Exception {
|
static void checkKRaftVersions(Admin admin, short finalized) throws Exception {
|
||||||
FeatureMetadata featureMetadata = admin.describeFeatures().featureMetadata().get();
|
FeatureMetadata featureMetadata = admin.describeFeatures().featureMetadata().get();
|
||||||
|
@ -70,7 +76,7 @@ public class ReconfigurableQuorumIntegrationTest {
|
||||||
).build()) {
|
).build()) {
|
||||||
cluster.format();
|
cluster.format();
|
||||||
cluster.startup();
|
cluster.startup();
|
||||||
try (Admin admin = Admin.create(cluster.clientProperties())) {
|
try (var admin = Admin.create(cluster.clientProperties())) {
|
||||||
TestUtils.retryOnExceptionWithTimeout(30_000, () -> {
|
TestUtils.retryOnExceptionWithTimeout(30_000, () -> {
|
||||||
checkKRaftVersions(admin, KRaftVersion.KRAFT_VERSION_0.featureLevel());
|
checkKRaftVersions(admin, KRaftVersion.KRAFT_VERSION_0.featureLevel());
|
||||||
});
|
});
|
||||||
|
@ -88,7 +94,7 @@ public class ReconfigurableQuorumIntegrationTest {
|
||||||
).setStandalone(true).build()) {
|
).setStandalone(true).build()) {
|
||||||
cluster.format();
|
cluster.format();
|
||||||
cluster.startup();
|
cluster.startup();
|
||||||
try (Admin admin = Admin.create(cluster.clientProperties())) {
|
try (var admin = Admin.create(cluster.clientProperties())) {
|
||||||
TestUtils.retryOnExceptionWithTimeout(30_000, () -> {
|
TestUtils.retryOnExceptionWithTimeout(30_000, () -> {
|
||||||
checkKRaftVersions(admin, KRaftVersion.KRAFT_VERSION_1.featureLevel());
|
checkKRaftVersions(admin, KRaftVersion.KRAFT_VERSION_1.featureLevel());
|
||||||
});
|
});
|
||||||
|
@ -126,7 +132,7 @@ public class ReconfigurableQuorumIntegrationTest {
|
||||||
) {
|
) {
|
||||||
cluster.format();
|
cluster.format();
|
||||||
cluster.startup();
|
cluster.startup();
|
||||||
try (Admin admin = Admin.create(cluster.clientProperties())) {
|
try (var admin = Admin.create(cluster.clientProperties())) {
|
||||||
TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> {
|
TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> {
|
||||||
Map<Integer, Uuid> voters = findVoterDirs(admin);
|
Map<Integer, Uuid> voters = findVoterDirs(admin);
|
||||||
assertEquals(Set.of(3000, 3001, 3002), voters.keySet());
|
assertEquals(Set.of(3000, 3001, 3002), voters.keySet());
|
||||||
|
@ -161,7 +167,7 @@ public class ReconfigurableQuorumIntegrationTest {
|
||||||
) {
|
) {
|
||||||
cluster.format();
|
cluster.format();
|
||||||
cluster.startup();
|
cluster.startup();
|
||||||
try (Admin admin = Admin.create(cluster.clientProperties())) {
|
try (var admin = Admin.create(cluster.clientProperties())) {
|
||||||
TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> {
|
TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> {
|
||||||
Map<Integer, Uuid> voters = findVoterDirs(admin);
|
Map<Integer, Uuid> voters = findVoterDirs(admin);
|
||||||
assertEquals(Set.of(3000, 3001, 3002, 3003), voters.keySet());
|
assertEquals(Set.of(3000, 3001, 3002, 3003), voters.keySet());
|
||||||
|
@ -200,7 +206,7 @@ public class ReconfigurableQuorumIntegrationTest {
|
||||||
) {
|
) {
|
||||||
cluster.format();
|
cluster.format();
|
||||||
cluster.startup();
|
cluster.startup();
|
||||||
try (Admin admin = Admin.create(cluster.clientProperties())) {
|
try (var admin = Admin.create(cluster.clientProperties())) {
|
||||||
TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> {
|
TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> {
|
||||||
Map<Integer, Uuid> voters = findVoterDirs(admin);
|
Map<Integer, Uuid> voters = findVoterDirs(admin);
|
||||||
assertEquals(Set.of(3000, 3001, 3002), voters.keySet());
|
assertEquals(Set.of(3000, 3001, 3002), voters.keySet());
|
||||||
|
@ -238,7 +244,7 @@ public class ReconfigurableQuorumIntegrationTest {
|
||||||
) {
|
) {
|
||||||
cluster.format();
|
cluster.format();
|
||||||
cluster.startup();
|
cluster.startup();
|
||||||
try (Admin admin = Admin.create(cluster.clientProperties())) {
|
try (var admin = Admin.create(cluster.clientProperties())) {
|
||||||
TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> {
|
TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> {
|
||||||
Map<Integer, Uuid> voters = findVoterDirs(admin);
|
Map<Integer, Uuid> voters = findVoterDirs(admin);
|
||||||
assertEquals(Set.of(3000, 3001, 3002), voters.keySet());
|
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);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -72,9 +72,10 @@ import org.junit.jupiter.api.AfterAll;
|
||||||
import org.junit.jupiter.api.BeforeAll;
|
import org.junit.jupiter.api.BeforeAll;
|
||||||
import org.junit.jupiter.api.BeforeEach;
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
import org.junit.jupiter.api.Tag;
|
import org.junit.jupiter.api.Tag;
|
||||||
import org.junit.jupiter.api.Test;
|
|
||||||
import org.junit.jupiter.api.Timeout;
|
import org.junit.jupiter.api.Timeout;
|
||||||
import org.junit.jupiter.params.ParameterizedTest;
|
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.junit.jupiter.params.provider.ValueSource;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
@ -169,6 +170,15 @@ public class EosIntegrationTest {
|
||||||
|
|
||||||
private String stateTmpDir;
|
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
|
@BeforeEach
|
||||||
public void createTopics() throws Exception {
|
public void createTopics() throws Exception {
|
||||||
applicationId = "appId-" + TEST_NUMBER.getAndIncrement();
|
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_INPUT_TOPIC, NUM_TOPIC_PARTITIONS, 1);
|
||||||
CLUSTER.createTopic(MULTI_PARTITION_THROUGH_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.createTopic(MULTI_PARTITION_OUTPUT_TOPIC, NUM_TOPIC_PARTITIONS, 1);
|
||||||
|
CLUSTER.setGroupStandbyReplicas(applicationId, 1);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@ParameterizedTest
|
||||||
public void shouldBeAbleToRunWithEosEnabled() throws Exception {
|
@ValueSource(strings = {"classic", "streams"})
|
||||||
runSimpleCopyTest(1, SINGLE_PARTITION_INPUT_TOPIC, null, SINGLE_PARTITION_OUTPUT_TOPIC, false);
|
public void shouldBeAbleToRunWithEosEnabled(final String groupProtocol) throws Exception {
|
||||||
|
runSimpleCopyTest(1, SINGLE_PARTITION_INPUT_TOPIC, null, SINGLE_PARTITION_OUTPUT_TOPIC, false, groupProtocol);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@ParameterizedTest
|
||||||
public void shouldCommitCorrectOffsetIfInputTopicIsTransactional() throws Exception {
|
@ValueSource(strings = {"classic", "streams"})
|
||||||
runSimpleCopyTest(1, SINGLE_PARTITION_INPUT_TOPIC, null, SINGLE_PARTITION_OUTPUT_TOPIC, true);
|
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())));
|
try (final Admin adminClient = Admin.create(mkMap(mkEntry(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers())));
|
||||||
final Consumer<byte[], byte[]> consumer = new KafkaConsumer<>(mkMap(
|
final Consumer<byte[], byte[]> consumer = new KafkaConsumer<>(mkMap(
|
||||||
|
@ -215,36 +228,42 @@ public class EosIntegrationTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@ParameterizedTest
|
||||||
public void shouldBeAbleToRestartAfterClose() throws Exception {
|
@ValueSource(strings = {"classic", "streams"})
|
||||||
runSimpleCopyTest(2, SINGLE_PARTITION_INPUT_TOPIC, null, SINGLE_PARTITION_OUTPUT_TOPIC, false);
|
public void shouldBeAbleToRestartAfterClose(final String groupProtocol) throws Exception {
|
||||||
|
runSimpleCopyTest(2, SINGLE_PARTITION_INPUT_TOPIC, null, SINGLE_PARTITION_OUTPUT_TOPIC, false, groupProtocol);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@ParameterizedTest
|
||||||
public void shouldBeAbleToCommitToMultiplePartitions() throws Exception {
|
@ValueSource(strings = {"classic", "streams"})
|
||||||
runSimpleCopyTest(1, SINGLE_PARTITION_INPUT_TOPIC, null, MULTI_PARTITION_OUTPUT_TOPIC, false);
|
public void shouldBeAbleToCommitToMultiplePartitions(final String groupProtocol) throws Exception {
|
||||||
|
runSimpleCopyTest(1, SINGLE_PARTITION_INPUT_TOPIC, null, MULTI_PARTITION_OUTPUT_TOPIC, false, groupProtocol);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@ParameterizedTest
|
||||||
public void shouldBeAbleToCommitMultiplePartitionOffsets() throws Exception {
|
@ValueSource(strings = {"classic", "streams"})
|
||||||
runSimpleCopyTest(1, MULTI_PARTITION_INPUT_TOPIC, null, SINGLE_PARTITION_OUTPUT_TOPIC, false);
|
public void shouldBeAbleToCommitMultiplePartitionOffsets(final String groupProtocol) throws Exception {
|
||||||
|
runSimpleCopyTest(1, MULTI_PARTITION_INPUT_TOPIC, null, SINGLE_PARTITION_OUTPUT_TOPIC, false, groupProtocol);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@ParameterizedTest
|
||||||
public void shouldBeAbleToRunWithTwoSubtopologies() throws Exception {
|
@ValueSource(strings = {"classic", "streams"})
|
||||||
runSimpleCopyTest(1, SINGLE_PARTITION_INPUT_TOPIC, SINGLE_PARTITION_THROUGH_TOPIC, SINGLE_PARTITION_OUTPUT_TOPIC, false);
|
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
|
@ParameterizedTest
|
||||||
public void shouldBeAbleToRunWithTwoSubtopologiesAndMultiplePartitions() throws Exception {
|
@ValueSource(strings = {"classic", "streams"})
|
||||||
runSimpleCopyTest(1, MULTI_PARTITION_INPUT_TOPIC, MULTI_PARTITION_THROUGH_TOPIC, MULTI_PARTITION_OUTPUT_TOPIC, false);
|
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,
|
private void runSimpleCopyTest(final int numberOfRestarts,
|
||||||
final String inputTopic,
|
final String inputTopic,
|
||||||
final String throughTopic,
|
final String throughTopic,
|
||||||
final String outputTopic,
|
final String outputTopic,
|
||||||
final boolean inputTopicTransactional) throws Exception {
|
final boolean inputTopicTransactional,
|
||||||
|
final String groupProtocol) throws Exception {
|
||||||
final StreamsBuilder builder = new StreamsBuilder();
|
final StreamsBuilder builder = new StreamsBuilder();
|
||||||
final KStream<Long, Long> input = builder.stream(inputTopic);
|
final KStream<Long, Long> input = builder.stream(inputTopic);
|
||||||
KStream<Long, Long> output = input;
|
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.AUTO_OFFSET_RESET_CONFIG), "earliest");
|
||||||
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), MAX_POLL_INTERVAL_MS - 1);
|
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.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) {
|
for (int i = 0; i < numberOfRestarts; ++i) {
|
||||||
final Properties config = StreamsTestUtils.getStreamsConfig(
|
final Properties config = StreamsTestUtils.getStreamsConfig(
|
||||||
|
@ -326,8 +346,9 @@ public class EosIntegrationTest {
|
||||||
return recordsPerKey;
|
return recordsPerKey;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@ParameterizedTest
|
||||||
public void shouldBeAbleToPerformMultipleTransactions() throws Exception {
|
@ValueSource(strings = {"classic", "streams"})
|
||||||
|
public void shouldBeAbleToPerformMultipleTransactions(final String groupProtocol) throws Exception {
|
||||||
final StreamsBuilder builder = new StreamsBuilder();
|
final StreamsBuilder builder = new StreamsBuilder();
|
||||||
builder.stream(SINGLE_PARTITION_INPUT_TOPIC).to(SINGLE_PARTITION_OUTPUT_TOPIC);
|
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(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
||||||
properties.put(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1000");
|
properties.put(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1000");
|
||||||
properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
||||||
|
properties.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol);
|
||||||
|
|
||||||
final Properties config = StreamsTestUtils.getStreamsConfig(
|
final Properties config = StreamsTestUtils.getStreamsConfig(
|
||||||
applicationId,
|
applicationId,
|
||||||
|
@ -374,8 +396,8 @@ public class EosIntegrationTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@ValueSource(booleans = {true, false})
|
@MethodSource("groupProtocolAndProcessingThreadsParameters")
|
||||||
public void shouldNotViolateEosIfOneTaskFails(final boolean processingThreadsEnabled) throws Exception {
|
public void shouldNotViolateEosIfOneTaskFails(final String groupProtocol, final boolean processingThreadsEnabled) throws Exception {
|
||||||
|
|
||||||
// this test writes 10 + 5 + 5 records per partition (running with 2 partitions)
|
// 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
|
// 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
|
// -> the failure only kills one thread
|
||||||
// after fail over, we should read 40 committed records (even if 50 record got written)
|
// 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);
|
startApplicationAndWaitUntilRunning(streams);
|
||||||
|
|
||||||
final List<KeyValue<Long, Long>> committedDataBeforeFailure = prepareData(0L, 10L, 0L, 1L);
|
final List<KeyValue<Long, Long>> committedDataBeforeFailure = prepareData(0L, 10L, 0L, 1L);
|
||||||
|
@ -476,8 +498,8 @@ public class EosIntegrationTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@ValueSource(booleans = {true, false})
|
@MethodSource("groupProtocolAndProcessingThreadsParameters")
|
||||||
public void shouldNotViolateEosIfOneTaskFailsWithState(final boolean processingThreadsEnabled) throws Exception {
|
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)
|
// 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
|
// 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
|
// 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
|
// 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);
|
startApplicationAndWaitUntilRunning(streams);
|
||||||
|
|
||||||
final List<KeyValue<Long, Long>> committedDataBeforeFailure = prepareData(0L, 10L, 0L, 1L);
|
final List<KeyValue<Long, Long>> committedDataBeforeFailure = prepareData(0L, 10L, 0L, 1L);
|
||||||
|
@ -594,8 +616,8 @@ public class EosIntegrationTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@ValueSource(booleans = {true, false})
|
@MethodSource("groupProtocolAndProcessingThreadsParameters")
|
||||||
public void shouldNotViolateEosIfOneTaskGetsFencedUsingIsolatedAppInstances(final boolean processingThreadsEnabled) throws Exception {
|
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)
|
// 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
|
// 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
|
// afterward, the "stalling" thread resumes, and another rebalance should get triggered
|
||||||
// we write the remaining 20 records and verify to read 60 result records
|
// we write the remaining 20 records and verify to read 60 result records
|
||||||
|
|
||||||
try (
|
try (
|
||||||
final KafkaStreams streams1 = getKafkaStreams("streams1", false, "appDir1", 1, processingThreadsEnabled);
|
final KafkaStreams streams1 = getKafkaStreams("streams1", false, "appDir1", 1, groupProtocol, processingThreadsEnabled);
|
||||||
final KafkaStreams streams2 = getKafkaStreams("streams2", false, "appDir2", 1, processingThreadsEnabled)
|
final KafkaStreams streams2 = getKafkaStreams("streams2", false, "appDir2", 1, groupProtocol, processingThreadsEnabled)
|
||||||
) {
|
) {
|
||||||
startApplicationAndWaitUntilRunning(streams1);
|
startApplicationAndWaitUntilRunning(streams1);
|
||||||
startApplicationAndWaitUntilRunning(streams2);
|
startApplicationAndWaitUntilRunning(streams2);
|
||||||
|
@ -667,13 +688,10 @@ public class EosIntegrationTest {
|
||||||
"Expected a host to start stalling"
|
"Expected a host to start stalling"
|
||||||
);
|
);
|
||||||
final String observedStallingHost = stallingHost.get();
|
final String observedStallingHost = stallingHost.get();
|
||||||
final KafkaStreams stallingInstance;
|
|
||||||
final KafkaStreams remainingInstance;
|
final KafkaStreams remainingInstance;
|
||||||
if ("streams1".equals(observedStallingHost)) {
|
if ("streams1".equals(observedStallingHost)) {
|
||||||
stallingInstance = streams1;
|
|
||||||
remainingInstance = streams2;
|
remainingInstance = streams2;
|
||||||
} else if ("streams2".equals(observedStallingHost)) {
|
} else if ("streams2".equals(observedStallingHost)) {
|
||||||
stallingInstance = streams2;
|
|
||||||
remainingInstance = streams1;
|
remainingInstance = streams1;
|
||||||
} else {
|
} else {
|
||||||
throw new IllegalArgumentException("unexpected host name: " + observedStallingHost);
|
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
|
// the assignment is. We only really care that the remaining instance only sees one host
|
||||||
// that owns both partitions.
|
// that owns both partitions.
|
||||||
waitForCondition(
|
waitForCondition(
|
||||||
() -> stallingInstance.metadataForAllStreamsClients().size() == 2
|
() -> remainingInstance.metadataForAllStreamsClients().size() == 1
|
||||||
&& remainingInstance.metadataForAllStreamsClients().size() == 1
|
|
||||||
&& remainingInstance.metadataForAllStreamsClients().iterator().next().topicPartitions().size() == 2,
|
&& remainingInstance.metadataForAllStreamsClients().iterator().next().topicPartitions().size() == 2,
|
||||||
MAX_WAIT_TIME_MS,
|
MAX_WAIT_TIME_MS,
|
||||||
() -> "Should have rebalanced.\n" +
|
() -> "Should have rebalanced.\n" +
|
||||||
|
@ -755,12 +772,12 @@ public class EosIntegrationTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@ValueSource(booleans = {true, false})
|
@MethodSource("groupProtocolAndProcessingThreadsParameters")
|
||||||
public void shouldWriteLatestOffsetsToCheckpointOnShutdown(final boolean processingThreadsEnabled) throws Exception {
|
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>> writtenData = prepareData(0L, 10, 0L, 1L);
|
||||||
final List<KeyValue<Long, Long>> expectedResult = computeExpectedResult(writtenData);
|
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);
|
writeInputData(writtenData);
|
||||||
|
|
||||||
startApplicationAndWaitUntilRunning(streams);
|
startApplicationAndWaitUntilRunning(streams);
|
||||||
|
@ -787,9 +804,9 @@ public class EosIntegrationTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@ValueSource(booleans = {true, false})
|
@MethodSource("groupProtocolAndProcessingThreadsParameters")
|
||||||
public void shouldCheckpointRestoredOffsetsWhenClosingCleanDuringRestoring(
|
public void shouldCheckpointRestoredOffsetsWhenClosingCleanDuringRestoring(
|
||||||
final boolean processingThreadsEnabled) throws Exception {
|
final String groupProtocol, final boolean processingThreadsEnabled) throws Exception {
|
||||||
|
|
||||||
final Properties streamsConfiguration = new Properties();
|
final Properties streamsConfiguration = new Properties();
|
||||||
streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId);
|
streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId);
|
||||||
|
@ -801,6 +818,7 @@ public class EosIntegrationTest {
|
||||||
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
||||||
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(applicationId).getPath());
|
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(applicationId).getPath());
|
||||||
streamsConfiguration.put(InternalConfig.PROCESSING_THREADS_ENABLED, processingThreadsEnabled);
|
streamsConfiguration.put(InternalConfig.PROCESSING_THREADS_ENABLED, processingThreadsEnabled);
|
||||||
|
streamsConfiguration.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol);
|
||||||
streamsConfiguration.put(StreamsConfig.restoreConsumerPrefix(ConsumerConfig.MAX_POLL_RECORDS_CONFIG), 100);
|
streamsConfiguration.put(StreamsConfig.restoreConsumerPrefix(ConsumerConfig.MAX_POLL_RECORDS_CONFIG), 100);
|
||||||
final String stateStoreName = "stateStore";
|
final String stateStoreName = "stateStore";
|
||||||
|
|
||||||
|
@ -934,8 +952,13 @@ public class EosIntegrationTest {
|
||||||
static final AtomicReference<TaskId> TASK_WITH_DATA = new AtomicReference<>();
|
static final AtomicReference<TaskId> TASK_WITH_DATA = new AtomicReference<>();
|
||||||
static final AtomicBoolean DID_REVOKE_IDLE_TASK = new AtomicBoolean(false);
|
static final AtomicBoolean DID_REVOKE_IDLE_TASK = new AtomicBoolean(false);
|
||||||
|
|
||||||
@Test
|
@ParameterizedTest
|
||||||
public void shouldNotCommitActiveTasksWithPendingInputIfRevokedTaskDidNotMakeProgress() throws Exception {
|
@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 AtomicBoolean requestCommit = new AtomicBoolean(false);
|
||||||
|
|
||||||
final StreamsBuilder builder = new StreamsBuilder();
|
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.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.producerPrefix(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG), Integer.MAX_VALUE);
|
||||||
properties.put(StreamsConfig.TASK_ASSIGNOR_CLASS_CONFIG, TestTaskAssignor.class.getName());
|
properties.put(StreamsConfig.TASK_ASSIGNOR_CLASS_CONFIG, TestTaskAssignor.class.getName());
|
||||||
|
properties.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol);
|
||||||
|
|
||||||
final Properties config = StreamsTestUtils.getStreamsConfig(
|
final Properties config = StreamsTestUtils.getStreamsConfig(
|
||||||
applicationId,
|
applicationId,
|
||||||
|
@ -1003,9 +1027,9 @@ public class EosIntegrationTest {
|
||||||
// add second thread, to trigger rebalance
|
// add second thread, to trigger rebalance
|
||||||
// expect idle task to get revoked -- this should not trigger a TX commit
|
// expect idle task to get revoked -- this should not trigger a TX commit
|
||||||
streams.addStreamThread();
|
streams.addStreamThread();
|
||||||
|
if (groupProtocol.equals("classic")) {
|
||||||
waitForCondition(DID_REVOKE_IDLE_TASK::get, "Idle Task was not revoked as expected.");
|
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)
|
// best-effort sanity check (might pass and not detect issue in slow environments)
|
||||||
try {
|
try {
|
||||||
readResult(SINGLE_PARTITION_OUTPUT_TOPIC, 1, "consumer", 10_000L);
|
readResult(SINGLE_PARTITION_OUTPUT_TOPIC, 1, "consumer", 10_000L);
|
||||||
|
@ -1104,6 +1128,7 @@ public class EosIntegrationTest {
|
||||||
final boolean withState,
|
final boolean withState,
|
||||||
final String appDir,
|
final String appDir,
|
||||||
final int numberOfStreamsThreads,
|
final int numberOfStreamsThreads,
|
||||||
|
final String groupProtocol,
|
||||||
final boolean processingThreadsEnabled) {
|
final boolean processingThreadsEnabled) {
|
||||||
commitRequested = new AtomicInteger(0);
|
commitRequested = new AtomicInteger(0);
|
||||||
errorInjected = new AtomicBoolean(false);
|
errorInjected = new AtomicBoolean(false);
|
||||||
|
@ -1212,6 +1237,7 @@ public class EosIntegrationTest {
|
||||||
properties.put(StreamsConfig.STATE_DIR_CONFIG, stateTmpDir + appDir);
|
properties.put(StreamsConfig.STATE_DIR_CONFIG, stateTmpDir + appDir);
|
||||||
properties.put(StreamsConfig.APPLICATION_SERVER_CONFIG, dummyHostName + ":2142");
|
properties.put(StreamsConfig.APPLICATION_SERVER_CONFIG, dummyHostName + ":2142");
|
||||||
properties.put(InternalConfig.PROCESSING_THREADS_ENABLED, processingThreadsEnabled);
|
properties.put(InternalConfig.PROCESSING_THREADS_ENABLED, processingThreadsEnabled);
|
||||||
|
properties.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol);
|
||||||
|
|
||||||
final Properties config = StreamsTestUtils.getStreamsConfig(
|
final Properties config = StreamsTestUtils.getStreamsConfig(
|
||||||
applicationId,
|
applicationId,
|
||||||
|
|
|
@ -28,8 +28,8 @@ import org.apache.kafka.common.serialization.StringSerializer;
|
||||||
import org.apache.kafka.common.utils.MockTime;
|
import org.apache.kafka.common.utils.MockTime;
|
||||||
import org.apache.kafka.common.utils.Utils;
|
import org.apache.kafka.common.utils.Utils;
|
||||||
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig;
|
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;
|
||||||
import org.apache.kafka.streams.KafkaStreams.CloseOptions;
|
|
||||||
import org.apache.kafka.streams.KeyValue;
|
import org.apache.kafka.streams.KeyValue;
|
||||||
import org.apache.kafka.streams.StreamsBuilder;
|
import org.apache.kafka.streams.StreamsBuilder;
|
||||||
import org.apache.kafka.streams.StreamsConfig;
|
import org.apache.kafka.streams.StreamsConfig;
|
||||||
|
@ -159,7 +159,7 @@ public class KafkaStreamsCloseOptionsIntegrationTest {
|
||||||
IntegrationTestUtils.startApplicationAndWaitUntilRunning(streams);
|
IntegrationTestUtils.startApplicationAndWaitUntilRunning(streams);
|
||||||
IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived(resultConsumerConfig, OUTPUT_TOPIC, 10);
|
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);
|
waitForEmptyConsumerGroup(adminClient, streamsConfig.getProperty(StreamsConfig.APPLICATION_ID_CONFIG), 0);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -49,6 +49,7 @@ import org.apache.kafka.streams.errors.StreamsStoppedException;
|
||||||
import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler;
|
import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler;
|
||||||
import org.apache.kafka.streams.errors.UnknownStateStoreException;
|
import org.apache.kafka.streams.errors.UnknownStateStoreException;
|
||||||
import org.apache.kafka.streams.internals.ClientInstanceIdsImpl;
|
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.ClientMetrics;
|
||||||
import org.apache.kafka.streams.internals.metrics.StreamsClientMetricsDelegatingReporter;
|
import org.apache.kafka.streams.internals.metrics.StreamsClientMetricsDelegatingReporter;
|
||||||
import org.apache.kafka.streams.processor.StandbyUpdateListener;
|
import org.apache.kafka.streams.processor.StandbyUpdateListener;
|
||||||
|
@ -488,7 +489,7 @@ public class KafkaStreams implements AutoCloseable {
|
||||||
closeToError();
|
closeToError();
|
||||||
}
|
}
|
||||||
final StreamThread deadThread = (StreamThread) Thread.currentThread();
|
final StreamThread deadThread = (StreamThread) Thread.currentThread();
|
||||||
deadThread.shutdown(false);
|
deadThread.shutdown(org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP);
|
||||||
addStreamThread();
|
addStreamThread();
|
||||||
if (throwable instanceof RuntimeException) {
|
if (throwable instanceof RuntimeException) {
|
||||||
throw (RuntimeException) throwable;
|
throw (RuntimeException) throwable;
|
||||||
|
@ -765,7 +766,7 @@ public class KafkaStreams implements AutoCloseable {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void onUpdateStart(final TopicPartition topicPartition,
|
public void onUpdateStart(final TopicPartition topicPartition,
|
||||||
final String storeName,
|
final String storeName,
|
||||||
final long startingOffset) {
|
final long startingOffset) {
|
||||||
if (userStandbyListener != null) {
|
if (userStandbyListener != null) {
|
||||||
try {
|
try {
|
||||||
|
@ -1136,7 +1137,7 @@ public class KafkaStreams implements AutoCloseable {
|
||||||
return Optional.of(streamThread.getName());
|
return Optional.of(streamThread.getName());
|
||||||
} else {
|
} else {
|
||||||
log.warn("Terminating the new thread because the Kafka Streams client is in state {}", state);
|
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);
|
threads.remove(streamThread);
|
||||||
final long cacheSizePerThread = cacheSizePerThread(numLiveStreamThreads());
|
final long cacheSizePerThread = cacheSizePerThread(numLiveStreamThreads());
|
||||||
log.info("Resizing thread cache due to terminating added thread, new cache size per thread is {}", cacheSizePerThread);
|
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());
|
final boolean callingThreadIsNotCurrentStreamThread = !streamThread.getName().equals(Thread.currentThread().getName());
|
||||||
if (streamThread.isThreadAlive() && (callingThreadIsNotCurrentStreamThread || numLiveStreamThreads() == 1)) {
|
if (streamThread.isThreadAlive() && (callingThreadIsNotCurrentStreamThread || numLiveStreamThreads() == 1)) {
|
||||||
log.info("Removing StreamThread {}", streamThread.getName());
|
log.info("Removing StreamThread {}", streamThread.getName());
|
||||||
streamThread.shutdown(true);
|
streamThread.shutdown(org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
|
||||||
if (callingThreadIsNotCurrentStreamThread) {
|
if (callingThreadIsNotCurrentStreamThread) {
|
||||||
final long remainingTimeMs = timeoutMs - (time.milliseconds() - startMs);
|
final long remainingTimeMs = timeoutMs - (time.milliseconds() - startMs);
|
||||||
if (remainingTimeMs <= 0 || !streamThread.waitOnThreadState(StreamThread.State.DEAD, remainingTimeMs)) {
|
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
|
* Class that handles options passed in case of {@code KafkaStreams} instance scale down
|
||||||
*/
|
*/
|
||||||
|
@Deprecated(since = "4.2")
|
||||||
public static class CloseOptions {
|
public static class CloseOptions {
|
||||||
private Duration timeout = Duration.ofMillis(Long.MAX_VALUE);
|
private Duration timeout = Duration.ofMillis(Long.MAX_VALUE);
|
||||||
private boolean leaveGroup = false;
|
private boolean leaveGroup = false;
|
||||||
|
|
||||||
|
@Deprecated(since = "4.2")
|
||||||
public CloseOptions timeout(final Duration timeout) {
|
public CloseOptions timeout(final Duration timeout) {
|
||||||
this.timeout = timeout;
|
this.timeout = timeout;
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Deprecated(since = "4.2")
|
||||||
public CloseOptions leaveGroup(final boolean leaveGroup) {
|
public CloseOptions leaveGroup(final boolean leaveGroup) {
|
||||||
this.leaveGroup = leaveGroup;
|
this.leaveGroup = leaveGroup;
|
||||||
return this;
|
return this;
|
||||||
|
@ -1438,10 +1442,14 @@ public class KafkaStreams implements AutoCloseable {
|
||||||
* This will block until all threads have stopped.
|
* This will block until all threads have stopped.
|
||||||
*/
|
*/
|
||||||
public void close() {
|
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();
|
stateDirCleaner.shutdownNow();
|
||||||
if (rocksDBMetricsRecordingService != null) {
|
if (rocksDBMetricsRecordingService != null) {
|
||||||
rocksDBMetricsRecordingService.shutdownNow();
|
rocksDBMetricsRecordingService.shutdownNow();
|
||||||
|
@ -1453,7 +1461,9 @@ public class KafkaStreams implements AutoCloseable {
|
||||||
return new Thread(() -> {
|
return new Thread(() -> {
|
||||||
// notify all the threads to stop; avoid deadlocks by stopping any
|
// notify all the threads to stop; avoid deadlocks by stopping any
|
||||||
// further state reports from the thread since we're shutting down
|
// 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);
|
log.info("Shutting down {} stream threads", numStreamThreads);
|
||||||
|
|
||||||
|
@ -1513,7 +1523,7 @@ public class KafkaStreams implements AutoCloseable {
|
||||||
}, clientId + "-CloseThread");
|
}, 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;
|
final long timeoutMs;
|
||||||
if (timeout.isPresent()) {
|
if (timeout.isPresent()) {
|
||||||
timeoutMs = timeout.get();
|
timeoutMs = timeout.get();
|
||||||
|
@ -1544,7 +1554,7 @@ public class KafkaStreams implements AutoCloseable {
|
||||||
+ "PENDING_SHUTDOWN, PENDING_ERROR, ERROR, or NOT_RUNNING");
|
+ "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.setDaemon(true);
|
||||||
shutdownThread.start();
|
shutdownThread.start();
|
||||||
|
@ -1562,7 +1572,7 @@ public class KafkaStreams implements AutoCloseable {
|
||||||
if (!setState(State.PENDING_ERROR)) {
|
if (!setState(State.PENDING_ERROR)) {
|
||||||
log.info("Skipping shutdown since we are already in {}", state());
|
log.info("Skipping shutdown since we are already in {}", state());
|
||||||
} else {
|
} 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.setDaemon(true);
|
||||||
shutdownThread.start();
|
shutdownThread.start();
|
||||||
|
@ -1588,12 +1598,13 @@ public class KafkaStreams implements AutoCloseable {
|
||||||
throw new IllegalArgumentException("Timeout can't be negative.");
|
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
|
* Shutdown this {@code KafkaStreams} by signaling all the threads to stop, and then wait up to the timeout for the
|
||||||
* threads to join.
|
* 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
|
* @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
|
* trigger consumer leave call
|
||||||
* @return {@code true} if all threads were successfully stopped—{@code false} if the timeout was reached
|
* @return {@code true} if all threads were successfully stopped—{@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}.
|
* 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}
|
* @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 {
|
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—{@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");
|
Objects.requireNonNull(options, "options cannot be null");
|
||||||
final String msgPrefix = prepareMillisCheckFailMsgPrefix(options.timeout, "timeout");
|
final CloseOptionsInternal optionsInternal = new CloseOptionsInternal(options);
|
||||||
final long timeoutMs = validateMillisecondDuration(options.timeout, msgPrefix);
|
final String msgPrefix = prepareMillisCheckFailMsgPrefix(optionsInternal.timeout(), "timeout");
|
||||||
|
final long timeoutMs = validateMillisecondDuration(optionsInternal.timeout().get(), msgPrefix);
|
||||||
if (timeoutMs < 0) {
|
if (timeoutMs < 0) {
|
||||||
throw new IllegalArgumentException("Timeout can't be negative.");
|
throw new IllegalArgumentException("Timeout can't be negative.");
|
||||||
}
|
}
|
||||||
|
|
||||||
return close(Optional.of(timeoutMs), options.leaveGroup);
|
return close(Optional.of(timeoutMs), optionsInternal.operation());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -91,9 +91,9 @@ import java.util.Optional;
|
||||||
import java.util.Queue;
|
import java.util.Queue;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.UUID;
|
import java.util.UUID;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
import java.util.function.BiConsumer;
|
import java.util.function.BiConsumer;
|
||||||
import java.util.stream.Collectors;
|
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
|
// 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 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 AtomicLong lastShutdownWarningTimestamp = new AtomicLong(0L);
|
||||||
private final boolean eosEnabled;
|
private final boolean eosEnabled;
|
||||||
private final boolean stateUpdaterEnabled;
|
private final boolean stateUpdaterEnabled;
|
||||||
|
@ -898,7 +899,7 @@ public class StreamThread extends Thread implements ProcessingThread {
|
||||||
cleanRun = runLoop();
|
cleanRun = runLoop();
|
||||||
} catch (final Throwable e) {
|
} catch (final Throwable e) {
|
||||||
failedStreamThreadSensor.record();
|
failedStreamThreadSensor.record();
|
||||||
leaveGroupRequested.set(true);
|
leaveGroupRequested.set(org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
|
||||||
streamsUncaughtExceptionHandler.accept(e, false);
|
streamsUncaughtExceptionHandler.accept(e, false);
|
||||||
// Note: the above call currently rethrows the exception, so nothing below this line will be executed
|
// Note: the above call currently rethrows the exception, so nothing below this line will be executed
|
||||||
} finally {
|
} finally {
|
||||||
|
@ -1547,7 +1548,7 @@ public class StreamThread extends Thread implements ProcessingThread {
|
||||||
if (streamsRebalanceData.isPresent()) {
|
if (streamsRebalanceData.isPresent()) {
|
||||||
boolean hasMissingSourceTopics = false;
|
boolean hasMissingSourceTopics = false;
|
||||||
String missingTopicsDetail = null;
|
String missingTopicsDetail = null;
|
||||||
|
|
||||||
for (final StreamsGroupHeartbeatResponseData.Status status : streamsRebalanceData.get().statuses()) {
|
for (final StreamsGroupHeartbeatResponseData.Status status : streamsRebalanceData.get().statuses()) {
|
||||||
if (status.statusCode() == StreamsGroupHeartbeatResponse.Status.SHUTDOWN_APPLICATION.code()) {
|
if (status.statusCode() == StreamsGroupHeartbeatResponse.Status.SHUTDOWN_APPLICATION.code()) {
|
||||||
shutdownErrorHook.run();
|
shutdownErrorHook.run();
|
||||||
|
@ -1560,7 +1561,7 @@ public class StreamThread extends Thread implements ProcessingThread {
|
||||||
throw new TopologyException(errorMsg);
|
throw new TopologyException(errorMsg);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (hasMissingSourceTopics) {
|
if (hasMissingSourceTopics) {
|
||||||
handleMissingSourceTopicsWithTimeout(missingTopicsDetail);
|
handleMissingSourceTopicsWithTimeout(missingTopicsDetail);
|
||||||
} else {
|
} else {
|
||||||
|
@ -1589,25 +1590,25 @@ public class StreamThread extends Thread implements ProcessingThread {
|
||||||
// Start timeout tracking on first encounter with missing topics
|
// Start timeout tracking on first encounter with missing topics
|
||||||
if (topicsReadyTimer == null) {
|
if (topicsReadyTimer == null) {
|
||||||
topicsReadyTimer = time.timer(maxPollTimeMs);
|
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);
|
missingTopicsDetail, maxPollTimeMs);
|
||||||
} else {
|
} else {
|
||||||
topicsReadyTimer.update();
|
topicsReadyTimer.update();
|
||||||
}
|
}
|
||||||
|
|
||||||
if (topicsReadyTimer.isExpired()) {
|
if (topicsReadyTimer.isExpired()) {
|
||||||
final long elapsedTime = topicsReadyTimer.elapsedMs();
|
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);
|
missingTopicsDetail, elapsedTime);
|
||||||
log.error(errorMsg);
|
log.error(errorMsg);
|
||||||
|
|
||||||
throw new MissingSourceTopicException(errorMsg);
|
throw new MissingSourceTopicException(errorMsg);
|
||||||
} else {
|
} 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());
|
missingTopicsDetail, topicsReadyTimer.elapsedMs(), topicsReadyTimer.remainingMs());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
static Map<TopicPartition, PartitionInfo> getTopicPartitionInfo(final Map<HostInfo, Set<TopicPartition>> partitionsByHost) {
|
static Map<TopicPartition, PartitionInfo> getTopicPartitionInfo(final Map<HostInfo, Set<TopicPartition>> partitionsByHost) {
|
||||||
final Map<TopicPartition, PartitionInfo> topicToPartitionInfo = new HashMap<>();
|
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
|
* 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
|
* (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");
|
log.info("Informed to shut down");
|
||||||
final State oldState = setState(State.PENDING_SHUTDOWN);
|
final State oldState = setState(State.PENDING_SHUTDOWN);
|
||||||
leaveGroupRequested.set(leaveGroup);
|
leaveGroupRequested.set(operation);
|
||||||
if (oldState == State.CREATED) {
|
if (oldState == State.CREATED) {
|
||||||
// The thread may not have been started. Take responsibility for shutting down
|
// The thread may not have been started. Take responsibility for shutting down
|
||||||
completeShutdown(true);
|
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);
|
log.error("Failed to close changelog reader due to the following error:", e);
|
||||||
}
|
}
|
||||||
try {
|
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));
|
mainConsumer.close(CloseOptions.groupMembershipOperation(membershipOperation));
|
||||||
} catch (final Throwable e) {
|
} catch (final Throwable e) {
|
||||||
log.error("Failed to close consumer due to the following error:", e);
|
log.error("Failed to close consumer due to the following error:", e);
|
||||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.kafka.streams;
|
||||||
import org.apache.kafka.clients.admin.Admin;
|
import org.apache.kafka.clients.admin.Admin;
|
||||||
import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo;
|
import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo;
|
||||||
import org.apache.kafka.clients.admin.MockAdminClient;
|
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.clients.producer.MockProducer;
|
||||||
import org.apache.kafka.common.Cluster;
|
import org.apache.kafka.common.Cluster;
|
||||||
import org.apache.kafka.common.KafkaFuture;
|
import org.apache.kafka.common.KafkaFuture;
|
||||||
|
@ -310,8 +309,12 @@ public class KafkaStreamsTest {
|
||||||
|
|
||||||
private void prepareConsumer(final StreamThread thread, final AtomicReference<StreamThread.State> state) {
|
private void prepareConsumer(final StreamThread thread, final AtomicReference<StreamThread.State> state) {
|
||||||
doAnswer(invocation -> {
|
doAnswer(invocation -> {
|
||||||
supplier.consumer.close(CloseOptions.groupMembershipOperation(CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP));
|
supplier.consumer.close(
|
||||||
supplier.restoreConsumer.close(CloseOptions.groupMembershipOperation(CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP));
|
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) {
|
for (final MockProducer<byte[], byte[]> producer : supplier.producers) {
|
||||||
producer.close();
|
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.PENDING_SHUTDOWN, StreamThread.State.RUNNING);
|
||||||
threadStateListenerCapture.getValue().onChange(thread, StreamThread.State.DEAD, StreamThread.State.PENDING_SHUTDOWN);
|
threadStateListenerCapture.getValue().onChange(thread, StreamThread.State.DEAD, StreamThread.State.PENDING_SHUTDOWN);
|
||||||
return null;
|
return null;
|
||||||
}).when(thread).shutdown(false);
|
}).when(thread).shutdown(CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void prepareThreadLock(final StreamThread thread) {
|
private void prepareThreadLock(final StreamThread thread) {
|
||||||
|
@ -571,7 +574,7 @@ public class KafkaStreamsTest {
|
||||||
|
|
||||||
for (int i = 0; i < NUM_THREADS; i++) {
|
for (int i = 0; i < NUM_THREADS; i++) {
|
||||||
final StreamThread tmpThread = streams.threads.get(i);
|
final StreamThread tmpThread = streams.threads.get(i);
|
||||||
tmpThread.shutdown(false);
|
tmpThread.shutdown(CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP);
|
||||||
waitForCondition(() -> tmpThread.state() == StreamThread.State.DEAD,
|
waitForCondition(() -> tmpThread.state() == StreamThread.State.DEAD,
|
||||||
"Thread never stopped.");
|
"Thread never stopped.");
|
||||||
streams.threads.get(i).join();
|
streams.threads.get(i).join();
|
||||||
|
@ -790,7 +793,7 @@ public class KafkaStreamsTest {
|
||||||
prepareThreadLock(streamThreadTwo);
|
prepareThreadLock(streamThreadTwo);
|
||||||
try (final KafkaStreams streams = new KafkaStreams(getBuilderWithSource().build(), props, supplier, time)) {
|
try (final KafkaStreams streams = new KafkaStreams(getBuilderWithSource().build(), props, supplier, time)) {
|
||||||
streams.start();
|
streams.start();
|
||||||
streamThreadOne.shutdown(true);
|
streamThreadOne.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
|
||||||
final Set<ThreadMetadata> threads = streams.metadataForLocalThreads();
|
final Set<ThreadMetadata> threads = streams.metadataForLocalThreads();
|
||||||
assertThat(threads.size(), equalTo(1));
|
assertThat(threads.size(), equalTo(1));
|
||||||
assertThat(threads, hasItem(streamThreadTwo.threadMetadata()));
|
assertThat(threads, hasItem(streamThreadTwo.threadMetadata()));
|
||||||
|
@ -1016,9 +1019,8 @@ public class KafkaStreamsTest {
|
||||||
() -> streams.state() == KafkaStreams.State.RUNNING,
|
() -> streams.state() == KafkaStreams.State.RUNNING,
|
||||||
"Streams never started.");
|
"Streams never started.");
|
||||||
|
|
||||||
final KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
|
final CloseOptions closeOptions = CloseOptions.timeout(Duration.ZERO)
|
||||||
closeOptions.timeout(Duration.ZERO);
|
.withGroupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
|
||||||
closeOptions.leaveGroup(true);
|
|
||||||
|
|
||||||
streams.close(closeOptions);
|
streams.close(closeOptions);
|
||||||
assertThat(streams.state() == State.PENDING_SHUTDOWN, equalTo(true));
|
assertThat(streams.state() == State.PENDING_SHUTDOWN, equalTo(true));
|
||||||
|
@ -1041,8 +1043,7 @@ public class KafkaStreamsTest {
|
||||||
() -> streams.state() == KafkaStreams.State.RUNNING,
|
() -> streams.state() == KafkaStreams.State.RUNNING,
|
||||||
"Streams never started.");
|
"Streams never started.");
|
||||||
|
|
||||||
final KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
|
final CloseOptions closeOptions = CloseOptions.timeout(Duration.ZERO);
|
||||||
closeOptions.timeout(Duration.ZERO);
|
|
||||||
|
|
||||||
streams.close(closeOptions);
|
streams.close(closeOptions);
|
||||||
assertThat(streams.state() == State.PENDING_SHUTDOWN, equalTo(true));
|
assertThat(streams.state() == State.PENDING_SHUTDOWN, equalTo(true));
|
||||||
|
@ -1229,8 +1230,7 @@ public class KafkaStreamsTest {
|
||||||
prepareStreamThread(streamThreadTwo, 2);
|
prepareStreamThread(streamThreadTwo, 2);
|
||||||
prepareTerminableThread(streamThreadOne);
|
prepareTerminableThread(streamThreadOne);
|
||||||
|
|
||||||
final KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
|
final CloseOptions closeOptions = CloseOptions.timeout(Duration.ofMillis(10L));
|
||||||
closeOptions.timeout(Duration.ofMillis(10L));
|
|
||||||
try (final KafkaStreams streams = new KafkaStreamsWithTerminableThread(getBuilderWithSource().build(), props, supplier)) {
|
try (final KafkaStreams streams = new KafkaStreamsWithTerminableThread(getBuilderWithSource().build(), props, supplier)) {
|
||||||
assertFalse(streams.close(closeOptions));
|
assertFalse(streams.close(closeOptions));
|
||||||
}
|
}
|
||||||
|
@ -1243,8 +1243,7 @@ public class KafkaStreamsTest {
|
||||||
prepareStreamThread(streamThreadTwo, 2);
|
prepareStreamThread(streamThreadTwo, 2);
|
||||||
prepareTerminableThread(streamThreadOne);
|
prepareTerminableThread(streamThreadOne);
|
||||||
|
|
||||||
final KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
|
final CloseOptions closeOptions = CloseOptions.timeout(Duration.ofMillis(-1L));
|
||||||
closeOptions.timeout(Duration.ofMillis(-1L));
|
|
||||||
try (final KafkaStreams streams = new KafkaStreamsWithTerminableThread(getBuilderWithSource().build(), props, supplier, time)) {
|
try (final KafkaStreams streams = new KafkaStreamsWithTerminableThread(getBuilderWithSource().build(), props, supplier, time)) {
|
||||||
assertThrows(IllegalArgumentException.class, () -> streams.close(closeOptions));
|
assertThrows(IllegalArgumentException.class, () -> streams.close(closeOptions));
|
||||||
}
|
}
|
||||||
|
@ -1257,8 +1256,7 @@ public class KafkaStreamsTest {
|
||||||
prepareStreamThread(streamThreadTwo, 2);
|
prepareStreamThread(streamThreadTwo, 2);
|
||||||
prepareTerminableThread(streamThreadOne);
|
prepareTerminableThread(streamThreadOne);
|
||||||
|
|
||||||
final KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
|
final CloseOptions closeOptions = CloseOptions.timeout(Duration.ZERO);
|
||||||
closeOptions.timeout(Duration.ZERO);
|
|
||||||
try (final KafkaStreams streams = new KafkaStreamsWithTerminableThread(getBuilderWithSource().build(), props, supplier)) {
|
try (final KafkaStreams streams = new KafkaStreamsWithTerminableThread(getBuilderWithSource().build(), props, supplier)) {
|
||||||
assertFalse(streams.close(closeOptions));
|
assertFalse(streams.close(closeOptions));
|
||||||
}
|
}
|
||||||
|
@ -1275,9 +1273,8 @@ public class KafkaStreamsTest {
|
||||||
|
|
||||||
when(mockClientSupplier.getAdmin(any())).thenReturn(adminClient);
|
when(mockClientSupplier.getAdmin(any())).thenReturn(adminClient);
|
||||||
|
|
||||||
final KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
|
final CloseOptions closeOptions = CloseOptions.timeout(Duration.ofMillis(10L))
|
||||||
closeOptions.timeout(Duration.ofMillis(10L));
|
.withGroupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
|
||||||
closeOptions.leaveGroup(true);
|
|
||||||
try (final KafkaStreams streams = new KafkaStreamsWithTerminableThread(getBuilderWithSource().build(), props, mockClientSupplier)) {
|
try (final KafkaStreams streams = new KafkaStreamsWithTerminableThread(getBuilderWithSource().build(), props, mockClientSupplier)) {
|
||||||
assertFalse(streams.close(closeOptions));
|
assertFalse(streams.close(closeOptions));
|
||||||
}
|
}
|
||||||
|
@ -1293,9 +1290,8 @@ public class KafkaStreamsTest {
|
||||||
final MockClientSupplier mockClientSupplier = spy(MockClientSupplier.class);
|
final MockClientSupplier mockClientSupplier = spy(MockClientSupplier.class);
|
||||||
when(mockClientSupplier.getAdmin(any())).thenReturn(adminClient);
|
when(mockClientSupplier.getAdmin(any())).thenReturn(adminClient);
|
||||||
|
|
||||||
final KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
|
final CloseOptions closeOptions = CloseOptions.timeout(Duration.ofMillis(-1L))
|
||||||
closeOptions.timeout(Duration.ofMillis(-1L));
|
.withGroupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
|
||||||
closeOptions.leaveGroup(true);
|
|
||||||
try (final KafkaStreams streams = new KafkaStreamsWithTerminableThread(getBuilderWithSource().build(), props, mockClientSupplier, time)) {
|
try (final KafkaStreams streams = new KafkaStreamsWithTerminableThread(getBuilderWithSource().build(), props, mockClientSupplier, time)) {
|
||||||
assertThrows(IllegalArgumentException.class, () -> streams.close(closeOptions));
|
assertThrows(IllegalArgumentException.class, () -> streams.close(closeOptions));
|
||||||
}
|
}
|
||||||
|
@ -1312,9 +1308,8 @@ public class KafkaStreamsTest {
|
||||||
|
|
||||||
when(mockClientSupplier.getAdmin(any())).thenReturn(adminClient);
|
when(mockClientSupplier.getAdmin(any())).thenReturn(adminClient);
|
||||||
|
|
||||||
final KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
|
final CloseOptions closeOptions = CloseOptions.timeout(Duration.ZERO)
|
||||||
closeOptions.timeout(Duration.ZERO);
|
.withGroupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
|
||||||
closeOptions.leaveGroup(true);
|
|
||||||
try (final KafkaStreams streams = new KafkaStreamsWithTerminableThread(getBuilderWithSource().build(), props, mockClientSupplier)) {
|
try (final KafkaStreams streams = new KafkaStreamsWithTerminableThread(getBuilderWithSource().build(), props, mockClientSupplier)) {
|
||||||
assertFalse(streams.close(closeOptions));
|
assertFalse(streams.close(closeOptions));
|
||||||
}
|
}
|
||||||
|
@ -1720,7 +1715,7 @@ public class KafkaStreamsTest {
|
||||||
producerFuture.complete(producerInstanceId);
|
producerFuture.complete(producerInstanceId);
|
||||||
final Uuid adminInstanceId = Uuid.randomUuid();
|
final Uuid adminInstanceId = Uuid.randomUuid();
|
||||||
adminClient.setClientInstanceId(adminInstanceId);
|
adminClient.setClientInstanceId(adminInstanceId);
|
||||||
|
|
||||||
final Map<String, KafkaFuture<Uuid>> expectedClientIds = Map.of("main-consumer", consumerFuture, "some-thread-producer", producerFuture);
|
final Map<String, KafkaFuture<Uuid>> expectedClientIds = Map.of("main-consumer", consumerFuture, "some-thread-producer", producerFuture);
|
||||||
when(streamThreadOne.clientInstanceIds(any())).thenReturn(expectedClientIds);
|
when(streamThreadOne.clientInstanceIds(any())).thenReturn(expectedClientIds);
|
||||||
|
|
||||||
|
|
|
@ -60,6 +60,7 @@ import org.apache.kafka.common.utils.LogCaptureAppender;
|
||||||
import org.apache.kafka.common.utils.LogContext;
|
import org.apache.kafka.common.utils.LogContext;
|
||||||
import org.apache.kafka.common.utils.MockTime;
|
import org.apache.kafka.common.utils.MockTime;
|
||||||
import org.apache.kafka.common.utils.Time;
|
import org.apache.kafka.common.utils.Time;
|
||||||
|
import org.apache.kafka.streams.CloseOptions;
|
||||||
import org.apache.kafka.streams.GroupProtocol;
|
import org.apache.kafka.streams.GroupProtocol;
|
||||||
import org.apache.kafka.streams.StreamsConfig;
|
import org.apache.kafka.streams.StreamsConfig;
|
||||||
import org.apache.kafka.streams.StreamsConfig.InternalConfig;
|
import org.apache.kafka.streams.StreamsConfig.InternalConfig;
|
||||||
|
@ -247,7 +248,7 @@ public class StreamThreadTest {
|
||||||
if (thread.state() != State.CREATED) {
|
if (thread.state() != State.CREATED) {
|
||||||
thread.taskManager().shutdown(false);
|
thread.taskManager().shutdown(false);
|
||||||
}
|
}
|
||||||
thread.shutdown(true);
|
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
|
||||||
thread = null;
|
thread = null;
|
||||||
}
|
}
|
||||||
final Set<Thread> t = Collections.unmodifiableSet(Thread.getAllStackTraces().keySet());
|
final Set<Thread> t = Collections.unmodifiableSet(Thread.getAllStackTraces().keySet());
|
||||||
|
@ -409,12 +410,12 @@ public class StreamThreadTest {
|
||||||
assertEquals(4, stateListener.numChanges);
|
assertEquals(4, stateListener.numChanges);
|
||||||
assertEquals(StreamThread.State.PARTITIONS_ASSIGNED, stateListener.oldState);
|
assertEquals(StreamThread.State.PARTITIONS_ASSIGNED, stateListener.oldState);
|
||||||
|
|
||||||
thread.shutdown(true);
|
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
|
||||||
assertSame(StreamThread.State.PENDING_SHUTDOWN, thread.state());
|
assertSame(StreamThread.State.PENDING_SHUTDOWN, thread.state());
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldChangeStateAtStartClose(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
|
public void shouldChangeStateAtStartClose(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
|
||||||
thread = createStreamThread(CLIENT_ID, new MockTime(1), stateUpdaterEnabled, processingThreadsEnabled);
|
thread = createStreamThread(CLIENT_ID, new MockTime(1), stateUpdaterEnabled, processingThreadsEnabled);
|
||||||
|
|
||||||
|
@ -427,18 +428,18 @@ public class StreamThreadTest {
|
||||||
10 * 1000,
|
10 * 1000,
|
||||||
"Thread never started.");
|
"Thread never started.");
|
||||||
|
|
||||||
thread.shutdown(true);
|
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
|
||||||
TestUtils.waitForCondition(
|
TestUtils.waitForCondition(
|
||||||
() -> thread.state() == StreamThread.State.DEAD,
|
() -> thread.state() == StreamThread.State.DEAD,
|
||||||
10 * 1000,
|
10 * 1000,
|
||||||
"Thread never shut down.");
|
"Thread never shut down.");
|
||||||
|
|
||||||
thread.shutdown(true);
|
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
|
||||||
assertEquals(thread.state(), StreamThread.State.DEAD);
|
assertEquals(State.DEAD, thread.state());
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldCreateMetricsAtStartup(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldCreateMetricsAtStartup(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
thread = createStreamThread(CLIENT_ID, new MockTime(1), stateUpdaterEnabled, processingThreadsEnabled);
|
thread = createStreamThread(CLIENT_ID, new MockTime(1), stateUpdaterEnabled, processingThreadsEnabled);
|
||||||
final String defaultGroupName = "stream-thread-metrics";
|
final String defaultGroupName = "stream-thread-metrics";
|
||||||
|
@ -538,7 +539,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldNotCommitBeforeTheCommitInterval(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldNotCommitBeforeTheCommitInterval(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final long commitInterval = 1000L;
|
final long commitInterval = 1000L;
|
||||||
final Properties props = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
|
final Properties props = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
|
||||||
|
@ -565,7 +566,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldNotPurgeBeforeThePurgeInterval(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldNotPurgeBeforeThePurgeInterval(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final long commitInterval = 1000L;
|
final long commitInterval = 1000L;
|
||||||
final long purgeInterval = 2000L;
|
final long purgeInterval = 2000L;
|
||||||
|
@ -593,7 +594,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldAlsoPurgeWhenNothingGetsCommitted(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldAlsoPurgeWhenNothingGetsCommitted(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final long purgeInterval = 1000L;
|
final long purgeInterval = 1000L;
|
||||||
final Properties props = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
|
final Properties props = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
|
||||||
|
@ -658,7 +659,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldNotProcessWhenPartitionRevoked(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldNotProcessWhenPartitionRevoked(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
assumeFalse(processingThreadsEnabled);
|
assumeFalse(processingThreadsEnabled);
|
||||||
|
|
||||||
|
@ -682,7 +683,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldProcessWhenRunning(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldProcessWhenRunning(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
assumeFalse(processingThreadsEnabled);
|
assumeFalse(processingThreadsEnabled);
|
||||||
final Properties props = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
|
final Properties props = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
|
||||||
|
@ -707,7 +708,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldProcessWhenPartitionAssigned(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldProcessWhenPartitionAssigned(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
assumeTrue(stateUpdaterEnabled);
|
assumeTrue(stateUpdaterEnabled);
|
||||||
assumeFalse(processingThreadsEnabled);
|
assumeFalse(processingThreadsEnabled);
|
||||||
|
@ -732,7 +733,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldProcessWhenStarting(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldProcessWhenStarting(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
assumeTrue(stateUpdaterEnabled);
|
assumeTrue(stateUpdaterEnabled);
|
||||||
assumeFalse(processingThreadsEnabled);
|
assumeFalse(processingThreadsEnabled);
|
||||||
|
@ -757,7 +758,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldEnforceRebalanceWhenScheduledAndNotCurrentlyRebalancing(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws InterruptedException {
|
public void shouldEnforceRebalanceWhenScheduledAndNotCurrentlyRebalancing(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws InterruptedException {
|
||||||
final Time mockTime = new MockTime(1);
|
final Time mockTime = new MockTime(1);
|
||||||
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
||||||
|
@ -812,7 +813,7 @@ public class StreamThreadTest {
|
||||||
10 * 1000,
|
10 * 1000,
|
||||||
"Thread never started.");
|
"Thread never started.");
|
||||||
|
|
||||||
thread.shutdown(true);
|
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
|
||||||
TestUtils.waitForCondition(
|
TestUtils.waitForCondition(
|
||||||
() -> thread.state() == StreamThread.State.DEAD,
|
() -> thread.state() == StreamThread.State.DEAD,
|
||||||
10 * 1000,
|
10 * 1000,
|
||||||
|
@ -822,7 +823,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldNotEnforceRebalanceWhenCurrentlyRebalancing(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws InterruptedException {
|
public void shouldNotEnforceRebalanceWhenCurrentlyRebalancing(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws InterruptedException {
|
||||||
final Time mockTime = new MockTime(1);
|
final Time mockTime = new MockTime(1);
|
||||||
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
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
|
// 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
|
// don't trigger one before we can shut down, since the rebalance must be ended
|
||||||
|
@ -918,7 +919,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldRespectNumIterationsInMainLoopWithoutProcessingThreads(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldRespectNumIterationsInMainLoopWithoutProcessingThreads(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
// With processing threads, there is no guarantee how many iterations will be performed
|
// With processing threads, there is no guarantee how many iterations will be performed
|
||||||
assumeFalse(processingThreadsEnabled);
|
assumeFalse(processingThreadsEnabled);
|
||||||
|
@ -1047,7 +1048,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldNotCauseExceptionIfNothingCommitted(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldNotCauseExceptionIfNothingCommitted(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final long commitInterval = 1000L;
|
final long commitInterval = 1000L;
|
||||||
final Properties props = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
|
final Properties props = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
|
||||||
|
@ -1076,7 +1077,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldCommitAfterCommitInterval(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldCommitAfterCommitInterval(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final long commitInterval = 100L;
|
final long commitInterval = 100L;
|
||||||
final long commitLatency = 10L;
|
final long commitLatency = 10L;
|
||||||
|
@ -1137,7 +1138,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldPurgeAfterPurgeInterval(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldPurgeAfterPurgeInterval(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final long commitInterval = 100L;
|
final long commitInterval = 100L;
|
||||||
final long purgeInterval = 200L;
|
final long purgeInterval = 200L;
|
||||||
|
@ -1170,7 +1171,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldRecordCommitLatency(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldRecordCommitLatency(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class);
|
final ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class);
|
||||||
when(consumer.groupMetadata()).thenReturn(consumerGroupMetadata);
|
when(consumer.groupMetadata()).thenReturn(consumerGroupMetadata);
|
||||||
|
@ -1279,7 +1280,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldInjectSharedProducerForAllTasksUsingClientSupplierOnCreateIfEosDisabled(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldInjectSharedProducerForAllTasksUsingClientSupplierOnCreateIfEosDisabled(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
internalTopologyBuilder.addSource(null, "source1", null, null, null, topic1);
|
internalTopologyBuilder.addSource(null, "source1", null, null, null, topic1);
|
||||||
internalStreamsBuilder.buildAndOptimizeTopology();
|
internalStreamsBuilder.buildAndOptimizeTopology();
|
||||||
|
@ -1319,7 +1320,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldInjectProducerPerThreadUsingClientSupplierOnCreateIfEosV2Enabled(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldInjectProducerPerThreadUsingClientSupplierOnCreateIfEosV2Enabled(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
internalTopologyBuilder.addSource(null, "source1", null, null, null, topic1);
|
internalTopologyBuilder.addSource(null, "source1", null, null, null, topic1);
|
||||||
|
|
||||||
|
@ -1357,7 +1358,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldOnlyCompleteShutdownAfterRebalanceNotInProgress(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws InterruptedException {
|
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()
|
// 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
|
// 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,
|
10 * 1000,
|
||||||
"Thread never started.");
|
"Thread never started.");
|
||||||
|
|
||||||
thread.shutdown(true);
|
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
|
||||||
|
|
||||||
// even if thread is no longer running, it should still be polling
|
// even if thread is no longer running, it should still be polling
|
||||||
// as long as the rebalance is still ongoing
|
// as long as the rebalance is still ongoing
|
||||||
|
@ -1411,7 +1412,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldShutdownTaskManagerOnClose(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldShutdownTaskManagerOnClose(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class);
|
final ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class);
|
||||||
when(consumer.groupMetadata()).thenReturn(consumerGroupMetadata);
|
when(consumer.groupMetadata()).thenReturn(consumerGroupMetadata);
|
||||||
|
@ -1426,7 +1427,7 @@ public class StreamThreadTest {
|
||||||
thread.setStateListener(
|
thread.setStateListener(
|
||||||
(t, newState, oldState) -> {
|
(t, newState, oldState) -> {
|
||||||
if (oldState == StreamThread.State.CREATED && newState == StreamThread.State.STARTING) {
|
if (oldState == StreamThread.State.CREATED && newState == StreamThread.State.STARTING) {
|
||||||
thread.shutdown(true);
|
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
thread.run();
|
thread.run();
|
||||||
|
@ -1435,7 +1436,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldNotReturnDataAfterTaskMigrated(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldNotReturnDataAfterTaskMigrated(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final TaskManager taskManager = mock(TaskManager.class);
|
final TaskManager taskManager = mock(TaskManager.class);
|
||||||
final InternalTopologyBuilder internalTopologyBuilder = mock(InternalTopologyBuilder.class);
|
final InternalTopologyBuilder internalTopologyBuilder = mock(InternalTopologyBuilder.class);
|
||||||
|
@ -1512,7 +1513,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldShutdownTaskManagerOnCloseWithoutStart(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldShutdownTaskManagerOnCloseWithoutStart(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class);
|
final ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class);
|
||||||
when(consumer.groupMetadata()).thenReturn(consumerGroupMetadata);
|
when(consumer.groupMetadata()).thenReturn(consumerGroupMetadata);
|
||||||
|
@ -1524,13 +1525,13 @@ public class StreamThreadTest {
|
||||||
topologyMetadata.buildAndRewriteTopology();
|
topologyMetadata.buildAndRewriteTopology();
|
||||||
thread = buildStreamThread(consumer, taskManager, config, topologyMetadata)
|
thread = buildStreamThread(consumer, taskManager, config, topologyMetadata)
|
||||||
.updateThreadMetadata(adminClientId(CLIENT_ID));
|
.updateThreadMetadata(adminClientId(CLIENT_ID));
|
||||||
thread.shutdown(true);
|
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
|
||||||
|
|
||||||
verify(taskManager).shutdown(true);
|
verify(taskManager).shutdown(true);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldOnlyShutdownOnce(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldOnlyShutdownOnce(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class);
|
final ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class);
|
||||||
when(consumer.groupMetadata()).thenReturn(consumerGroupMetadata);
|
when(consumer.groupMetadata()).thenReturn(consumerGroupMetadata);
|
||||||
|
@ -1542,7 +1543,7 @@ public class StreamThreadTest {
|
||||||
topologyMetadata.buildAndRewriteTopology();
|
topologyMetadata.buildAndRewriteTopology();
|
||||||
thread = buildStreamThread(consumer, taskManager, config, topologyMetadata)
|
thread = buildStreamThread(consumer, taskManager, config, topologyMetadata)
|
||||||
.updateThreadMetadata(adminClientId(CLIENT_ID));
|
.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
|
// Execute the run method. Verification of the mock will check that shutdown was only done once
|
||||||
thread.run();
|
thread.run();
|
||||||
|
|
||||||
|
@ -1550,7 +1551,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldNotThrowWhenStandbyTasksAssignedAndNoStateStoresForTopology(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldNotThrowWhenStandbyTasksAssignedAndNoStateStoresForTopology(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
internalTopologyBuilder.addSource(null, "name", null, null, null, "topic");
|
internalTopologyBuilder.addSource(null, "name", null, null, null, "topic");
|
||||||
internalTopologyBuilder.addSink("out", "output", null, null, null, "name");
|
internalTopologyBuilder.addSink("out", "output", null, null, null, "name");
|
||||||
|
@ -1572,7 +1573,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldNotCloseTaskAndRemoveFromTaskManagerIfProducerWasFencedWhileProcessing(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
|
public void shouldNotCloseTaskAndRemoveFromTaskManagerIfProducerWasFencedWhileProcessing(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
|
||||||
internalTopologyBuilder.addSource(null, "source", null, null, null, topic1);
|
internalTopologyBuilder.addSource(null, "source", null, null, null, topic1);
|
||||||
internalTopologyBuilder.addSink("sink", "dummyTopic", null, null, null, "source");
|
internalTopologyBuilder.addSink("sink", "dummyTopic", null, null, null, "source");
|
||||||
|
@ -1688,18 +1689,18 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldNotCloseTaskAndRemoveFromTaskManagerIfProducerGotFencedInCommitTransactionWhenSuspendingTasks(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
|
public void shouldNotCloseTaskAndRemoveFromTaskManagerIfProducerGotFencedInCommitTransactionWhenSuspendingTasks(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
|
||||||
testThrowingDurringCommitTransactionException(new ProducerFencedException("Producer is fenced"), stateUpdaterEnabled, processingThreadsEnabled);
|
testThrowingDurringCommitTransactionException(new ProducerFencedException("Producer is fenced"), stateUpdaterEnabled, processingThreadsEnabled);
|
||||||
}
|
}
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldNotCloseTaskAndRemoveFromTaskManagerIfInvalidPidMappingOccurredInCommitTransactionWhenSuspendingTasks(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
|
public void shouldNotCloseTaskAndRemoveFromTaskManagerIfInvalidPidMappingOccurredInCommitTransactionWhenSuspendingTasks(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
|
||||||
testThrowingDurringCommitTransactionException(new InvalidPidMappingException("PidMapping is invalid"), stateUpdaterEnabled, processingThreadsEnabled);
|
testThrowingDurringCommitTransactionException(new InvalidPidMappingException("PidMapping is invalid"), stateUpdaterEnabled, processingThreadsEnabled);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldReinitializeRevivedTasksInAnyState(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
|
public void shouldReinitializeRevivedTasksInAnyState(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
|
||||||
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
||||||
thread = createStreamThread(CLIENT_ID, config, new MockTime(1));
|
thread = createStreamThread(CLIENT_ID, config, new MockTime(1));
|
||||||
|
@ -1873,19 +1874,19 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldNotCloseTaskAndRemoveFromTaskManagerIfProducerGotFencedInCommitTransactionWhenCommitting(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldNotCloseTaskAndRemoveFromTaskManagerIfProducerGotFencedInCommitTransactionWhenCommitting(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
testNotCloseTaskAndRemoveFromTaskManagerInCommitTransactionWhenCommitting(new ProducerFencedException("Producer is fenced"), stateUpdaterEnabled, processingThreadsEnabled);
|
testNotCloseTaskAndRemoveFromTaskManagerInCommitTransactionWhenCommitting(new ProducerFencedException("Producer is fenced"), stateUpdaterEnabled, processingThreadsEnabled);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldNotCloseTaskAndRemoveFromTaskManagerIfPidMappingIsInvalidInCommitTransactionWhenCommitting(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldNotCloseTaskAndRemoveFromTaskManagerIfPidMappingIsInvalidInCommitTransactionWhenCommitting(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
testNotCloseTaskAndRemoveFromTaskManagerInCommitTransactionWhenCommitting(new InvalidPidMappingException("PID Mapping is invalid"), stateUpdaterEnabled, processingThreadsEnabled);
|
testNotCloseTaskAndRemoveFromTaskManagerInCommitTransactionWhenCommitting(new InvalidPidMappingException("PID Mapping is invalid"), stateUpdaterEnabled, processingThreadsEnabled);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldNotCloseTaskProducerWhenSuspending(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
|
public void shouldNotCloseTaskProducerWhenSuspending(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
|
||||||
final StreamsConfig config = new StreamsConfig(configProps(true, stateUpdaterEnabled, processingThreadsEnabled));
|
final StreamsConfig config = new StreamsConfig(configProps(true, stateUpdaterEnabled, processingThreadsEnabled));
|
||||||
thread = createStreamThread(CLIENT_ID, config);
|
thread = createStreamThread(CLIENT_ID, config);
|
||||||
|
@ -1933,7 +1934,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldReturnActiveTaskMetadataWhileRunningState(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldReturnActiveTaskMetadataWhileRunningState(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
||||||
internalTopologyBuilder.addSource(null, "source", null, null, null, topic1);
|
internalTopologyBuilder.addSource(null, "source", null, null, null, topic1);
|
||||||
|
@ -2011,7 +2012,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldReturnStandbyTaskMetadataWhileRunningState(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldReturnStandbyTaskMetadataWhileRunningState(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
||||||
internalStreamsBuilder.stream(Collections.singleton(topic1), consumed)
|
internalStreamsBuilder.stream(Collections.singleton(topic1), consumed)
|
||||||
|
@ -2059,7 +2060,7 @@ public class StreamThreadTest {
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldUpdateStandbyTask(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
|
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
|
// Updating standby tasks on the stream thread only happens when the state updater is disabled
|
||||||
assumeFalse(stateUpdaterEnabled);
|
assumeFalse(stateUpdaterEnabled);
|
||||||
|
@ -2183,7 +2184,7 @@ public class StreamThreadTest {
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldNotUpdateStandbyTaskWhenPaused(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
|
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
|
// Updating standby tasks on the stream thread only happens when the state updater is disabled
|
||||||
assumeFalse(stateUpdaterEnabled);
|
assumeFalse(stateUpdaterEnabled);
|
||||||
|
@ -2243,7 +2244,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldCreateStandbyTask(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldCreateStandbyTask(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
||||||
setupInternalTopologyWithoutState(config);
|
setupInternalTopologyWithoutState(config);
|
||||||
|
@ -2253,7 +2254,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldNotCreateStandbyTaskWithoutStateStores(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldNotCreateStandbyTaskWithoutStateStores(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
||||||
setupInternalTopologyWithoutState(config);
|
setupInternalTopologyWithoutState(config);
|
||||||
|
@ -2262,7 +2263,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldNotCreateStandbyTaskIfStateStoresHaveLoggingDisabled(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldNotCreateStandbyTaskIfStateStoresHaveLoggingDisabled(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
||||||
setupInternalTopologyWithoutState(config);
|
setupInternalTopologyWithoutState(config);
|
||||||
|
@ -2275,7 +2276,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
@SuppressWarnings("deprecation")
|
@SuppressWarnings("deprecation")
|
||||||
public void shouldPunctuateActiveTask(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldPunctuateActiveTask(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
assumeFalse(processingThreadsEnabled);
|
assumeFalse(processingThreadsEnabled);
|
||||||
|
@ -2426,7 +2427,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldAlwaysUpdateTasksMetadataAfterChangingState(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldAlwaysUpdateTasksMetadataAfterChangingState(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
||||||
thread = createStreamThread(CLIENT_ID, config);
|
thread = createStreamThread(CLIENT_ID, config);
|
||||||
|
@ -2442,7 +2443,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldRecoverFromInvalidOffsetExceptionOnRestoreAndFinishRestore(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
|
public void shouldRecoverFromInvalidOffsetExceptionOnRestoreAndFinishRestore(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
|
||||||
internalStreamsBuilder.stream(Collections.singleton("topic"), consumed)
|
internalStreamsBuilder.stream(Collections.singleton("topic"), consumed)
|
||||||
.groupByKey()
|
.groupByKey()
|
||||||
|
@ -2632,7 +2633,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldThrowTaskMigratedExceptionHandlingTaskLost(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldThrowTaskMigratedExceptionHandlingTaskLost(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
||||||
final Set<TopicPartition> assignedPartitions = Collections.singleton(t1p1);
|
final Set<TopicPartition> assignedPartitions = Collections.singleton(t1p1);
|
||||||
|
@ -2660,7 +2661,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldThrowTaskMigratedExceptionHandlingRevocation(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldThrowTaskMigratedExceptionHandlingRevocation(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
||||||
final Set<TopicPartition> assignedPartitions = Collections.singleton(t1p1);
|
final Set<TopicPartition> assignedPartitions = Collections.singleton(t1p1);
|
||||||
|
@ -2688,7 +2689,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
public void shouldCatchHandleCorruptionOnTaskCorruptedExceptionPath(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldCatchHandleCorruptionOnTaskCorruptedExceptionPath(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
||||||
|
@ -2749,7 +2750,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
public void shouldCatchTimeoutExceptionFromHandleCorruptionAndInvokeExceptionHandler(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldCatchTimeoutExceptionFromHandleCorruptionAndInvokeExceptionHandler(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
||||||
|
@ -2815,7 +2816,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
public void shouldCatchTaskMigratedExceptionOnOnTaskCorruptedExceptionPath(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldCatchTaskMigratedExceptionOnOnTaskCorruptedExceptionPath(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
||||||
|
@ -2881,7 +2882,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
public void shouldEnforceRebalanceWhenTaskCorruptedExceptionIsThrownForAnActiveTask(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldEnforceRebalanceWhenTaskCorruptedExceptionIsThrownForAnActiveTask(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final StreamsConfig config = new StreamsConfig(configProps(true, stateUpdaterEnabled, processingThreadsEnabled));
|
final StreamsConfig config = new StreamsConfig(configProps(true, stateUpdaterEnabled, processingThreadsEnabled));
|
||||||
|
@ -2946,7 +2947,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
public void shouldNotEnforceRebalanceWhenTaskCorruptedExceptionIsThrownForAnInactiveTask(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldNotEnforceRebalanceWhenTaskCorruptedExceptionIsThrownForAnInactiveTask(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final StreamsConfig config = new StreamsConfig(configProps(true, stateUpdaterEnabled, processingThreadsEnabled));
|
final StreamsConfig config = new StreamsConfig(configProps(true, stateUpdaterEnabled, processingThreadsEnabled));
|
||||||
|
@ -3009,7 +3010,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldNotCommitNonRunningNonRestoringTasks(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldNotCommitNonRunningNonRestoringTasks(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
||||||
final TaskManager taskManager = mock(TaskManager.class);
|
final TaskManager taskManager = mock(TaskManager.class);
|
||||||
|
@ -3048,7 +3049,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldLogAndRecordSkippedRecordsForInvalidTimestamps(
|
public void shouldLogAndRecordSkippedRecordsForInvalidTimestamps(
|
||||||
final boolean stateUpdaterEnabled,
|
final boolean stateUpdaterEnabled,
|
||||||
final boolean processingThreadsEnabled
|
final boolean processingThreadsEnabled
|
||||||
|
@ -3155,7 +3156,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldTransmitTaskManagerMetrics(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldTransmitTaskManagerMetrics(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
||||||
final ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class);
|
final ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class);
|
||||||
|
@ -3182,7 +3183,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldConstructAdminMetrics(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldConstructAdminMetrics(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
|
||||||
final Node broker1 = new Node(0, "dummyHost-1", 1234);
|
final Node broker1 = new Node(0, "dummyHost-1", 1234);
|
||||||
|
@ -3239,13 +3240,13 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldNotRecordFailedStreamThread(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldNotRecordFailedStreamThread(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
runAndVerifyFailedStreamThreadRecording(false, stateUpdaterEnabled, processingThreadsEnabled);
|
runAndVerifyFailedStreamThreadRecording(false, stateUpdaterEnabled, processingThreadsEnabled);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldRecordFailedStreamThread(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldRecordFailedStreamThread(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
runAndVerifyFailedStreamThreadRecording(true, stateUpdaterEnabled, processingThreadsEnabled);
|
runAndVerifyFailedStreamThreadRecording(true, stateUpdaterEnabled, processingThreadsEnabled);
|
||||||
}
|
}
|
||||||
|
@ -3308,7 +3309,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldCheckStateUpdater(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldCheckStateUpdater(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
assumeTrue(stateUpdaterEnabled);
|
assumeTrue(stateUpdaterEnabled);
|
||||||
final Properties streamsConfigProps = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
|
final Properties streamsConfigProps = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
|
||||||
|
@ -3326,7 +3327,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldCheckStateUpdaterInBetweenProcessCalls(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldCheckStateUpdaterInBetweenProcessCalls(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
assumeTrue(stateUpdaterEnabled);
|
assumeTrue(stateUpdaterEnabled);
|
||||||
assumeFalse(processingThreadsEnabled);
|
assumeFalse(processingThreadsEnabled);
|
||||||
|
@ -3344,7 +3345,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldUpdateLagsAfterPolling(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldUpdateLagsAfterPolling(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final Properties streamsConfigProps = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
|
final Properties streamsConfigProps = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
|
||||||
thread = setUpThread(streamsConfigProps);
|
thread = setUpThread(streamsConfigProps);
|
||||||
|
@ -3362,7 +3363,7 @@ public class StreamThreadTest {
|
||||||
|
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldResumePollingForPartitionsWithAvailableSpaceBeforePolling(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldResumePollingForPartitionsWithAvailableSpaceBeforePolling(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final Properties streamsConfigProps = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
|
final Properties streamsConfigProps = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
|
||||||
thread = setUpThread(streamsConfigProps);
|
thread = setUpThread(streamsConfigProps);
|
||||||
|
@ -3377,7 +3378,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldRespectPollTimeInPartitionsAssignedStateWithStateUpdater(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldRespectPollTimeInPartitionsAssignedStateWithStateUpdater(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
assumeTrue(stateUpdaterEnabled);
|
assumeTrue(stateUpdaterEnabled);
|
||||||
final Properties streamsConfigProps = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
|
final Properties streamsConfigProps = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
|
||||||
|
@ -3393,7 +3394,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldNotBlockWhenPollingInPartitionsAssignedStateWithoutStateUpdater(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldNotBlockWhenPollingInPartitionsAssignedStateWithoutStateUpdater(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
assumeFalse(stateUpdaterEnabled);
|
assumeFalse(stateUpdaterEnabled);
|
||||||
final Properties streamsConfigProps = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
|
final Properties streamsConfigProps = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
|
||||||
|
@ -3407,13 +3408,13 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldGetMainAndRestoreConsumerInstanceId(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
|
public void shouldGetMainAndRestoreConsumerInstanceId(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
|
||||||
getClientInstanceId(false, stateUpdaterEnabled, processingThreadsEnabled);
|
getClientInstanceId(false, stateUpdaterEnabled, processingThreadsEnabled);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldGetMainAndRestoreConsumerInstanceIdWithInternalTimeout(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
|
public void shouldGetMainAndRestoreConsumerInstanceIdWithInternalTimeout(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
|
||||||
getClientInstanceId(true, stateUpdaterEnabled, processingThreadsEnabled);
|
getClientInstanceId(true, stateUpdaterEnabled, processingThreadsEnabled);
|
||||||
}
|
}
|
||||||
|
@ -3460,7 +3461,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldReturnErrorIfMainConsumerInstanceIdNotInitialized(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldReturnErrorIfMainConsumerInstanceIdNotInitialized(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
thread = createStreamThread("clientId", stateUpdaterEnabled, processingThreadsEnabled);
|
thread = createStreamThread("clientId", stateUpdaterEnabled, processingThreadsEnabled);
|
||||||
thread.setState(State.STARTING);
|
thread.setState(State.STARTING);
|
||||||
|
@ -3477,7 +3478,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldReturnErrorIfRestoreConsumerInstanceIdNotInitialized(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldReturnErrorIfRestoreConsumerInstanceIdNotInitialized(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
thread = createStreamThread("clientId", stateUpdaterEnabled, processingThreadsEnabled);
|
thread = createStreamThread("clientId", stateUpdaterEnabled, processingThreadsEnabled);
|
||||||
thread.setState(State.STARTING);
|
thread.setState(State.STARTING);
|
||||||
|
@ -3494,7 +3495,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldReturnErrorIfProducerInstanceIdNotInitialized(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldReturnErrorIfProducerInstanceIdNotInitialized(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
thread = createStreamThread("clientId", stateUpdaterEnabled, processingThreadsEnabled);
|
thread = createStreamThread("clientId", stateUpdaterEnabled, processingThreadsEnabled);
|
||||||
thread.setState(State.STARTING);
|
thread.setState(State.STARTING);
|
||||||
|
@ -3511,7 +3512,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldReturnNullIfMainConsumerTelemetryDisabled(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
|
public void shouldReturnNullIfMainConsumerTelemetryDisabled(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
|
||||||
clientSupplier.consumer.disableTelemetry();
|
clientSupplier.consumer.disableTelemetry();
|
||||||
thread = createStreamThread("clientId", stateUpdaterEnabled, processingThreadsEnabled);
|
thread = createStreamThread("clientId", stateUpdaterEnabled, processingThreadsEnabled);
|
||||||
|
@ -3528,7 +3529,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldReturnNullIfRestoreConsumerTelemetryDisabled(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
|
public void shouldReturnNullIfRestoreConsumerTelemetryDisabled(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
|
||||||
clientSupplier.restoreConsumer.disableTelemetry();
|
clientSupplier.restoreConsumer.disableTelemetry();
|
||||||
|
|
||||||
|
@ -3546,7 +3547,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldReturnNullIfProducerTelemetryDisabled(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
|
public void shouldReturnNullIfProducerTelemetryDisabled(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
|
||||||
final MockProducer<byte[], byte[]> producer = new MockProducer<>();
|
final MockProducer<byte[], byte[]> producer = new MockProducer<>();
|
||||||
producer.disableTelemetry();
|
producer.disableTelemetry();
|
||||||
|
@ -3566,7 +3567,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldTimeOutOnMainConsumerInstanceId(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldTimeOutOnMainConsumerInstanceId(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
clientSupplier.consumer.setClientInstanceId(Uuid.randomUuid());
|
clientSupplier.consumer.setClientInstanceId(Uuid.randomUuid());
|
||||||
clientSupplier.consumer.injectTimeoutException(-1);
|
clientSupplier.consumer.injectTimeoutException(-1);
|
||||||
|
@ -3591,7 +3592,7 @@ public class StreamThreadTest {
|
||||||
|
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldTimeOutOnRestoreConsumerInstanceId(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldTimeOutOnRestoreConsumerInstanceId(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
clientSupplier.restoreConsumer.setClientInstanceId(Uuid.randomUuid());
|
clientSupplier.restoreConsumer.setClientInstanceId(Uuid.randomUuid());
|
||||||
clientSupplier.restoreConsumer.injectTimeoutException(-1);
|
clientSupplier.restoreConsumer.injectTimeoutException(-1);
|
||||||
|
@ -3616,7 +3617,7 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void shouldTimeOutOnProducerInstanceId(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
public void shouldTimeOutOnProducerInstanceId(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
||||||
final MockProducer<byte[], byte[]> producer = new MockProducer<>();
|
final MockProducer<byte[], byte[]> producer = new MockProducer<>();
|
||||||
producer.setClientInstanceId(Uuid.randomUuid());
|
producer.setClientInstanceId(Uuid.randomUuid());
|
||||||
|
@ -3964,13 +3965,13 @@ public class StreamThreadTest {
|
||||||
.setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_SOURCE_TOPICS.code())
|
.setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_SOURCE_TOPICS.code())
|
||||||
.setStatusDetail("Missing source topics")
|
.setStatusDetail("Missing source topics")
|
||||||
));
|
));
|
||||||
|
|
||||||
// First call should not throw exception (within timeout)
|
// First call should not throw exception (within timeout)
|
||||||
thread.runOnceWithoutProcessingThreads();
|
thread.runOnceWithoutProcessingThreads();
|
||||||
|
|
||||||
// Advance time beyond max.poll.interval.ms (default is 300000ms) to trigger timeout
|
// Advance time beyond max.poll.interval.ms (default is 300000ms) to trigger timeout
|
||||||
mockTime.sleep(300001);
|
mockTime.sleep(300001);
|
||||||
|
|
||||||
final MissingSourceTopicException exception = assertThrows(MissingSourceTopicException.class, () -> thread.runOnceWithoutProcessingThreads());
|
final MissingSourceTopicException exception = assertThrows(MissingSourceTopicException.class, () -> thread.runOnceWithoutProcessingThreads());
|
||||||
assertTrue(exception.getMessage().contains("Missing source topics"));
|
assertTrue(exception.getMessage().contains("Missing source topics"));
|
||||||
assertTrue(exception.getMessage().contains("Timeout exceeded"));
|
assertTrue(exception.getMessage().contains("Timeout exceeded"));
|
||||||
|
@ -4032,7 +4033,7 @@ public class StreamThreadTest {
|
||||||
));
|
));
|
||||||
|
|
||||||
// Should immediately throw TopologyException (no timeout like MISSING_SOURCE_TOPICS)
|
// Should immediately throw TopologyException (no timeout like MISSING_SOURCE_TOPICS)
|
||||||
final TopologyException exception = assertThrows(TopologyException.class,
|
final TopologyException exception = assertThrows(TopologyException.class,
|
||||||
() -> thread.runOnceWithoutProcessingThreads());
|
() -> thread.runOnceWithoutProcessingThreads());
|
||||||
assertTrue(exception.getMessage().contains("Topics are incorrectly partitioned"));
|
assertTrue(exception.getMessage().contains("Topics are incorrectly partitioned"));
|
||||||
}
|
}
|
||||||
|
@ -4151,13 +4152,13 @@ public class StreamThreadTest {
|
||||||
.setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_SOURCE_TOPICS.code())
|
.setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_SOURCE_TOPICS.code())
|
||||||
.setStatusDetail("Missing source topics")
|
.setStatusDetail("Missing source topics")
|
||||||
));
|
));
|
||||||
|
|
||||||
// First call should not throw exception (within timeout)
|
// First call should not throw exception (within timeout)
|
||||||
thread.runOnceWithProcessingThreads();
|
thread.runOnceWithProcessingThreads();
|
||||||
|
|
||||||
// Advance time beyond max.poll.interval.ms (default is 300000ms) to trigger timeout
|
// Advance time beyond max.poll.interval.ms (default is 300000ms) to trigger timeout
|
||||||
mockTime.sleep(300001);
|
mockTime.sleep(300001);
|
||||||
|
|
||||||
final MissingSourceTopicException exception = assertThrows(MissingSourceTopicException.class, () -> thread.runOnceWithProcessingThreads());
|
final MissingSourceTopicException exception = assertThrows(MissingSourceTopicException.class, () -> thread.runOnceWithProcessingThreads());
|
||||||
assertTrue(exception.getMessage().contains("Missing source topics"));
|
assertTrue(exception.getMessage().contains("Missing source topics"));
|
||||||
assertTrue(exception.getMessage().contains("Timeout exceeded"));
|
assertTrue(exception.getMessage().contains("Timeout exceeded"));
|
||||||
|
@ -4219,35 +4220,35 @@ public class StreamThreadTest {
|
||||||
.setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_SOURCE_TOPICS.code())
|
.setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_SOURCE_TOPICS.code())
|
||||||
.setStatusDetail("Missing source topics")
|
.setStatusDetail("Missing source topics")
|
||||||
));
|
));
|
||||||
|
|
||||||
// First call should not throw exception (within timeout)
|
// First call should not throw exception (within timeout)
|
||||||
thread.runOnceWithoutProcessingThreads();
|
thread.runOnceWithoutProcessingThreads();
|
||||||
|
|
||||||
// Advance time but not beyond timeout
|
// Advance time but not beyond timeout
|
||||||
mockTime.sleep(150000); // Half of max.poll.interval.ms
|
mockTime.sleep(150000); // Half of max.poll.interval.ms
|
||||||
|
|
||||||
// Should still not throw exception
|
// Should still not throw exception
|
||||||
thread.runOnceWithoutProcessingThreads();
|
thread.runOnceWithoutProcessingThreads();
|
||||||
|
|
||||||
// Clear the missing source topics (simulate recovery)
|
// Clear the missing source topics (simulate recovery)
|
||||||
streamsRebalanceData.setStatuses(List.of());
|
streamsRebalanceData.setStatuses(List.of());
|
||||||
|
|
||||||
// Should complete without exception (recovery successful)
|
// Should complete without exception (recovery successful)
|
||||||
assertDoesNotThrow(() -> thread.runOnceWithoutProcessingThreads());
|
assertDoesNotThrow(() -> thread.runOnceWithoutProcessingThreads());
|
||||||
|
|
||||||
// Set missing topics again - should reset the timeout
|
// Set missing topics again - should reset the timeout
|
||||||
streamsRebalanceData.setStatuses(List.of(
|
streamsRebalanceData.setStatuses(List.of(
|
||||||
new StreamsGroupHeartbeatResponseData.Status()
|
new StreamsGroupHeartbeatResponseData.Status()
|
||||||
.setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_SOURCE_TOPICS.code())
|
.setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_SOURCE_TOPICS.code())
|
||||||
.setStatusDetail("Different missing topics")
|
.setStatusDetail("Different missing topics")
|
||||||
));
|
));
|
||||||
|
|
||||||
// Advance time by 250 seconds to test if timer was reset
|
// Advance time by 250 seconds to test if timer was reset
|
||||||
// Total time from beginning: 150000 + 250000 = 400000ms (400s)
|
// Total time from beginning: 150000 + 250000 = 400000ms (400s)
|
||||||
// If timer was NOT reset: elapsed time = 400s > 300s → should throw
|
// If timer was NOT reset: elapsed time = 400s > 300s → should throw
|
||||||
// If timer WAS reset: elapsed time = 250s < 300s → should NOT throw
|
// If timer WAS reset: elapsed time = 250s < 300s → should NOT throw
|
||||||
mockTime.sleep(250000); // Advance by 250 seconds
|
mockTime.sleep(250000); // Advance by 250 seconds
|
||||||
|
|
||||||
// Should not throw because timer was reset - only 250s elapsed from reset point
|
// Should not throw because timer was reset - only 250s elapsed from reset point
|
||||||
assertDoesNotThrow(() -> thread.runOnceWithoutProcessingThreads());
|
assertDoesNotThrow(() -> thread.runOnceWithoutProcessingThreads());
|
||||||
}
|
}
|
||||||
|
@ -4427,7 +4428,7 @@ public class StreamThreadTest {
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void runOnce(final boolean processingThreadsEnabled) {
|
private void runOnce(final boolean processingThreadsEnabled) {
|
||||||
if (processingThreadsEnabled) {
|
if (processingThreadsEnabled) {
|
||||||
thread.runOnceWithProcessingThreads();
|
thread.runOnceWithProcessingThreads();
|
||||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.kafka.common.protocol.Errors;
|
||||||
import org.apache.kafka.common.serialization.Serdes;
|
import org.apache.kafka.common.serialization.Serdes;
|
||||||
import org.apache.kafka.common.serialization.StringSerializer;
|
import org.apache.kafka.common.serialization.StringSerializer;
|
||||||
import org.apache.kafka.common.utils.Exit;
|
import org.apache.kafka.common.utils.Exit;
|
||||||
|
import org.apache.kafka.streams.CloseOptions;
|
||||||
import org.apache.kafka.streams.GroupProtocol;
|
import org.apache.kafka.streams.GroupProtocol;
|
||||||
import org.apache.kafka.streams.KafkaStreams;
|
import org.apache.kafka.streams.KafkaStreams;
|
||||||
import org.apache.kafka.streams.KeyValueTimestamp;
|
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 {
|
private void stopKSApp(String appId, String topic, KafkaStreams streams, StreamsGroupCommand.StreamsGroupService service) throws InterruptedException {
|
||||||
if (streams != null) {
|
if (streams != null) {
|
||||||
KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
|
CloseOptions closeOptions = CloseOptions.timeout(Duration.ofSeconds(30))
|
||||||
closeOptions.timeout(Duration.ofSeconds(30));
|
.withGroupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
|
||||||
closeOptions.leaveGroup(true);
|
|
||||||
streams.close(closeOptions);
|
streams.close(closeOptions);
|
||||||
streams.cleanUp();
|
streams.cleanUp();
|
||||||
|
|
||||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.kafka.common.serialization.Serdes;
|
||||||
import org.apache.kafka.common.serialization.StringSerializer;
|
import org.apache.kafka.common.serialization.StringSerializer;
|
||||||
import org.apache.kafka.common.utils.Exit;
|
import org.apache.kafka.common.utils.Exit;
|
||||||
import org.apache.kafka.common.utils.Utils;
|
import org.apache.kafka.common.utils.Utils;
|
||||||
|
import org.apache.kafka.streams.CloseOptions;
|
||||||
import org.apache.kafka.streams.GroupProtocol;
|
import org.apache.kafka.streams.GroupProtocol;
|
||||||
import org.apache.kafka.streams.KafkaStreams;
|
import org.apache.kafka.streams.KafkaStreams;
|
||||||
import org.apache.kafka.streams.KeyValueTimestamp;
|
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 {
|
private void stopKSApp(String appId, KafkaStreams streams, StreamsGroupCommand.StreamsGroupService service) throws InterruptedException {
|
||||||
if (streams != null) {
|
if (streams != null) {
|
||||||
KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
|
CloseOptions closeOptions = CloseOptions.timeout(Duration.ofSeconds(30))
|
||||||
closeOptions.timeout(Duration.ofSeconds(30));
|
.withGroupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
|
||||||
closeOptions.leaveGroup(true);
|
|
||||||
streams.close(closeOptions);
|
streams.close(closeOptions);
|
||||||
streams.cleanUp();
|
streams.cleanUp();
|
||||||
|
|
||||||
|
|
|
@ -23,7 +23,7 @@
|
||||||
// and not the version installed on the machine running the task.
|
// 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
|
// Read more about the wrapper here: https://docs.gradle.org/current/userguide/gradle_wrapper.html
|
||||||
wrapper {
|
wrapper {
|
||||||
gradleVersion = project.gradleVersion
|
gradleVersion = versions.gradle
|
||||||
}
|
}
|
||||||
|
|
||||||
// Custom task to inject support for downloading the gradle wrapper jar if it doesn't exist.
|
// 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 wrapperBasePath = "\$APP_HOME/gradle/wrapper"
|
||||||
def wrapperJarPath = wrapperBasePath + "/gradle-wrapper.jar"
|
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
|
// 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
|
// 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
|
// 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
|
// of curl (built against OpenSSL library that doesn't support TLSv1.2) would fail to
|
||||||
// fetch the jar.
|
// 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 wrapperJarUrl = wrapperBaseUrl + "/gradle-wrapper.jar"
|
||||||
|
|
||||||
def bootstrapString = """
|
def bootstrapString = """
|
||||||
|
@ -59,13 +57,15 @@ task bootstrapWrapper() {
|
||||||
done
|
done
|
||||||
""".stripIndent()
|
""".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 wrapperScript = wrapper.scriptFile
|
||||||
def wrapperLines = wrapperScript.readLines()
|
def wrapperLines = wrapperScript.readLines()
|
||||||
wrapperScript.withPrintWriter { out ->
|
wrapperScript.withPrintWriter { out ->
|
||||||
def bootstrapWritten = false
|
def bootstrapWritten = false
|
||||||
wrapperLines.each { line ->
|
wrapperLines.each { line ->
|
||||||
// Print the wrapper bootstrap before the first usage of the wrapper jar.
|
// 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)
|
out.println(bootstrapString)
|
||||||
bootstrapWritten = true
|
bootstrapWritten = true
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue