mirror of https://github.com/apache/kafka.git
Compare commits
36 Commits
c986d7b7d3
...
94c21c2ddd
Author | SHA1 | Date |
---|---|---|
|
94c21c2ddd | |
|
2938c4242e | |
|
ebae768bd8 | |
|
fa2496bb91 | |
|
c6bbbbe24d | |
|
f5a87b3703 | |
|
2da02d9fcf | |
|
162db130f6 | |
|
f68a149a18 | |
|
24cad50840 | |
|
71a7d85955 | |
|
611f4128b4 | |
|
d76442e5a6 | |
|
5736d506f0 | |
|
bc2f23b879 | |
|
0dd8471174 | |
|
7ddd0d7cce | |
|
68f1da8474 | |
|
28e7803037 | |
|
8468317dac | |
|
33cd114375 | |
|
7f65b1fa96 | |
|
0ddc69da70 | |
|
7426629ba4 | |
|
423330ebe7 | |
|
d1a821226c | |
|
1ebca7817b | |
|
71c5a426b8 | |
|
92169b8f08 | |
|
3c0843961b | |
|
c2aeec46a2 | |
|
e27ea8d4db | |
|
7d098cfbbd | |
|
60ad638a35 | |
|
d2a699954d | |
|
41611b4bd2 |
|
@ -42,7 +42,7 @@ runs:
|
|||
distribution: temurin
|
||||
java-version: ${{ inputs.java-version }}
|
||||
- name: Setup Gradle
|
||||
uses: gradle/actions/setup-gradle@94baf225fe0a508e581a564467443d0e2379123b # v4.3.0
|
||||
uses: gradle/actions/setup-gradle@748248ddd2a24f49513d8f472f81c3a07d4d50e1 # v4.4.4
|
||||
env:
|
||||
GRADLE_BUILD_ACTION_CACHE_DEBUG_ENABLED: true
|
||||
with:
|
||||
|
|
|
@ -38,7 +38,7 @@ run-name: Build Scans for ${{ github.event.workflow_run.display_title}}
|
|||
jobs:
|
||||
upload-build-scan:
|
||||
# Skip this workflow if the CI run was skipped or cancelled
|
||||
if: (github.event.workflow_run.conclusion == 'success' || github.event.workflow_run.conclusion == 'failure')
|
||||
if: (github.event.workflow_run.conclusion == 'success' || github.event.workflow_run.conclusion == 'failure') && github.event.workflow_run.head_branch != '4.0'
|
||||
runs-on: ubuntu-latest
|
||||
strategy:
|
||||
fail-fast: false
|
||||
|
|
|
@ -211,7 +211,7 @@ License Version 2.0:
|
|||
- commons-digester-2.1
|
||||
- commons-lang3-3.18.0
|
||||
- commons-logging-1.3.5
|
||||
- commons-validator-1.9.0
|
||||
- commons-validator-1.10.0
|
||||
- hash4j-0.22.0
|
||||
- jackson-annotations-2.19.0
|
||||
- jackson-core-2.19.0
|
||||
|
|
29
build.gradle
29
build.gradle
|
@ -29,22 +29,21 @@ buildscript {
|
|||
}
|
||||
|
||||
plugins {
|
||||
id 'com.github.ben-manes.versions' version '0.48.0'
|
||||
id 'com.github.ben-manes.versions' version '0.52.0'
|
||||
id 'idea'
|
||||
id 'jacoco'
|
||||
id 'java-library'
|
||||
id 'org.owasp.dependencycheck' version '8.2.1'
|
||||
id 'org.owasp.dependencycheck' version '12.1.3'
|
||||
id 'org.nosphere.apache.rat' version "0.8.1"
|
||||
id "io.swagger.core.v3.swagger-gradle-plugin" version "${swaggerVersion}"
|
||||
|
||||
id "com.github.spotbugs" version '6.2.3' apply false
|
||||
id "com.github.spotbugs" version '6.2.5' apply false
|
||||
id 'org.scoverage' version '8.0.3' apply false
|
||||
id 'com.gradleup.shadow' version '8.3.6' apply false
|
||||
id 'com.diffplug.spotless' version "6.25.0"
|
||||
id 'com.gradleup.shadow' version '8.3.9' apply false
|
||||
id 'com.diffplug.spotless' version "7.2.1"
|
||||
}
|
||||
|
||||
ext {
|
||||
gradleVersion = versions.gradle
|
||||
minClientJavaVersion = 11
|
||||
minNonClientJavaVersion = 17
|
||||
modulesNeedingJava11 = [":clients", ":generator", ":streams", ":streams:test-utils", ":streams:examples", ":streams-scala", ":test-common:test-common-util"]
|
||||
|
@ -193,8 +192,6 @@ allprojects {
|
|||
// ensure we have a single version in the classpath despite transitive dependencies
|
||||
libs.scalaLibrary,
|
||||
libs.scalaReflect,
|
||||
// Workaround before `commons-validator` has new release. See KAFKA-19359.
|
||||
libs.commonsBeanutils,
|
||||
libs.jacksonAnnotations,
|
||||
libs.commonsLang
|
||||
)
|
||||
|
@ -299,7 +296,7 @@ if (repo != null) {
|
|||
} else {
|
||||
rat.enabled = false
|
||||
}
|
||||
println("Starting build with version $version (commit id ${commitId == null ? "null" : commitId.take(8)}) using Gradle $gradleVersion, Java ${JavaVersion.current()} and Scala ${versions.scala}")
|
||||
println("Starting build with version $version (commit id ${commitId == null ? "null" : commitId.take(8)}) using Gradle $versions.gradle, Java ${JavaVersion.current()} and Scala ${versions.scala}")
|
||||
println("Build properties: ignoreFailures=$userIgnoreFailures, maxParallelForks=$maxTestForks, maxScalacThreads=$maxScalacThreads, maxTestRetries=$userMaxTestRetries")
|
||||
|
||||
subprojects {
|
||||
|
@ -330,6 +327,16 @@ subprojects {
|
|||
tasks.register('uploadArchives').configure { dependsOn(publish) }
|
||||
}
|
||||
|
||||
tasks.withType(AbstractArchiveTask).configureEach {
|
||||
reproducibleFileOrder = false
|
||||
preserveFileTimestamps = true
|
||||
useFileSystemPermissions()
|
||||
}
|
||||
|
||||
tasks.withType(AbstractTestTask).configureEach {
|
||||
failOnNoDiscoveredTests = false
|
||||
}
|
||||
|
||||
// apply the eclipse plugin only to subprojects that hold code. 'connect' is just a folder.
|
||||
if (!project.name.equals('connect')) {
|
||||
apply plugin: 'eclipse'
|
||||
|
@ -1041,6 +1048,8 @@ project(':core') {
|
|||
implementation project(':transaction-coordinator')
|
||||
implementation project(':metadata')
|
||||
implementation project(':storage:storage-api')
|
||||
// tools-api is automatically included in releaseTarGz via core's runtimeClasspath.
|
||||
// If removed from here, remember to explicitly add it back in the releaseTarGz task.
|
||||
implementation project(':tools:tools-api')
|
||||
implementation project(':raft')
|
||||
implementation project(':storage')
|
||||
|
@ -1263,8 +1272,6 @@ project(':core') {
|
|||
from(project(':streams:test-utils').configurations.runtimeClasspath) { into("libs/") }
|
||||
from(project(':streams:examples').jar) { into("libs/") }
|
||||
from(project(':streams:examples').configurations.runtimeClasspath) { into("libs/") }
|
||||
from(project(':tools:tools-api').jar) { into("libs/") }
|
||||
from(project(':tools:tools-api').configurations.runtimeClasspath) { into("libs/") }
|
||||
duplicatesStrategy 'exclude'
|
||||
}
|
||||
|
||||
|
|
|
@ -497,6 +497,7 @@
|
|||
<allow pkg="org.apache.kafka.server.common.serialization" />
|
||||
<allow pkg="org.apache.kafka.server.config" />
|
||||
<allow pkg="org.apache.kafka.server.fault"/>
|
||||
<allow pkg="org.apache.kafka.server.metrics" />
|
||||
<allow pkg="org.apache.kafka.server.util" />
|
||||
<allow pkg="org.apache.kafka.test"/>
|
||||
<allow pkg="com.fasterxml.jackson" />
|
||||
|
|
|
@ -0,0 +1,132 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.kafka.clients.admin;
|
||||
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.errors.InconsistentClusterIdException;
|
||||
import org.apache.kafka.common.test.KafkaClusterTestKit;
|
||||
import org.apache.kafka.common.test.TestKitNodes;
|
||||
import org.apache.kafka.test.TestUtils;
|
||||
|
||||
import org.junit.jupiter.api.Tag;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
@Tag("integration")
|
||||
public class ReconfigurableQuorumIntegrationTest {
|
||||
|
||||
static Map<Integer, Uuid> descVoterDirs(Admin admin) throws ExecutionException, InterruptedException {
|
||||
var quorumInfo = admin.describeMetadataQuorum().quorumInfo().get();
|
||||
return quorumInfo.voters().stream().collect(Collectors.toMap(QuorumInfo.ReplicaState::replicaId, QuorumInfo.ReplicaState::replicaDirectoryId));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRemoveAndAddVoterWithValidClusterId() throws Exception {
|
||||
final var nodes = new TestKitNodes.Builder()
|
||||
.setClusterId("test-cluster")
|
||||
.setNumBrokerNodes(1)
|
||||
.setNumControllerNodes(3)
|
||||
.build();
|
||||
|
||||
final Map<Integer, Uuid> initialVoters = new HashMap<>();
|
||||
for (final var controllerNode : nodes.controllerNodes().values()) {
|
||||
initialVoters.put(
|
||||
controllerNode.id(),
|
||||
controllerNode.metadataDirectoryId()
|
||||
);
|
||||
}
|
||||
|
||||
try (var cluster = new KafkaClusterTestKit.Builder(nodes).setInitialVoterSet(initialVoters).build()) {
|
||||
cluster.format();
|
||||
cluster.startup();
|
||||
try (Admin admin = Admin.create(cluster.clientProperties())) {
|
||||
TestUtils.waitForCondition(() -> {
|
||||
Map<Integer, Uuid> voters = descVoterDirs(admin);
|
||||
assertEquals(Set.of(3000, 3001, 3002), voters.keySet());
|
||||
return voters.values().stream().noneMatch(directory -> directory.equals(Uuid.ZERO_UUID));
|
||||
}, "Initial quorum voters should be {3000, 3001, 3002} and all should have non-zero directory IDs");
|
||||
|
||||
Uuid dirId = cluster.nodes().controllerNodes().get(3000).metadataDirectoryId();
|
||||
admin.removeRaftVoter(
|
||||
3000,
|
||||
dirId,
|
||||
new RemoveRaftVoterOptions().setClusterId(Optional.of("test-cluster"))
|
||||
).all().get();
|
||||
TestUtils.waitForCondition(() -> {
|
||||
Map<Integer, Uuid> voters = descVoterDirs(admin);
|
||||
assertEquals(Set.of(3001, 3002), voters.keySet());
|
||||
return voters.values().stream().noneMatch(directory -> directory.equals(Uuid.ZERO_UUID));
|
||||
}, "After removing voter 3000, remaining voters should be {3001, 3002} with non-zero directory IDs");
|
||||
|
||||
admin.addRaftVoter(
|
||||
3000,
|
||||
dirId,
|
||||
Set.of(new RaftVoterEndpoint("CONTROLLER", "example.com", 8080)),
|
||||
new AddRaftVoterOptions().setClusterId(Optional.of("test-cluster"))
|
||||
).all().get();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRemoveAndAddVoterWithInconsistentClusterId() throws Exception {
|
||||
final var nodes = new TestKitNodes.Builder()
|
||||
.setClusterId("test-cluster")
|
||||
.setNumBrokerNodes(1)
|
||||
.setNumControllerNodes(3)
|
||||
.build();
|
||||
|
||||
final Map<Integer, Uuid> initialVoters = new HashMap<>();
|
||||
for (final var controllerNode : nodes.controllerNodes().values()) {
|
||||
initialVoters.put(
|
||||
controllerNode.id(),
|
||||
controllerNode.metadataDirectoryId()
|
||||
);
|
||||
}
|
||||
|
||||
try (var cluster = new KafkaClusterTestKit.Builder(nodes).setInitialVoterSet(initialVoters).build()) {
|
||||
cluster.format();
|
||||
cluster.startup();
|
||||
try (Admin admin = Admin.create(cluster.clientProperties())) {
|
||||
Uuid dirId = cluster.nodes().controllerNodes().get(3000).metadataDirectoryId();
|
||||
var removeFuture = admin.removeRaftVoter(
|
||||
3000,
|
||||
dirId,
|
||||
new RemoveRaftVoterOptions().setClusterId(Optional.of("inconsistent"))
|
||||
).all();
|
||||
TestUtils.assertFutureThrows(InconsistentClusterIdException.class, removeFuture);
|
||||
|
||||
var addFuture = admin.addRaftVoter(
|
||||
3000,
|
||||
dirId,
|
||||
Set.of(new RaftVoterEndpoint("CONTROLLER", "example.com", 8080)),
|
||||
new AddRaftVoterOptions().setClusterId(Optional.of("inconsistent"))
|
||||
).all();
|
||||
TestUtils.assertFutureThrows(InconsistentClusterIdException.class, addFuture);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -28,6 +28,7 @@ import org.apache.kafka.common.test.api.ClusterTest;
|
|||
import org.apache.kafka.common.test.api.ClusterTestDefaults;
|
||||
import org.apache.kafka.common.test.api.Type;
|
||||
import org.apache.kafka.test.MockConsumerInterceptor;
|
||||
import org.apache.kafka.test.TestUtils;
|
||||
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
|
||||
|
@ -452,6 +453,40 @@ public class PlaintextConsumerCommitTest {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This is testing when closing the consumer but commit request has already been sent.
|
||||
* During the closing, the consumer won't find the coordinator anymore.
|
||||
*/
|
||||
@ClusterTest
|
||||
public void testCommitAsyncFailsWhenCoordinatorUnavailableDuringClose() throws InterruptedException {
|
||||
try (Producer<byte[], byte[]> producer = cluster.producer();
|
||||
var consumer = createConsumer(GroupProtocol.CONSUMER, false)
|
||||
) {
|
||||
sendRecords(producer, tp, 3, System.currentTimeMillis());
|
||||
consumer.assign(List.of(tp));
|
||||
|
||||
var callback = new CountConsumerCommitCallback();
|
||||
|
||||
// Close the coordinator before committing because otherwise the commit will fail to find the coordinator.
|
||||
cluster.brokerIds().forEach(cluster::shutdownBroker);
|
||||
|
||||
TestUtils.waitForCondition(() -> cluster.aliveBrokers().isEmpty(), "All brokers should be shut down");
|
||||
|
||||
consumer.poll(Duration.ofMillis(500));
|
||||
consumer.commitAsync(Map.of(tp, new OffsetAndMetadata(1L)), callback);
|
||||
|
||||
long startTime = System.currentTimeMillis();
|
||||
consumer.close(CloseOptions.timeout(Duration.ofMillis(500)));
|
||||
long closeDuration = System.currentTimeMillis() - startTime;
|
||||
|
||||
assertTrue(closeDuration < 1000, "The closing process for the consumer was too long: " + closeDuration + " ms");
|
||||
assertTrue(callback.lastError.isPresent());
|
||||
assertEquals(CommitFailedException.class, callback.lastError.get().getClass());
|
||||
assertEquals("Failed to commit offsets: Coordinator unknown and consumer is closing", callback.lastError.get().getMessage());
|
||||
assertEquals(1, callback.exceptionCount);
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: This only works in the new consumer, but should be fixed for the old consumer as well
|
||||
@ClusterTest
|
||||
public void testCommitAsyncCompletedBeforeConsumerCloses() throws InterruptedException {
|
||||
|
@ -575,6 +610,7 @@ public class PlaintextConsumerCommitTest {
|
|||
|
||||
private static class CountConsumerCommitCallback implements OffsetCommitCallback {
|
||||
private int successCount = 0;
|
||||
private int exceptionCount = 0;
|
||||
private Optional<Exception> lastError = Optional.empty();
|
||||
|
||||
@Override
|
||||
|
@ -582,6 +618,7 @@ public class PlaintextConsumerCommitTest {
|
|||
if (exception == null) {
|
||||
successCount += 1;
|
||||
} else {
|
||||
exceptionCount += 1;
|
||||
lastError = Optional.of(exception);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -46,7 +46,6 @@ public class GroupRebalanceConfig {
|
|||
public final Optional<String> rackId;
|
||||
public final long retryBackoffMs;
|
||||
public final long retryBackoffMaxMs;
|
||||
public final boolean leaveGroupOnClose;
|
||||
|
||||
public GroupRebalanceConfig(AbstractConfig config, ProtocolType protocolType) {
|
||||
this.sessionTimeoutMs = config.getInt(CommonClientConfigs.SESSION_TIMEOUT_MS_CONFIG);
|
||||
|
@ -80,13 +79,6 @@ public class GroupRebalanceConfig {
|
|||
|
||||
this.retryBackoffMs = config.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG);
|
||||
this.retryBackoffMaxMs = config.getLong(CommonClientConfigs.RETRY_BACKOFF_MAX_MS_CONFIG);
|
||||
|
||||
// Internal leave group config is only defined in Consumer.
|
||||
if (protocolType == ProtocolType.CONSUMER) {
|
||||
this.leaveGroupOnClose = config.getBoolean("internal.leave.group.on.close");
|
||||
} else {
|
||||
this.leaveGroupOnClose = true;
|
||||
}
|
||||
}
|
||||
|
||||
// For testing purpose.
|
||||
|
@ -97,8 +89,7 @@ public class GroupRebalanceConfig {
|
|||
Optional<String> groupInstanceId,
|
||||
String rackId,
|
||||
long retryBackoffMs,
|
||||
long retryBackoffMaxMs,
|
||||
boolean leaveGroupOnClose) {
|
||||
long retryBackoffMaxMs) {
|
||||
this.sessionTimeoutMs = sessionTimeoutMs;
|
||||
this.rebalanceTimeoutMs = rebalanceTimeoutMs;
|
||||
this.heartbeatIntervalMs = heartbeatIntervalMs;
|
||||
|
@ -107,6 +98,5 @@ public class GroupRebalanceConfig {
|
|||
this.rackId = rackId == null || rackId.isEmpty() ? Optional.empty() : Optional.of(rackId);
|
||||
this.retryBackoffMs = retryBackoffMs;
|
||||
this.retryBackoffMaxMs = retryBackoffMaxMs;
|
||||
this.leaveGroupOnClose = leaveGroupOnClose;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,11 +17,20 @@
|
|||
package org.apache.kafka.clients.admin;
|
||||
|
||||
import org.apache.kafka.common.annotation.InterfaceStability;
|
||||
import org.apache.kafka.common.protocol.Errors;
|
||||
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* Options for {@link Admin#addRaftVoter}.
|
||||
*
|
||||
* <p>
|
||||
* The clusterId is optional.
|
||||
* <p>
|
||||
* If provided, the request will only succeed if the cluster id matches the id of the current cluster.
|
||||
* If the cluster id does not match, the request will fail with {@link Errors#INCONSISTENT_CLUSTER_ID}.
|
||||
* <p>
|
||||
* If not provided, the cluster id check is skipped.
|
||||
*/
|
||||
@InterfaceStability.Stable
|
||||
public class AddRaftVoterOptions extends AbstractOptions<AddRaftVoterOptions> {
|
||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.kafka.common.acl.AclBindingFilter;
|
|||
import org.apache.kafka.common.annotation.InterfaceStability;
|
||||
import org.apache.kafka.common.config.ConfigResource;
|
||||
import org.apache.kafka.common.errors.FeatureUpdateFailedException;
|
||||
import org.apache.kafka.common.errors.InconsistentClusterIdException;
|
||||
import org.apache.kafka.common.errors.InterruptException;
|
||||
import org.apache.kafka.common.metrics.KafkaMetric;
|
||||
import org.apache.kafka.common.quota.ClientQuotaAlteration;
|
||||
|
@ -1866,10 +1867,17 @@ public interface Admin extends AutoCloseable {
|
|||
/**
|
||||
* Add a new voter node to the KRaft metadata quorum.
|
||||
*
|
||||
* <p>
|
||||
* The clusterId in {@link AddRaftVoterOptions} is optional.
|
||||
* If provided, the operation will only succeed if the cluster id matches the id
|
||||
* of the current cluster. If the cluster id does not match, the operation
|
||||
* will fail with {@link InconsistentClusterIdException}.
|
||||
* If not provided, the cluster id check is skipped.
|
||||
*
|
||||
* @param voterId The node ID of the voter.
|
||||
* @param voterDirectoryId The directory ID of the voter.
|
||||
* @param endpoints The endpoints that the new voter has.
|
||||
* @param options The options to use when adding the new voter node.
|
||||
* @param options Additional options for the operation, including optional cluster ID.
|
||||
*/
|
||||
AddRaftVoterResult addRaftVoter(
|
||||
int voterId,
|
||||
|
@ -1894,9 +1902,16 @@ public interface Admin extends AutoCloseable {
|
|||
/**
|
||||
* Remove a voter node from the KRaft metadata quorum.
|
||||
*
|
||||
* <p>
|
||||
* The clusterId in {@link RemoveRaftVoterOptions} is optional.
|
||||
* If provided, the operation will only succeed if the cluster id matches the id
|
||||
* of the current cluster. If the cluster id does not match, the operation
|
||||
* will fail with {@link InconsistentClusterIdException}.
|
||||
* If not provided, the cluster id check is skipped.
|
||||
*
|
||||
* @param voterId The node ID of the voter.
|
||||
* @param voterDirectoryId The directory ID of the voter.
|
||||
* @param options The options to use when removing the voter node.
|
||||
* @param options Additional options for the operation, including optional cluster ID.
|
||||
*/
|
||||
RemoveRaftVoterResult removeRaftVoter(
|
||||
int voterId,
|
||||
|
|
|
@ -46,7 +46,7 @@ public class ListShareGroupOffsetsResult {
|
|||
/**
|
||||
* Return the future when the requests for all groups succeed.
|
||||
*
|
||||
* @return Future which yields all {@code Map<String, Map<TopicPartition, Long>>} objects, if requests for all the groups succeed.
|
||||
* @return Future which yields all {@code Map<String, Map<TopicPartition, OffsetAndMetadata>>} objects, if requests for all the groups succeed.
|
||||
*/
|
||||
public KafkaFuture<Map<String, Map<TopicPartition, OffsetAndMetadata>>> all() {
|
||||
return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture<?>[0])).thenApply(
|
||||
|
|
|
@ -17,11 +17,20 @@
|
|||
package org.apache.kafka.clients.admin;
|
||||
|
||||
import org.apache.kafka.common.annotation.InterfaceStability;
|
||||
import org.apache.kafka.common.protocol.Errors;
|
||||
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* Options for {@link Admin#removeRaftVoter}.
|
||||
*
|
||||
* <p>
|
||||
* The clusterId is optional.
|
||||
* <p>
|
||||
* If provided, the request will only succeed if the cluster id matches the id of the current cluster.
|
||||
* If the cluster id does not match, the request will fail with {@link Errors#INCONSISTENT_CLUSTER_ID}.
|
||||
* <p>
|
||||
* If not provided, the cluster id check is skipped.
|
||||
*/
|
||||
@InterfaceStability.Stable
|
||||
public class RemoveRaftVoterOptions extends AbstractOptions<RemoveRaftVoterOptions> {
|
||||
|
|
|
@ -330,17 +330,6 @@ public class ConsumerConfig extends AbstractConfig {
|
|||
"be excluded from the subscription. It is always possible to explicitly subscribe to an internal topic.";
|
||||
public static final boolean DEFAULT_EXCLUDE_INTERNAL_TOPICS = true;
|
||||
|
||||
/**
|
||||
* <code>internal.leave.group.on.close</code>
|
||||
* Whether or not the consumer should leave the group on close. If set to <code>false</code> then a rebalance
|
||||
* won't occur until <code>session.timeout.ms</code> expires.
|
||||
*
|
||||
* <p>
|
||||
* Note: this is an internal configuration and could be changed in the future in a backward incompatible way
|
||||
*
|
||||
*/
|
||||
static final String LEAVE_GROUP_ON_CLOSE_CONFIG = "internal.leave.group.on.close";
|
||||
|
||||
/**
|
||||
* <code>internal.throw.on.fetch.stable.offset.unsupported</code>
|
||||
* Whether or not the consumer should throw when the new stable offset feature is supported.
|
||||
|
@ -634,10 +623,6 @@ public class ConsumerConfig extends AbstractConfig {
|
|||
DEFAULT_EXCLUDE_INTERNAL_TOPICS,
|
||||
Importance.MEDIUM,
|
||||
EXCLUDE_INTERNAL_TOPICS_DOC)
|
||||
.defineInternal(LEAVE_GROUP_ON_CLOSE_CONFIG,
|
||||
Type.BOOLEAN,
|
||||
true,
|
||||
Importance.LOW)
|
||||
.defineInternal(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED,
|
||||
Type.BOOLEAN,
|
||||
false,
|
||||
|
|
|
@ -1170,7 +1170,7 @@ public abstract class AbstractCoordinator implements Closeable {
|
|||
public synchronized RequestFuture<Void> maybeLeaveGroup(CloseOptions.GroupMembershipOperation membershipOperation, String leaveReason) {
|
||||
RequestFuture<Void> future = null;
|
||||
|
||||
if (rebalanceConfig.leaveGroupOnClose && shouldSendLeaveGroupRequest(membershipOperation)) {
|
||||
if (shouldSendLeaveGroupRequest(membershipOperation)) {
|
||||
log.info("Member {} sending LeaveGroup request to coordinator {} due to {}",
|
||||
generation.memberId, coordinator, leaveReason);
|
||||
LeaveGroupRequest.Builder request = new LeaveGroupRequest.Builder(
|
||||
|
|
|
@ -331,8 +331,7 @@ public class ClassicKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
|
|||
groupInstanceId,
|
||||
rackId,
|
||||
retryBackoffMs,
|
||||
retryBackoffMaxMs,
|
||||
true
|
||||
retryBackoffMaxMs
|
||||
);
|
||||
this.coordinator = new ConsumerCoordinator(
|
||||
rebalanceConfig,
|
||||
|
|
|
@ -181,6 +181,14 @@ public class CommitRequestManager implements RequestManager, MemberStateListener
|
|||
// poll when the coordinator node is known and fatal error is not present
|
||||
if (coordinatorRequestManager.coordinator().isEmpty()) {
|
||||
pendingRequests.maybeFailOnCoordinatorFatalError();
|
||||
|
||||
if (closing && pendingRequests.hasUnsentRequests()) {
|
||||
CommitFailedException exception = new CommitFailedException(
|
||||
"Failed to commit offsets: Coordinator unknown and consumer is closing");
|
||||
pendingRequests.drainPendingCommits()
|
||||
.forEach(request -> request.future().completeExceptionally(exception));
|
||||
}
|
||||
|
||||
return EMPTY;
|
||||
}
|
||||
|
||||
|
|
|
@ -421,7 +421,7 @@ public class ConsumerMembershipManager extends AbstractMembershipManager<Consume
|
|||
@Override
|
||||
public boolean isLeavingGroup() {
|
||||
CloseOptions.GroupMembershipOperation leaveGroupOperation = leaveGroupOperation();
|
||||
if (REMAIN_IN_GROUP == leaveGroupOperation) {
|
||||
if (REMAIN_IN_GROUP == leaveGroupOperation && groupInstanceId.isEmpty()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
|
@ -432,7 +432,8 @@ public class ConsumerMembershipManager extends AbstractMembershipManager<Consume
|
|||
boolean hasLeaveOperation = DEFAULT == leaveGroupOperation ||
|
||||
// Leave operation: both static and dynamic consumers will send a leave heartbeat
|
||||
LEAVE_GROUP == leaveGroupOperation ||
|
||||
// Remain in group: only static consumers will send a leave heartbeat, while dynamic members will not
|
||||
// Remain in group: static consumers will send a leave heartbeat with -2 epoch to reflect that a member using the given
|
||||
// instance id decided to leave the group and would be back within the session timeout.
|
||||
groupInstanceId().isPresent();
|
||||
|
||||
return isLeavingState && hasLeaveOperation;
|
||||
|
|
|
@ -20,13 +20,4 @@ package org.apache.kafka.common.metrics;
|
|||
* A gauge metric is an instantaneous reading of a particular value.
|
||||
*/
|
||||
@FunctionalInterface
|
||||
public interface Gauge<T> extends MetricValueProvider<T> {
|
||||
|
||||
/**
|
||||
* Returns the current value associated with this gauge.
|
||||
* @param config The configuration for this metric
|
||||
* @param now The POSIX time in milliseconds the measurement is being taken
|
||||
*/
|
||||
T value(MetricConfig config, long now);
|
||||
|
||||
}
|
||||
public interface Gauge<T> extends MetricValueProvider<T> { }
|
||||
|
|
|
@ -20,6 +20,8 @@ import org.apache.kafka.common.Metric;
|
|||
import org.apache.kafka.common.MetricName;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
public final class KafkaMetric implements Metric {
|
||||
|
||||
private final MetricName metricName;
|
||||
|
@ -41,9 +43,7 @@ public final class KafkaMetric implements Metric {
|
|||
MetricConfig config, Time time) {
|
||||
this.metricName = metricName;
|
||||
this.lock = lock;
|
||||
if (!(valueProvider instanceof Measurable) && !(valueProvider instanceof Gauge))
|
||||
throw new IllegalArgumentException("Unsupported metric value provider of class " + valueProvider.getClass());
|
||||
this.metricValueProvider = valueProvider;
|
||||
this.metricValueProvider = Objects.requireNonNull(valueProvider, "valueProvider must not be null");
|
||||
this.config = config;
|
||||
this.time = time;
|
||||
}
|
||||
|
@ -67,20 +67,15 @@ public final class KafkaMetric implements Metric {
|
|||
}
|
||||
|
||||
/**
|
||||
* Take the metric and return the value, which could be a {@link Measurable} or a {@link Gauge}
|
||||
* Take the metric and return the value via {@link MetricValueProvider#value(MetricConfig, long)}.
|
||||
*
|
||||
* @return Return the metric value
|
||||
* @throws IllegalStateException if the underlying metric is not a {@link Measurable} or a {@link Gauge}.
|
||||
*/
|
||||
@Override
|
||||
public Object metricValue() {
|
||||
long now = time.milliseconds();
|
||||
synchronized (this.lock) {
|
||||
if (isMeasurable())
|
||||
return ((Measurable) metricValueProvider).measure(config, now);
|
||||
else if (this.metricValueProvider instanceof Gauge)
|
||||
return ((Gauge<?>) metricValueProvider).value(config, now);
|
||||
else
|
||||
throw new IllegalStateException("Not a valid metric: " + this.metricValueProvider.getClass());
|
||||
return metricValueProvider.value(config, now);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -22,11 +22,26 @@ package org.apache.kafka.common.metrics;
|
|||
public interface Measurable extends MetricValueProvider<Double> {
|
||||
|
||||
/**
|
||||
* Measure this quantity and return the result as a double
|
||||
* Measure this quantity and return the result as a double.
|
||||
*
|
||||
* @param config The configuration for this metric
|
||||
* @param now The POSIX time in milliseconds the measurement is being taken
|
||||
* @return The measured value
|
||||
*/
|
||||
double measure(MetricConfig config, long now);
|
||||
|
||||
/**
|
||||
* Measure this quantity and return the result as a double.
|
||||
*
|
||||
* This default implementation delegates to {@link #measure(MetricConfig, long)}.
|
||||
*
|
||||
* @param config The configuration for this metric
|
||||
* @param now The POSIX time in milliseconds the measurement is being taken
|
||||
* @return The measured value as a {@link Double}
|
||||
*/
|
||||
@Override
|
||||
default Double value(MetricConfig config, long now) {
|
||||
return measure(config, now);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -19,10 +19,17 @@ package org.apache.kafka.common.metrics;
|
|||
/**
|
||||
* Super-interface for {@link Measurable} or {@link Gauge} that provides
|
||||
* metric values.
|
||||
* <p>
|
||||
* In the future for Java8 and above, {@link Gauge#value(MetricConfig, long)} will be
|
||||
* moved to this interface with a default implementation in {@link Measurable} that returns
|
||||
* {@link Measurable#measure(MetricConfig, long)}.
|
||||
* </p>
|
||||
*/
|
||||
public interface MetricValueProvider<T> { }
|
||||
@FunctionalInterface
|
||||
public interface MetricValueProvider<T> {
|
||||
|
||||
/**
|
||||
* Returns the current value associated with this metric.
|
||||
*
|
||||
* @param config The configuration for this metric
|
||||
* @param now The POSIX time in milliseconds the measurement is being taken
|
||||
* @return the current metric value
|
||||
*/
|
||||
T value(MetricConfig config, long now);
|
||||
|
||||
}
|
|
@ -125,7 +125,7 @@ public class ChannelBuilders {
|
|||
switch (securityProtocol) {
|
||||
case SSL:
|
||||
requireNonNullMode(connectionMode, securityProtocol);
|
||||
channelBuilder = new SslChannelBuilder(connectionMode, listenerName, isInterBrokerListener, logContext);
|
||||
channelBuilder = new SslChannelBuilder(connectionMode, listenerName, isInterBrokerListener);
|
||||
break;
|
||||
case SASL_SSL:
|
||||
case SASL_PLAINTEXT:
|
||||
|
|
|
@ -26,7 +26,6 @@ import org.apache.kafka.common.security.auth.KafkaPrincipalSerde;
|
|||
import org.apache.kafka.common.security.auth.SslAuthenticationContext;
|
||||
import org.apache.kafka.common.security.ssl.SslFactory;
|
||||
import org.apache.kafka.common.security.ssl.SslPrincipalMapper;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
|
||||
import java.io.Closeable;
|
||||
|
@ -53,8 +52,7 @@ public class SslChannelBuilder implements ChannelBuilder, ListenerReconfigurable
|
|||
*/
|
||||
public SslChannelBuilder(ConnectionMode connectionMode,
|
||||
ListenerName listenerName,
|
||||
boolean isInterBrokerListener,
|
||||
LogContext logContext) {
|
||||
boolean isInterBrokerListener) {
|
||||
this.connectionMode = connectionMode;
|
||||
this.listenerName = listenerName;
|
||||
this.isInterBrokerListener = isInterBrokerListener;
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.kafka.common.protocol;
|
|||
import org.apache.kafka.common.message.RequestHeaderData;
|
||||
import org.apache.kafka.common.message.ResponseHeaderData;
|
||||
import org.apache.kafka.common.protocol.types.BoundField;
|
||||
import org.apache.kafka.common.protocol.types.Field;
|
||||
import org.apache.kafka.common.protocol.types.Schema;
|
||||
import org.apache.kafka.common.protocol.types.TaggedFields;
|
||||
import org.apache.kafka.common.protocol.types.Type;
|
||||
|
@ -27,6 +28,7 @@ import java.util.LinkedHashMap;
|
|||
import java.util.LinkedHashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
|
||||
public class Protocol {
|
||||
|
||||
|
@ -49,7 +51,23 @@ public class Protocol {
|
|||
subTypes.put(field.def.name, type.arrayElementType().get());
|
||||
}
|
||||
} else if (type instanceof TaggedFields) {
|
||||
b.append("_tagged_fields ");
|
||||
Map<Integer, Field> taggedFields = new TreeMap<>(((TaggedFields) type).fields());
|
||||
taggedFields.forEach((tag, taggedField) -> {
|
||||
if (taggedField.type.isArray()) {
|
||||
b.append("[");
|
||||
b.append(taggedField.name);
|
||||
b.append("]");
|
||||
if (!subTypes.containsKey(taggedField.name))
|
||||
subTypes.put(taggedField.name + "<tag: " + tag.toString() + ">", taggedField.type.arrayElementType().get());
|
||||
} else {
|
||||
b.append(taggedField.name);
|
||||
if (!subTypes.containsKey(taggedField.name))
|
||||
subTypes.put(taggedField.name + "<tag: " + tag.toString() + ">", taggedField.type);
|
||||
}
|
||||
b.append("<tag: ");
|
||||
b.append(tag);
|
||||
b.append("> ");
|
||||
});
|
||||
} else {
|
||||
b.append(field.def.name);
|
||||
b.append(" ");
|
||||
|
@ -90,6 +108,12 @@ public class Protocol {
|
|||
}
|
||||
}
|
||||
|
||||
private static void appendFieldNameToTable(String name, StringBuilder b) {
|
||||
b.append("<td>");
|
||||
b.append(name);
|
||||
b.append("</td>");
|
||||
}
|
||||
|
||||
private static void schemaToFieldTableHtml(Schema schema, StringBuilder b) {
|
||||
Set<BoundField> fields = new LinkedHashSet<>();
|
||||
populateSchemaFields(schema, fields);
|
||||
|
@ -101,28 +125,12 @@ public class Protocol {
|
|||
b.append("</tr>");
|
||||
for (BoundField field : fields) {
|
||||
b.append("<tr>\n");
|
||||
b.append("<td>");
|
||||
b.append(field.def.name);
|
||||
b.append("</td>");
|
||||
b.append("<td>");
|
||||
if (field.def.type instanceof TaggedFields) {
|
||||
TaggedFields taggedFields = (TaggedFields) field.def.type;
|
||||
// Only include the field in the table if there are actually tags defined
|
||||
if (taggedFields.numFields() > 0) {
|
||||
b.append("<table class=\"data-table\"><tbody>\n");
|
||||
b.append("<tr>");
|
||||
b.append("<th>Tag</th>\n");
|
||||
b.append("<th>Tagged field</th>\n");
|
||||
b.append("<th>Description</th>\n");
|
||||
b.append("</tr>");
|
||||
taggedFields.fields().forEach((tag, taggedField) -> {
|
||||
b.append("<tr>\n");
|
||||
b.append("<td>");
|
||||
b.append(tag);
|
||||
b.append("</td>");
|
||||
b.append("<td>");
|
||||
b.append(taggedField.name);
|
||||
b.append("</td>");
|
||||
appendFieldNameToTable(taggedField.name + "<tag: " + tag.toString() + ">", b);
|
||||
b.append("<td>");
|
||||
b.append(taggedField.docString);
|
||||
if (taggedField.type.isArray()) {
|
||||
|
@ -136,11 +144,10 @@ public class Protocol {
|
|||
b.append("</td>");
|
||||
b.append("</tr>\n");
|
||||
});
|
||||
b.append("</tbody></table>\n");
|
||||
} else {
|
||||
b.append(field.def.docString);
|
||||
}
|
||||
} else {
|
||||
appendFieldNameToTable(field.def.name, b);
|
||||
b.append("<td>");
|
||||
b.append(field.def.docString);
|
||||
}
|
||||
b.append("</td>");
|
||||
|
|
|
@ -135,12 +135,7 @@ public class AbstractCoordinatorTest {
|
|||
Optional.empty(), Optional.empty());
|
||||
}
|
||||
|
||||
|
||||
private void setupCoordinator(int retryBackoffMs, int retryBackoffMaxMs, int rebalanceTimeoutMs, Optional<String> groupInstanceId, Optional<Supplier<BaseHeartbeatThread>> heartbeatThreadSupplier) {
|
||||
setupCoordinator(retryBackoffMs, retryBackoffMaxMs, rebalanceTimeoutMs, groupInstanceId, heartbeatThreadSupplier, groupInstanceId.isEmpty());
|
||||
}
|
||||
|
||||
private void setupCoordinator(int retryBackoffMs, int retryBackoffMaxMs, int rebalanceTimeoutMs, Optional<String> groupInstanceId, Optional<Supplier<BaseHeartbeatThread>> heartbeatThreadSupplier, boolean leaveOnClose) {
|
||||
LogContext logContext = new LogContext();
|
||||
this.mockTime = new MockTime();
|
||||
ConsumerMetadata metadata = new ConsumerMetadata(retryBackoffMs, retryBackoffMaxMs, 60 * 60 * 1000L,
|
||||
|
@ -168,8 +163,7 @@ public class AbstractCoordinatorTest {
|
|||
groupInstanceId,
|
||||
null,
|
||||
retryBackoffMs,
|
||||
retryBackoffMaxMs,
|
||||
leaveOnClose);
|
||||
retryBackoffMaxMs);
|
||||
this.coordinator = new DummyCoordinator(rebalanceConfig,
|
||||
consumerClient,
|
||||
metrics,
|
||||
|
@ -1109,7 +1103,7 @@ public class AbstractCoordinatorTest {
|
|||
@ParameterizedTest
|
||||
@MethodSource("groupInstanceIdAndMembershipOperationMatrix")
|
||||
public void testLeaveGroupSentWithGroupInstanceIdUnSetAndDifferentGroupMembershipOperation(Optional<String> groupInstanceId, CloseOptions.GroupMembershipOperation operation) {
|
||||
checkLeaveGroupRequestSent(groupInstanceId, operation, Optional.empty(), true);
|
||||
checkLeaveGroupRequestSent(groupInstanceId, operation, Optional.empty());
|
||||
}
|
||||
|
||||
private static Stream<Arguments> groupInstanceIdAndMembershipOperationMatrix() {
|
||||
|
@ -1124,11 +1118,11 @@ public class AbstractCoordinatorTest {
|
|||
}
|
||||
|
||||
private void checkLeaveGroupRequestSent(Optional<String> groupInstanceId) {
|
||||
checkLeaveGroupRequestSent(groupInstanceId, CloseOptions.GroupMembershipOperation.DEFAULT, Optional.empty(), groupInstanceId.isEmpty());
|
||||
checkLeaveGroupRequestSent(groupInstanceId, CloseOptions.GroupMembershipOperation.DEFAULT, Optional.empty());
|
||||
}
|
||||
|
||||
private void checkLeaveGroupRequestSent(Optional<String> groupInstanceId, CloseOptions.GroupMembershipOperation operation, Optional<Supplier<BaseHeartbeatThread>> heartbeatThreadSupplier, boolean leaveOnClose) {
|
||||
setupCoordinator(RETRY_BACKOFF_MS, RETRY_BACKOFF_MAX_MS, Integer.MAX_VALUE, groupInstanceId, heartbeatThreadSupplier, leaveOnClose);
|
||||
private void checkLeaveGroupRequestSent(Optional<String> groupInstanceId, CloseOptions.GroupMembershipOperation operation, Optional<Supplier<BaseHeartbeatThread>> heartbeatThreadSupplier) {
|
||||
setupCoordinator(RETRY_BACKOFF_MS, RETRY_BACKOFF_MAX_MS, Integer.MAX_VALUE, groupInstanceId, heartbeatThreadSupplier);
|
||||
|
||||
mockClient.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
|
||||
mockClient.prepareResponse(joinGroupFollowerResponse(1, memberId, leaderId, Errors.NONE));
|
||||
|
|
|
@ -1494,6 +1494,47 @@ public class CommitRequestManagerTest {
|
|||
assertTrue(commitRequestManager.pendingRequests.unsentOffsetCommits.isEmpty());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPollWithFatalErrorDuringCoordinatorIsEmptyAndClosing() {
|
||||
CommitRequestManager commitRequestManager = create(true, 100);
|
||||
|
||||
Map<TopicPartition, OffsetAndMetadata> offsets = Map.of(new TopicPartition("topic", 1),
|
||||
new OffsetAndMetadata(0));
|
||||
|
||||
var commitFuture = commitRequestManager.commitAsync(offsets);
|
||||
|
||||
commitRequestManager.signalClose();
|
||||
when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty());
|
||||
when(coordinatorRequestManager.fatalError())
|
||||
.thenReturn(Optional.of(new GroupAuthorizationException("Fatal error")));
|
||||
|
||||
assertEquals(NetworkClientDelegate.PollResult.EMPTY, commitRequestManager.poll(time.milliseconds()));
|
||||
|
||||
assertTrue(commitFuture.isCompletedExceptionally());
|
||||
|
||||
TestUtils.assertFutureThrows(GroupAuthorizationException.class, commitFuture, "Fatal error");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPollWithClosingAndPendingRequests() {
|
||||
CommitRequestManager commitRequestManager = create(true, 100);
|
||||
|
||||
Map<TopicPartition, OffsetAndMetadata> offsets = Map.of(new TopicPartition("topic", 1),
|
||||
new OffsetAndMetadata(0));
|
||||
|
||||
var commitFuture = commitRequestManager.commitAsync(offsets);
|
||||
|
||||
commitRequestManager.signalClose();
|
||||
when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty());
|
||||
|
||||
assertEquals(NetworkClientDelegate.PollResult.EMPTY, commitRequestManager.poll(time.milliseconds()));
|
||||
|
||||
assertTrue(commitFuture.isCompletedExceptionally());
|
||||
|
||||
TestUtils.assertFutureThrows(CommitFailedException.class, commitFuture,
|
||||
"Failed to commit offsets: Coordinator unknown and consumer is closing");
|
||||
}
|
||||
|
||||
// Supplies (error, isRetriable)
|
||||
private static Stream<Arguments> partitionDataErrorSupplier() {
|
||||
return Stream.of(
|
||||
|
|
|
@ -224,8 +224,7 @@ public abstract class ConsumerCoordinatorTest {
|
|||
groupInstanceId,
|
||||
rackId,
|
||||
retryBackoffMs,
|
||||
retryBackoffMaxMs,
|
||||
groupInstanceId.isEmpty());
|
||||
retryBackoffMaxMs);
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
|
|
|
@ -47,8 +47,7 @@ public class HeartbeatTest {
|
|||
Optional.empty(),
|
||||
null,
|
||||
retryBackoffMs,
|
||||
retryBackoffMaxMs,
|
||||
true);
|
||||
retryBackoffMaxMs);
|
||||
heartbeat = new Heartbeat(rebalanceConfig, time);
|
||||
}
|
||||
|
||||
|
|
|
@ -19,45 +19,323 @@ package org.apache.kafka.clients.consumer.internals.metrics;
|
|||
import org.apache.kafka.clients.consumer.internals.AutoOffsetResetStrategy;
|
||||
import org.apache.kafka.clients.consumer.internals.SubscriptionState;
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.metrics.MetricConfig;
|
||||
import org.apache.kafka.common.metrics.Metrics;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.MockTime;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.mockito.Mockito.mock;
|
||||
|
||||
|
||||
class ConsumerRebalanceMetricsManagerTest {
|
||||
|
||||
private final Time time = new MockTime();
|
||||
private final Metrics metrics = new Metrics(time);
|
||||
private Time time;
|
||||
private Metrics metrics;
|
||||
private SubscriptionState subscriptionState;
|
||||
private ConsumerRebalanceMetricsManager metricsManager;
|
||||
private MetricConfig metricConfig;
|
||||
private long windowSizeMs;
|
||||
private int numSamples;
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() {
|
||||
time = new MockTime();
|
||||
// Use MetricConfig with its default values
|
||||
windowSizeMs = 30000; // 30 seconds - default value
|
||||
numSamples = 2; // default value
|
||||
metricConfig = new MetricConfig()
|
||||
.samples(numSamples)
|
||||
.timeWindow(windowSizeMs, java.util.concurrent.TimeUnit.MILLISECONDS);
|
||||
metrics = new Metrics(metricConfig, time);
|
||||
subscriptionState = new SubscriptionState(mock(LogContext.class), AutoOffsetResetStrategy.EARLIEST);
|
||||
metricsManager = new ConsumerRebalanceMetricsManager(metrics, subscriptionState);
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() {
|
||||
metrics.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAssignedPartitionCountMetric() {
|
||||
SubscriptionState subscriptionState = new SubscriptionState(mock(LogContext.class), AutoOffsetResetStrategy.EARLIEST);
|
||||
ConsumerRebalanceMetricsManager consumerRebalanceMetricsManager = new ConsumerRebalanceMetricsManager(metrics, subscriptionState);
|
||||
|
||||
assertNotNull(metrics.metric(consumerRebalanceMetricsManager.assignedPartitionsCount), "Metric assigned-partitions has not been registered as expected");
|
||||
assertNotNull(metrics.metric(metricsManager.assignedPartitionsCount), "Metric assigned-partitions has not been registered as expected");
|
||||
|
||||
// Check for manually assigned partitions
|
||||
subscriptionState.assignFromUser(Set.of(new TopicPartition("topic", 0), new TopicPartition("topic", 1)));
|
||||
assertEquals(2.0d, metrics.metric(consumerRebalanceMetricsManager.assignedPartitionsCount).metricValue());
|
||||
assertEquals(2.0d, metrics.metric(metricsManager.assignedPartitionsCount).metricValue());
|
||||
subscriptionState.assignFromUser(Set.of());
|
||||
assertEquals(0.0d, metrics.metric(consumerRebalanceMetricsManager.assignedPartitionsCount).metricValue());
|
||||
assertEquals(0.0d, metrics.metric(metricsManager.assignedPartitionsCount).metricValue());
|
||||
|
||||
subscriptionState.unsubscribe();
|
||||
assertEquals(0.0d, metrics.metric(consumerRebalanceMetricsManager.assignedPartitionsCount).metricValue());
|
||||
assertEquals(0.0d, metrics.metric(metricsManager.assignedPartitionsCount).metricValue());
|
||||
|
||||
// Check for automatically assigned partitions
|
||||
subscriptionState.subscribe(Set.of("topic"), Optional.empty());
|
||||
subscriptionState.assignFromSubscribed(Set.of(new TopicPartition("topic", 0)));
|
||||
assertEquals(1.0d, metrics.metric(consumerRebalanceMetricsManager.assignedPartitionsCount).metricValue());
|
||||
assertEquals(1.0d, metrics.metric(metricsManager.assignedPartitionsCount).metricValue());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRebalanceTimingMetrics() {
|
||||
|
||||
// Verify timing metrics are registered
|
||||
assertNotNull(metrics.metric(metricsManager.rebalanceLatencyAvg));
|
||||
assertNotNull(metrics.metric(metricsManager.rebalanceLatencyMax));
|
||||
assertNotNull(metrics.metric(metricsManager.rebalanceLatencyTotal));
|
||||
assertNotNull(metrics.metric(metricsManager.rebalanceTotal));
|
||||
|
||||
// Record first rebalance (10ms duration)
|
||||
metricsManager.recordRebalanceStarted(time.milliseconds());
|
||||
time.sleep(10);
|
||||
metricsManager.recordRebalanceEnded(time.milliseconds());
|
||||
|
||||
// Verify metrics after first rebalance
|
||||
assertEquals(10.0d, metrics.metric(metricsManager.rebalanceLatencyAvg).metricValue());
|
||||
assertEquals(10.0d, metrics.metric(metricsManager.rebalanceLatencyMax).metricValue());
|
||||
assertEquals(10.0d, metrics.metric(metricsManager.rebalanceLatencyTotal).metricValue());
|
||||
assertEquals(1.0d, metrics.metric(metricsManager.rebalanceTotal).metricValue());
|
||||
|
||||
// Record second rebalance (30ms duration)
|
||||
metricsManager.recordRebalanceStarted(time.milliseconds());
|
||||
time.sleep(30);
|
||||
metricsManager.recordRebalanceEnded(time.milliseconds());
|
||||
|
||||
// Verify metrics after second rebalance
|
||||
assertEquals(20.0d, metrics.metric(metricsManager.rebalanceLatencyAvg).metricValue(),
|
||||
"Average latency should be (10 + 30) / 2 = 20ms");
|
||||
assertEquals(30.0d, metrics.metric(metricsManager.rebalanceLatencyMax).metricValue(),
|
||||
"Max latency should be max(10, 30) = 30ms");
|
||||
assertEquals(40.0d, metrics.metric(metricsManager.rebalanceLatencyTotal).metricValue(),
|
||||
"Total latency should be 10 + 30 = 40ms");
|
||||
assertEquals(2.0d, metrics.metric(metricsManager.rebalanceTotal).metricValue());
|
||||
|
||||
// Record third rebalance (50ms duration)
|
||||
metricsManager.recordRebalanceStarted(time.milliseconds());
|
||||
time.sleep(50);
|
||||
metricsManager.recordRebalanceEnded(time.milliseconds());
|
||||
|
||||
// Verify metrics after third rebalance
|
||||
assertEquals(30.0d, metrics.metric(metricsManager.rebalanceLatencyAvg).metricValue(),
|
||||
"Average latency should be (10 + 30 + 50) / 3 = 30ms");
|
||||
assertEquals(50.0d, metrics.metric(metricsManager.rebalanceLatencyMax).metricValue(),
|
||||
"Max latency should be max(10, 30, 50) = 50ms");
|
||||
assertEquals(90.0d, metrics.metric(metricsManager.rebalanceLatencyTotal).metricValue(),
|
||||
"Total latency should be 10 + 30 + 50 = 90ms");
|
||||
assertEquals(3.0d, metrics.metric(metricsManager.rebalanceTotal).metricValue());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRebalanceRateMetric() {
|
||||
|
||||
// Verify rate metric is registered
|
||||
assertNotNull(metrics.metric(metricsManager.rebalanceRatePerHour));
|
||||
|
||||
// Record 3 rebalances within 30ms total (3 x 10ms)
|
||||
int rebalanceCount = 3;
|
||||
long startTime = time.milliseconds();
|
||||
for (int i = 0; i < rebalanceCount; i++) {
|
||||
metricsManager.recordRebalanceStarted(time.milliseconds());
|
||||
time.sleep(10);
|
||||
metricsManager.recordRebalanceEnded(time.milliseconds());
|
||||
}
|
||||
long endTime = time.milliseconds();
|
||||
long actualElapsedMs = endTime - startTime;
|
||||
|
||||
double ratePerHour = (Double) metrics.metric(metricsManager.rebalanceRatePerHour).metricValue();
|
||||
|
||||
// The Rate metric calculation:
|
||||
// - Uses elapsed time from the oldest sample
|
||||
// - Ensures minimum window size of (numSamples - 1) * windowSizeMs
|
||||
// - With default config: minWindow = (2-1) * 30000 = 30000ms
|
||||
long minWindowMs = (numSamples - 1) * windowSizeMs; // (2-1) * 30000 = 30000ms
|
||||
|
||||
// Since actualElapsedMs (30ms) is much less than minWindowMs (30000ms),
|
||||
// the rate calculation will use minWindowMs as the window
|
||||
// Rate per hour = count / (windowMs / 1000) * 3600
|
||||
double expectedRatePerHour = (double) rebalanceCount / (minWindowMs / 1000.0) * 3600.0;
|
||||
|
||||
assertEquals(expectedRatePerHour, ratePerHour, 1.0,
|
||||
String.format("With %d rebalances in %dms, min window %dms: expecting %.1f rebalances/hour",
|
||||
rebalanceCount, actualElapsedMs, minWindowMs, expectedRatePerHour));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFailedRebalanceMetrics() {
|
||||
|
||||
// Verify failed rebalance metrics are registered
|
||||
assertNotNull(metrics.metric(metricsManager.failedRebalanceTotal));
|
||||
assertNotNull(metrics.metric(metricsManager.failedRebalanceRate));
|
||||
|
||||
assertEquals(0.0d, metrics.metric(metricsManager.failedRebalanceTotal).metricValue(),
|
||||
"Initially, there should be no failed rebalances");
|
||||
|
||||
// Start a rebalance but don't complete it
|
||||
metricsManager.recordRebalanceStarted(time.milliseconds());
|
||||
time.sleep(10);
|
||||
|
||||
metricsManager.maybeRecordRebalanceFailed();
|
||||
assertEquals(1.0d, metrics.metric(metricsManager.failedRebalanceTotal).metricValue(),
|
||||
"Failed rebalance count should increment to 1 after recording failure");
|
||||
|
||||
// Complete a successful rebalance
|
||||
metricsManager.recordRebalanceStarted(time.milliseconds());
|
||||
time.sleep(10);
|
||||
metricsManager.recordRebalanceEnded(time.milliseconds());
|
||||
|
||||
metricsManager.maybeRecordRebalanceFailed();
|
||||
assertEquals(1.0d, metrics.metric(metricsManager.failedRebalanceTotal).metricValue(),
|
||||
"Failed count should not increment after successful rebalance completes");
|
||||
|
||||
// Start another rebalance, don't complete it, then record failure
|
||||
time.sleep(10);
|
||||
metricsManager.recordRebalanceStarted(time.milliseconds());
|
||||
assertTrue(metricsManager.rebalanceStarted(), "Rebalance should be in progress");
|
||||
time.sleep(10);
|
||||
// Don't call recordRebalanceEnded() to simulate an incomplete rebalance
|
||||
metricsManager.maybeRecordRebalanceFailed();
|
||||
assertEquals(2.0d, metrics.metric(metricsManager.failedRebalanceTotal).metricValue());
|
||||
|
||||
double failedRate = (Double) metrics.metric(metricsManager.failedRebalanceRate).metricValue();
|
||||
|
||||
// Calculate expected failed rate based on Rate metric behavior
|
||||
// We had 2 failures over ~40ms, but minimum window is (numSamples - 1) * windowSizeMs
|
||||
long minWindowMs = (numSamples - 1) * windowSizeMs; // (2-1) * 30000 = 30000ms
|
||||
double expectedFailedRatePerHour = 2.0 / (minWindowMs / 1000.0) * 3600.0;
|
||||
|
||||
assertEquals(expectedFailedRatePerHour, failedRate, 1.0,
|
||||
String.format("With 2 failures, min window %dms: expecting %.1f failures/hour",
|
||||
minWindowMs, expectedFailedRatePerHour));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLastRebalanceSecondsAgoMetric() {
|
||||
|
||||
// Verify metric is registered
|
||||
assertNotNull(metrics.metric(metricsManager.lastRebalanceSecondsAgo));
|
||||
|
||||
assertEquals(-1.0d, metrics.metric(metricsManager.lastRebalanceSecondsAgo).metricValue(),
|
||||
"Should return -1 when no rebalance has occurred");
|
||||
|
||||
// Complete a rebalance
|
||||
metricsManager.recordRebalanceStarted(time.milliseconds());
|
||||
time.sleep(10);
|
||||
metricsManager.recordRebalanceEnded(time.milliseconds());
|
||||
|
||||
assertEquals(0.0d, metrics.metric(metricsManager.lastRebalanceSecondsAgo).metricValue(),
|
||||
"Should return 0 immediately after rebalance completes");
|
||||
|
||||
// Advance time by 5 seconds
|
||||
time.sleep(5000);
|
||||
assertEquals(5.0d, metrics.metric(metricsManager.lastRebalanceSecondsAgo).metricValue());
|
||||
|
||||
// Advance time by another 10 seconds
|
||||
time.sleep(10000);
|
||||
assertEquals(15.0d, metrics.metric(metricsManager.lastRebalanceSecondsAgo).metricValue());
|
||||
|
||||
// Complete another rebalance
|
||||
metricsManager.recordRebalanceStarted(time.milliseconds());
|
||||
time.sleep(20);
|
||||
metricsManager.recordRebalanceEnded(time.milliseconds());
|
||||
|
||||
assertEquals(0.0d, metrics.metric(metricsManager.lastRebalanceSecondsAgo).metricValue(),
|
||||
"Should reset to 0 after a new rebalance completes");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRebalanceStartedFlag() {
|
||||
|
||||
assertFalse(metricsManager.rebalanceStarted(),
|
||||
"Initially, no rebalance should be in progress");
|
||||
|
||||
metricsManager.recordRebalanceStarted(time.milliseconds());
|
||||
assertTrue(metricsManager.rebalanceStarted(),
|
||||
"Rebalance should be marked as started after recordRebalanceStarted()");
|
||||
|
||||
time.sleep(10);
|
||||
metricsManager.recordRebalanceEnded(time.milliseconds());
|
||||
assertFalse(metricsManager.rebalanceStarted(),
|
||||
"Rebalance should not be in progress after recordRebalanceEnded()");
|
||||
|
||||
// Start another rebalance - advance time first
|
||||
time.sleep(100);
|
||||
metricsManager.recordRebalanceStarted(time.milliseconds());
|
||||
assertTrue(metricsManager.rebalanceStarted(),
|
||||
"New rebalance should be marked as started");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultipleConsecutiveFailures() {
|
||||
|
||||
// Record multiple consecutive failures
|
||||
for (int i = 0; i < 5; i++) {
|
||||
metricsManager.recordRebalanceStarted(time.milliseconds());
|
||||
time.sleep(10);
|
||||
metricsManager.maybeRecordRebalanceFailed();
|
||||
}
|
||||
|
||||
assertEquals(5.0d, metrics.metric(metricsManager.failedRebalanceTotal).metricValue(),
|
||||
"Should have recorded 5 consecutive failed rebalances");
|
||||
|
||||
assertEquals(0.0d, metrics.metric(metricsManager.rebalanceTotal).metricValue(),
|
||||
"Successful rebalance count should remain 0 when only failures occur");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMixedSuccessAndFailureScenarios() {
|
||||
|
||||
// Success -> Failure -> Success -> Failure pattern
|
||||
// First success
|
||||
metricsManager.recordRebalanceStarted(time.milliseconds());
|
||||
time.sleep(20);
|
||||
metricsManager.recordRebalanceEnded(time.milliseconds());
|
||||
assertEquals(1.0d, metrics.metric(metricsManager.rebalanceTotal).metricValue());
|
||||
|
||||
// First failure
|
||||
time.sleep(10);
|
||||
metricsManager.recordRebalanceStarted(time.milliseconds());
|
||||
assertTrue(metricsManager.rebalanceStarted(), "First failure rebalance should be in progress");
|
||||
time.sleep(30);
|
||||
metricsManager.maybeRecordRebalanceFailed();
|
||||
|
||||
double failedAfterFirst = (Double) metrics.metric(metricsManager.failedRebalanceTotal).metricValue();
|
||||
assertEquals(1.0d, failedAfterFirst, "Should have recorded one failed rebalance after first failure");
|
||||
|
||||
// Second success
|
||||
time.sleep(10);
|
||||
metricsManager.recordRebalanceStarted(time.milliseconds());
|
||||
time.sleep(40);
|
||||
metricsManager.recordRebalanceEnded(time.milliseconds());
|
||||
assertEquals(2.0d, metrics.metric(metricsManager.rebalanceTotal).metricValue());
|
||||
|
||||
// Second failure
|
||||
time.sleep(10);
|
||||
metricsManager.recordRebalanceStarted(time.milliseconds());
|
||||
assertTrue(metricsManager.rebalanceStarted(), "Second failure rebalance should be in progress");
|
||||
time.sleep(50);
|
||||
metricsManager.maybeRecordRebalanceFailed();
|
||||
|
||||
assertEquals(2.0d, metrics.metric(metricsManager.rebalanceTotal).metricValue(),
|
||||
"Should have 2 successful rebalances in mixed scenario");
|
||||
assertEquals(2.0d, metrics.metric(metricsManager.failedRebalanceTotal).metricValue(),
|
||||
"Should have 2 failed rebalances in mixed scenario");
|
||||
|
||||
assertEquals(30.0d, metrics.metric(metricsManager.rebalanceLatencyAvg).metricValue(),
|
||||
"Average latency should only include successful rebalances: (20 + 40) / 2 = 30ms");
|
||||
assertEquals(40.0d, metrics.metric(metricsManager.rebalanceLatencyMax).metricValue(),
|
||||
"Max latency should be 40ms from successful rebalances only");
|
||||
assertEquals(60.0d, metrics.metric(metricsManager.rebalanceLatencyTotal).metricValue(),
|
||||
"Total latency should only include successful rebalances: 20 + 40 = 60ms");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -48,4 +48,29 @@ public class KafkaMetricTest {
|
|||
assertThrows(IllegalStateException.class, metric::measurable);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMeasurableValueReturnsZeroWhenNotMeasurable() {
|
||||
MockTime time = new MockTime();
|
||||
MetricConfig config = new MetricConfig();
|
||||
Gauge<Integer> gauge = (c, now) -> 7;
|
||||
|
||||
KafkaMetric metric = new KafkaMetric(new Object(), METRIC_NAME, gauge, config, time);
|
||||
assertEquals(0.0d, metric.measurableValue(time.milliseconds()), 0.0d);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testKafkaMetricAcceptsNonMeasurableNonGaugeProvider() {
|
||||
MetricValueProvider<String> provider = (config, now) -> "metric value provider";
|
||||
KafkaMetric metric = new KafkaMetric(new Object(), METRIC_NAME, provider, new MetricConfig(), new MockTime());
|
||||
|
||||
Object value = metric.metricValue();
|
||||
assertEquals("metric value provider", value);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConstructorWithNullProvider() {
|
||||
assertThrows(NullPointerException.class, () ->
|
||||
new KafkaMetric(new Object(), METRIC_NAME, null, new MetricConfig(), new MockTime())
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -74,11 +74,10 @@ public abstract class SslSelectorTest extends SelectorTest {
|
|||
this.server.start();
|
||||
this.time = new MockTime();
|
||||
sslClientConfigs = createSslClientConfigs(trustStoreFile);
|
||||
LogContext logContext = new LogContext();
|
||||
this.channelBuilder = new SslChannelBuilder(ConnectionMode.CLIENT, null, false, logContext);
|
||||
this.channelBuilder = new SslChannelBuilder(ConnectionMode.CLIENT, null, false);
|
||||
this.channelBuilder.configure(sslClientConfigs);
|
||||
this.metrics = new Metrics();
|
||||
this.selector = new Selector(5000, metrics, time, "MetricGroup", channelBuilder, logContext);
|
||||
this.selector = new Selector(5000, metrics, time, "MetricGroup", channelBuilder, new LogContext());
|
||||
}
|
||||
|
||||
protected abstract Map<String, Object> createSslClientConfigs(File trustStoreFile) throws GeneralSecurityException, IOException;
|
||||
|
@ -255,7 +254,7 @@ public abstract class SslSelectorTest extends SelectorTest {
|
|||
.tlsProtocol(tlsProtocol)
|
||||
.createNewTrustStore(trustStoreFile)
|
||||
.build();
|
||||
channelBuilder = new SslChannelBuilder(ConnectionMode.SERVER, null, false, new LogContext());
|
||||
channelBuilder = new SslChannelBuilder(ConnectionMode.SERVER, null, false);
|
||||
channelBuilder.configure(sslServerConfigs);
|
||||
selector = new Selector(NetworkReceive.UNLIMITED, 5000, metrics, time, "MetricGroup",
|
||||
new HashMap<>(), true, false, channelBuilder, pool, new LogContext());
|
||||
|
@ -342,7 +341,7 @@ public abstract class SslSelectorTest extends SelectorTest {
|
|||
private static class TestSslChannelBuilder extends SslChannelBuilder {
|
||||
|
||||
public TestSslChannelBuilder(ConnectionMode connectionMode) {
|
||||
super(connectionMode, null, false, new LogContext());
|
||||
super(connectionMode, null, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -767,11 +767,10 @@ public class SslTransportLayerTest {
|
|||
@ParameterizedTest
|
||||
@ArgumentsSource(SslTransportLayerArgumentsProvider.class)
|
||||
public void testNetworkThreadTimeRecorded(Args args) throws Exception {
|
||||
LogContext logContext = new LogContext();
|
||||
ChannelBuilder channelBuilder = new SslChannelBuilder(ConnectionMode.CLIENT, null, false, logContext);
|
||||
ChannelBuilder channelBuilder = new SslChannelBuilder(ConnectionMode.CLIENT, null, false);
|
||||
channelBuilder.configure(args.sslClientConfigs);
|
||||
try (Selector selector = new Selector(NetworkReceive.UNLIMITED, Selector.NO_IDLE_TIMEOUT_MS, new Metrics(), Time.SYSTEM,
|
||||
"MetricGroup", new HashMap<>(), false, true, channelBuilder, MemoryPool.NONE, logContext)) {
|
||||
"MetricGroup", new HashMap<>(), false, true, channelBuilder, MemoryPool.NONE, new LogContext())) {
|
||||
|
||||
String node = "0";
|
||||
server = createEchoServer(args, SecurityProtocol.SSL);
|
||||
|
@ -967,7 +966,7 @@ public class SslTransportLayerTest {
|
|||
}
|
||||
|
||||
private SslChannelBuilder newClientChannelBuilder() {
|
||||
return new SslChannelBuilder(ConnectionMode.CLIENT, null, false, new LogContext());
|
||||
return new SslChannelBuilder(ConnectionMode.CLIENT, null, false);
|
||||
}
|
||||
|
||||
private void testClose(Args args, SecurityProtocol securityProtocol, ChannelBuilder clientChannelBuilder) throws Exception {
|
||||
|
@ -1311,10 +1310,9 @@ public class SslTransportLayerTest {
|
|||
}
|
||||
|
||||
private Selector createSelector(Args args) {
|
||||
LogContext logContext = new LogContext();
|
||||
ChannelBuilder channelBuilder = new SslChannelBuilder(ConnectionMode.CLIENT, null, false, logContext);
|
||||
ChannelBuilder channelBuilder = new SslChannelBuilder(ConnectionMode.CLIENT, null, false);
|
||||
channelBuilder.configure(args.sslClientConfigs);
|
||||
selector = new Selector(5000, new Metrics(), TIME, "MetricGroup", channelBuilder, logContext);
|
||||
selector = new Selector(5000, new Metrics(), TIME, "MetricGroup", channelBuilder, new LogContext());
|
||||
return selector;
|
||||
}
|
||||
|
||||
|
@ -1371,7 +1369,7 @@ public class SslTransportLayerTest {
|
|||
int flushDelayCount = 0;
|
||||
|
||||
public TestSslChannelBuilder(ConnectionMode connectionMode) {
|
||||
super(connectionMode, null, false, new LogContext());
|
||||
super(connectionMode, null, false);
|
||||
}
|
||||
|
||||
public void configureBufferSizes(Integer netReadBufSize, Integer netWriteBufSize, Integer appBufSize) {
|
||||
|
|
|
@ -50,10 +50,9 @@ public class SslTransportTls12Tls13Test {
|
|||
sslServerConfigs = serverCertStores.getTrustingConfig(clientCertStores);
|
||||
sslClientConfigs = clientCertStores.getTrustingConfig(serverCertStores);
|
||||
|
||||
LogContext logContext = new LogContext();
|
||||
ChannelBuilder channelBuilder = new SslChannelBuilder(ConnectionMode.CLIENT, null, false, logContext);
|
||||
ChannelBuilder channelBuilder = new SslChannelBuilder(ConnectionMode.CLIENT, null, false);
|
||||
channelBuilder.configure(sslClientConfigs);
|
||||
this.selector = new Selector(5000, new Metrics(), TIME, "MetricGroup", channelBuilder, logContext);
|
||||
this.selector = new Selector(5000, new Metrics(), TIME, "MetricGroup", channelBuilder, new LogContext());
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
|
|
|
@ -233,13 +233,13 @@ public class MirrorMaker {
|
|||
private void addHerder(SourceAndTarget sourceAndTarget) {
|
||||
log.info("creating herder for {}", sourceAndTarget.toString());
|
||||
Map<String, String> workerProps = config.workerConfig(sourceAndTarget);
|
||||
DistributedConfig distributedConfig = new DistributedConfig(workerProps);
|
||||
String encodedSource = encodePath(sourceAndTarget.source());
|
||||
String encodedTarget = encodePath(sourceAndTarget.target());
|
||||
List<String> restNamespace = List.of(encodedSource, encodedTarget);
|
||||
String workerId = generateWorkerId(sourceAndTarget);
|
||||
Plugins plugins = new Plugins(workerProps);
|
||||
plugins.compareAndSwapWithDelegatingLoader();
|
||||
DistributedConfig distributedConfig = new DistributedConfig(workerProps);
|
||||
String kafkaClusterId = distributedConfig.kafkaClusterId();
|
||||
String clientIdBase = ConnectUtils.clientIdBase(distributedConfig);
|
||||
// Create the admin client to be shared by all backing stores for this herder
|
||||
|
|
|
@ -114,14 +114,15 @@ public abstract class AbstractConnectCli<H extends Herder, T extends WorkerConfi
|
|||
log.info("Kafka Connect worker initializing ...");
|
||||
long initStart = time.hiResClockMs();
|
||||
|
||||
T config = createConfig(workerProps);
|
||||
log.debug("Kafka cluster ID: {}", config.kafkaClusterId());
|
||||
|
||||
WorkerInfo initInfo = new WorkerInfo();
|
||||
initInfo.logAll();
|
||||
|
||||
log.info("Scanning for plugin classes. This might take a moment ...");
|
||||
Plugins plugins = new Plugins(workerProps);
|
||||
plugins.compareAndSwapWithDelegatingLoader();
|
||||
T config = createConfig(workerProps);
|
||||
log.debug("Kafka cluster ID: {}", config.kafkaClusterId());
|
||||
|
||||
RestClient restClient = new RestClient(config);
|
||||
|
||||
|
|
|
@ -149,8 +149,7 @@ public class WorkerCoordinatorIncrementalTest {
|
|||
Optional.empty(),
|
||||
null,
|
||||
retryBackoffMs,
|
||||
retryBackoffMaxMs,
|
||||
true);
|
||||
retryBackoffMaxMs);
|
||||
this.coordinator = new WorkerCoordinator(rebalanceConfig,
|
||||
loggerFactory,
|
||||
consumerClient,
|
||||
|
|
|
@ -140,8 +140,7 @@ public class WorkerCoordinatorTest {
|
|||
Optional.empty(),
|
||||
null,
|
||||
retryBackoffMs,
|
||||
retryBackoffMaxMs,
|
||||
true);
|
||||
retryBackoffMaxMs);
|
||||
this.coordinator = new WorkerCoordinator(rebalanceConfig,
|
||||
logContext,
|
||||
consumerClient,
|
||||
|
|
|
@ -1923,11 +1923,6 @@ public class CoordinatorRuntime<S extends CoordinatorShard<U>, U> implements Aut
|
|||
*/
|
||||
private final String logPrefix;
|
||||
|
||||
/**
|
||||
* The log context.
|
||||
*/
|
||||
private final LogContext logContext;
|
||||
|
||||
/**
|
||||
* The logger.
|
||||
*/
|
||||
|
@ -2054,7 +2049,6 @@ public class CoordinatorRuntime<S extends CoordinatorShard<U>, U> implements Aut
|
|||
ExecutorService executorService
|
||||
) {
|
||||
this.logPrefix = logPrefix;
|
||||
this.logContext = logContext;
|
||||
this.log = logContext.logger(CoordinatorRuntime.class);
|
||||
this.time = time;
|
||||
this.timer = timer;
|
||||
|
|
|
@ -89,7 +89,7 @@ public class DescribeTopicPartitionsRequestHandler {
|
|||
// Do not disclose the existence of topics unauthorized for Describe, so we've not even checked if they exist or not
|
||||
Set<DescribeTopicPartitionsResponseTopic> unauthorizedForDescribeTopicMetadata = new HashSet<>();
|
||||
|
||||
Stream<String> authorizedTopicsStream = topics.stream().sorted().filter(topicName -> {
|
||||
Stream<String> authorizedTopicsStream = topics.stream().filter(topicName -> {
|
||||
boolean isAuthorized = authHelper.authorize(
|
||||
abstractRequest.context(), DESCRIBE, TOPIC, topicName, true, true, 1);
|
||||
if (!fetchAllTopics && !isAuthorized) {
|
||||
|
@ -99,7 +99,7 @@ public class DescribeTopicPartitionsRequestHandler {
|
|||
);
|
||||
}
|
||||
return isAuthorized;
|
||||
});
|
||||
}).sorted();
|
||||
|
||||
DescribeTopicPartitionsResponseData response = metadataCache.describeTopicResponse(
|
||||
authorizedTopicsStream.iterator(),
|
||||
|
|
|
@ -20,7 +20,6 @@ import java.lang.{Long => JLong}
|
|||
import java.util.concurrent.locks.ReentrantReadWriteLock
|
||||
import java.util.Optional
|
||||
import java.util.concurrent.{CompletableFuture, ConcurrentHashMap, CopyOnWriteArrayList}
|
||||
import kafka.controller.StateChangeLogger
|
||||
import kafka.log._
|
||||
import kafka.server._
|
||||
import kafka.server.share.DelayedShareFetch
|
||||
|
@ -37,6 +36,7 @@ import org.apache.kafka.common.record.{FileRecords, MemoryRecords, RecordBatch}
|
|||
import org.apache.kafka.common.requests._
|
||||
import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET}
|
||||
import org.apache.kafka.common.utils.Time
|
||||
import org.apache.kafka.logger.StateChangeLogger
|
||||
import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState, MetadataCache, PartitionRegistration}
|
||||
import org.apache.kafka.server.common.RequestLocal
|
||||
import org.apache.kafka.server.log.remote.TopicPartitionLog
|
||||
|
@ -322,7 +322,7 @@ class Partition(val topicPartition: TopicPartition,
|
|||
def topic: String = topicPartition.topic
|
||||
def partitionId: Int = topicPartition.partition
|
||||
|
||||
private val stateChangeLogger = new StateChangeLogger(localBrokerId, inControllerContext = false, None)
|
||||
private val stateChangeLogger = new StateChangeLogger(localBrokerId)
|
||||
private val remoteReplicasMap = new ConcurrentHashMap[Int, Replica]
|
||||
// The read lock is only required when multiple reads are executed and needs to be in a consistent manner
|
||||
private val leaderIsrUpdateLock = new ReentrantReadWriteLock
|
||||
|
@ -1658,7 +1658,7 @@ class Partition(val topicPartition: TopicPartition,
|
|||
def deleteRecordsOnLeader(offset: Long): LogDeleteRecordsResult = inReadLock(leaderIsrUpdateLock) {
|
||||
leaderLogIfLocal match {
|
||||
case Some(leaderLog) =>
|
||||
if (!leaderLog.config.delete)
|
||||
if (!leaderLog.config.delete && leaderLog.config.compact)
|
||||
throw new PolicyViolationException(s"Records of partition $topicPartition can not be deleted due to the configured policy")
|
||||
|
||||
val convertedOffset = if (offset == DeleteRecordsRequest.HIGH_WATERMARK)
|
||||
|
|
|
@ -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] "
|
||||
}
|
||||
|
||||
}
|
|
@ -41,7 +41,7 @@ import org.apache.kafka.coordinator.share.metrics.{ShareCoordinatorMetrics, Shar
|
|||
import org.apache.kafka.coordinator.share.{ShareCoordinator, ShareCoordinatorRecordSerde, ShareCoordinatorService}
|
||||
import org.apache.kafka.coordinator.transaction.ProducerIdManager
|
||||
import org.apache.kafka.image.publisher.{BrokerRegistrationTracker, MetadataPublisher}
|
||||
import org.apache.kafka.metadata.{BrokerState, ListenerInfo}
|
||||
import org.apache.kafka.metadata.{BrokerState, ListenerInfo, MetadataVersionConfigValidator}
|
||||
import org.apache.kafka.metadata.publisher.{AclPublisher, DelegationTokenPublisher, ScramPublisher}
|
||||
import org.apache.kafka.security.{CredentialProvider, DelegationTokenManager}
|
||||
import org.apache.kafka.server.authorizer.Authorizer
|
||||
|
@ -469,7 +469,10 @@ class BrokerServer(
|
|||
socketServer.dataPlaneRequestChannel, dataPlaneRequestProcessor, time,
|
||||
config.numIoThreads, "RequestHandlerAvgIdlePercent")
|
||||
|
||||
metadataPublishers.add(new MetadataVersionConfigValidator(config, sharedServer.metadataPublishingFaultHandler))
|
||||
metadataPublishers.add(new MetadataVersionConfigValidator(config.brokerId,
|
||||
() => config.processRoles.contains(ProcessRole.BrokerRole) && config.logDirs().size() > 1,
|
||||
sharedServer.metadataPublishingFaultHandler
|
||||
))
|
||||
brokerMetadataPublisher = new BrokerMetadataPublisher(config,
|
||||
metadataCache,
|
||||
logManager,
|
||||
|
|
|
@ -43,7 +43,6 @@ import org.apache.kafka.raft.{MetadataLogConfig, QuorumConfig}
|
|||
import org.apache.kafka.security.authorizer.AuthorizerUtils
|
||||
import org.apache.kafka.server.ProcessRole
|
||||
import org.apache.kafka.server.authorizer.Authorizer
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.config.AbstractKafkaConfig.getMap
|
||||
import org.apache.kafka.server.config.{AbstractKafkaConfig, KRaftConfigs, QuotaConfig, ReplicationConfigs, ServerConfigs, ServerLogConfigs}
|
||||
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
|
||||
|
@ -653,18 +652,6 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _])
|
|||
s"${BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG} must implement KafkaPrincipalSerde")
|
||||
}
|
||||
|
||||
/**
|
||||
* Validate some configurations for new MetadataVersion. A new MetadataVersion can take place when
|
||||
* a FeatureLevelRecord for "metadata.version" is read from the cluster metadata.
|
||||
*/
|
||||
def validateWithMetadataVersion(metadataVersion: MetadataVersion): Unit = {
|
||||
if (processRoles.contains(ProcessRole.BrokerRole) && logDirs.size > 1) {
|
||||
require(metadataVersion.isDirectoryAssignmentSupported,
|
||||
s"Multiple log directories (aka JBOD) are not supported in the current MetadataVersion ${metadataVersion}. " +
|
||||
s"Need ${MetadataVersion.IBP_3_7_IV2} or higher")
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Copy the subset of properties that are relevant to Logs. The individual properties
|
||||
* are listed here since the names are slightly different in each Config class...
|
||||
|
|
|
@ -18,7 +18,6 @@ package kafka.server
|
|||
|
||||
import com.yammer.metrics.core.Meter
|
||||
import kafka.cluster.{Partition, PartitionListener}
|
||||
import kafka.controller.StateChangeLogger
|
||||
import kafka.log.LogManager
|
||||
import kafka.server.HostedPartition.Online
|
||||
import kafka.server.QuotaFactory.QuotaManagers
|
||||
|
@ -48,6 +47,7 @@ import org.apache.kafka.common.requests._
|
|||
import org.apache.kafka.common.utils.{Exit, Time, Utils}
|
||||
import org.apache.kafka.coordinator.transaction.{AddPartitionsToTxnConfig, TransactionLogConfig}
|
||||
import org.apache.kafka.image.{LocalReplicaChanges, MetadataImage, TopicsDelta}
|
||||
import org.apache.kafka.logger.StateChangeLogger
|
||||
import org.apache.kafka.metadata.LeaderConstants.NO_LEADER
|
||||
import org.apache.kafka.metadata.MetadataCache
|
||||
import org.apache.kafka.server.common.{DirectoryEventHandler, RequestLocal, StopPartition}
|
||||
|
@ -272,7 +272,7 @@ class ReplicaManager(val config: KafkaConfig,
|
|||
@volatile private var isInControlledShutdown = false
|
||||
|
||||
this.logIdent = s"[ReplicaManager broker=$localBrokerId] "
|
||||
protected val stateChangeLogger = new StateChangeLogger(localBrokerId, inControllerContext = false, None)
|
||||
protected val stateChangeLogger = new StateChangeLogger(localBrokerId)
|
||||
|
||||
private var logDirFailureHandler: LogDirFailureHandler = _
|
||||
|
||||
|
@ -789,9 +789,9 @@ class ReplicaManager(val config: KafkaConfig,
|
|||
hasCustomErrorMessage = customException.isDefined
|
||||
)
|
||||
}
|
||||
// In non-transaction paths, errorResults is typically empty, so we can
|
||||
// In non-transaction paths, errorResults is typically empty, so we can
|
||||
// directly use entriesPerPartition instead of creating a new filtered collection
|
||||
val entriesWithoutErrorsPerPartition =
|
||||
val entriesWithoutErrorsPerPartition =
|
||||
if (errorResults.nonEmpty) entriesPerPartition.filter { case (key, _) => !errorResults.contains(key) }
|
||||
else entriesPerPartition
|
||||
|
||||
|
@ -1637,13 +1637,13 @@ class ReplicaManager(val config: KafkaConfig,
|
|||
remoteFetchPartitionStatus: Seq[(TopicIdPartition, FetchPartitionStatus)]): Unit = {
|
||||
val remoteFetchTasks = new util.HashMap[TopicIdPartition, Future[Void]]
|
||||
val remoteFetchResults = new util.HashMap[TopicIdPartition, CompletableFuture[RemoteLogReadResult]]
|
||||
|
||||
|
||||
remoteFetchInfos.forEach { (topicIdPartition, remoteFetchInfo) =>
|
||||
val (task, result) = processRemoteFetch(remoteFetchInfo)
|
||||
remoteFetchTasks.put(topicIdPartition, task)
|
||||
remoteFetchResults.put(topicIdPartition, result)
|
||||
}
|
||||
|
||||
|
||||
val remoteFetchMaxWaitMs = config.remoteLogManagerConfig.remoteFetchMaxWaitMs().toLong
|
||||
val remoteFetch = new DelayedRemoteFetch(remoteFetchTasks, remoteFetchResults, remoteFetchInfos, remoteFetchMaxWaitMs,
|
||||
remoteFetchPartitionStatus, params, logReadResults, this, responseCallback)
|
||||
|
|
|
@ -1,103 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package kafka.server;
|
||||
|
||||
import org.apache.kafka.common.metadata.FeatureLevelRecord;
|
||||
import org.apache.kafka.image.MetadataDelta;
|
||||
import org.apache.kafka.image.MetadataImage;
|
||||
import org.apache.kafka.image.MetadataProvenance;
|
||||
import org.apache.kafka.image.loader.LogDeltaManifest;
|
||||
import org.apache.kafka.raft.LeaderAndEpoch;
|
||||
import org.apache.kafka.server.common.MetadataVersion;
|
||||
import org.apache.kafka.server.fault.FaultHandler;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.ArgumentMatchers.eq;
|
||||
import static org.mockito.BDDMockito.willAnswer;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.times;
|
||||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.verifyNoMoreInteractions;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class MetadataVersionConfigValidatorTest {
|
||||
|
||||
private static final LogDeltaManifest TEST_MANIFEST = LogDeltaManifest.newBuilder()
|
||||
.provenance(MetadataProvenance.EMPTY)
|
||||
.leaderAndEpoch(LeaderAndEpoch.UNKNOWN)
|
||||
.numBatches(1)
|
||||
.elapsedNs(90)
|
||||
.numBytes(88)
|
||||
.build();
|
||||
public static final MetadataProvenance TEST_PROVENANCE =
|
||||
new MetadataProvenance(50, 3, 8000, true);
|
||||
|
||||
void testWith(MetadataVersion metadataVersion, KafkaConfig config, FaultHandler faultHandler) throws Exception {
|
||||
try (MetadataVersionConfigValidator validator = new MetadataVersionConfigValidator(config, faultHandler)) {
|
||||
MetadataDelta delta = new MetadataDelta.Builder()
|
||||
.setImage(MetadataImage.EMPTY)
|
||||
.build();
|
||||
if (metadataVersion != null) {
|
||||
delta.replay(new FeatureLevelRecord().
|
||||
setName(MetadataVersion.FEATURE_NAME).
|
||||
setFeatureLevel(metadataVersion.featureLevel()));
|
||||
}
|
||||
MetadataImage image = delta.apply(TEST_PROVENANCE);
|
||||
|
||||
validator.onMetadataUpdate(delta, image, TEST_MANIFEST);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testValidatesConfigOnMetadataChange() throws Exception {
|
||||
MetadataVersion metadataVersion = MetadataVersion.IBP_3_7_IV2;
|
||||
KafkaConfig config = mock(KafkaConfig.class);
|
||||
FaultHandler faultHandler = mock(FaultHandler.class);
|
||||
|
||||
when(config.brokerId()).thenReturn(8);
|
||||
|
||||
testWith(metadataVersion, config, faultHandler);
|
||||
|
||||
verify(config, times(1)).validateWithMetadataVersion(eq(metadataVersion));
|
||||
verifyNoMoreInteractions(faultHandler);
|
||||
}
|
||||
|
||||
@SuppressWarnings("ThrowableNotThrown")
|
||||
@Test
|
||||
void testInvokesFaultHandlerOnException() throws Exception {
|
||||
MetadataVersion metadataVersion = MetadataVersion.IBP_3_7_IV2;
|
||||
Exception exception = new Exception();
|
||||
KafkaConfig config = mock(KafkaConfig.class);
|
||||
FaultHandler faultHandler = mock(FaultHandler.class);
|
||||
|
||||
when(faultHandler.handleFault(any(), any())).thenReturn(new RuntimeException("returned exception"));
|
||||
when(config.brokerId()).thenReturn(8);
|
||||
willAnswer(invocation -> {
|
||||
throw exception;
|
||||
}).given(config).validateWithMetadataVersion(eq(metadataVersion));
|
||||
|
||||
testWith(metadataVersion, config, faultHandler);
|
||||
|
||||
verify(config, times(1)).validateWithMetadataVersion(eq(metadataVersion));
|
||||
verify(faultHandler, times(1)).handleFault(
|
||||
eq("Broker configuration does not support the cluster MetadataVersion"),
|
||||
eq(exception));
|
||||
}
|
||||
}
|
|
@ -24,7 +24,7 @@ import java.lang.{Long => JLong}
|
|||
import java.time.{Duration => JDuration}
|
||||
import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger}
|
||||
import java.util.concurrent.{CountDownLatch, ExecutionException, TimeUnit}
|
||||
import java.util.{Collections, Locale, Optional, Properties}
|
||||
import java.util.{Collections, Optional, Properties}
|
||||
import java.{time, util}
|
||||
import kafka.integration.KafkaServerTestHarness
|
||||
import kafka.server.KafkaConfig
|
||||
|
@ -2354,9 +2354,6 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
defaultConsumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false")
|
||||
defaultConsumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, testGroupId)
|
||||
defaultConsumerConfig.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, testClientId)
|
||||
// We need to set internal.leave.group.on.close to validate dynamic member removal, but it only works for ClassicConsumer
|
||||
// After KIP-1092, we can control dynamic member removal for both ClassicConsumer and AsyncConsumer
|
||||
defaultConsumerConfig.setProperty("internal.leave.group.on.close", "false")
|
||||
|
||||
val backgroundConsumerSet = new BackgroundConsumerSet(defaultConsumerConfig)
|
||||
groupInstanceSet.zip(topicSet).foreach { case (groupInstanceId, topic) =>
|
||||
|
@ -2405,14 +2402,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
var testGroupDescription = describeTestGroupResult.describedGroups().get(testGroupId).get()
|
||||
assertEquals(testGroupId, testGroupDescription.groupId)
|
||||
assertFalse(testGroupDescription.isSimpleConsumerGroup)
|
||||
|
||||
// Although we set `internal.leave.group.on.close` in the consumer, it only works for ClassicConsumer.
|
||||
// After KIP-1092, we can control dynamic member removal in consumer.close()
|
||||
if (groupProtocol == GroupProtocol.CLASSIC.name.toLowerCase(Locale.ROOT)) {
|
||||
assertEquals(3, testGroupDescription.members().size())
|
||||
} else if (groupProtocol == GroupProtocol.CONSUMER.name.toLowerCase(Locale.ROOT)) {
|
||||
assertEquals(2, testGroupDescription.members().size())
|
||||
}
|
||||
assertEquals(2, testGroupDescription.members().size())
|
||||
|
||||
// Test delete one static member
|
||||
removeMembersResult = client.removeMembersFromConsumerGroup(testGroupId,
|
||||
|
@ -2425,11 +2415,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
new DescribeConsumerGroupsOptions().includeAuthorizedOperations(true))
|
||||
testGroupDescription = describeTestGroupResult.describedGroups().get(testGroupId).get()
|
||||
|
||||
if (groupProtocol == GroupProtocol.CLASSIC.name.toLowerCase(Locale.ROOT)) {
|
||||
assertEquals(2, testGroupDescription.members().size())
|
||||
} else if (groupProtocol == GroupProtocol.CONSUMER.name.toLowerCase(Locale.ROOT)) {
|
||||
assertEquals(1, testGroupDescription.members().size())
|
||||
}
|
||||
assertEquals(1, testGroupDescription.members().size())
|
||||
|
||||
// Delete all active members remaining
|
||||
removeMembersResult = client.removeMembersFromConsumerGroup(testGroupId, new RemoveMembersFromConsumerGroupOptions())
|
||||
|
|
|
@ -21,7 +21,7 @@ import com.yammer.metrics.core.Metric
|
|||
import kafka.log.LogManager
|
||||
import kafka.server._
|
||||
import kafka.utils._
|
||||
import org.apache.kafka.common.errors.{ApiException, FencedLeaderEpochException, InconsistentTopicIdException, InvalidTxnStateException, NotLeaderOrFollowerException, OffsetNotAvailableException, OffsetOutOfRangeException, UnknownLeaderEpochException}
|
||||
import org.apache.kafka.common.errors.{ApiException, FencedLeaderEpochException, InconsistentTopicIdException, InvalidTxnStateException, NotLeaderOrFollowerException, OffsetNotAvailableException, OffsetOutOfRangeException, PolicyViolationException, UnknownLeaderEpochException}
|
||||
import org.apache.kafka.common.message.{AlterPartitionResponseData, FetchResponseData}
|
||||
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
|
||||
import org.apache.kafka.common.record.FileRecords.TimestampAndOffset
|
||||
|
@ -61,7 +61,7 @@ import org.apache.kafka.server.storage.log.{FetchIsolation, FetchParams, Unexpec
|
|||
import org.apache.kafka.server.util.{KafkaScheduler, MockTime}
|
||||
import org.apache.kafka.storage.internals.checkpoint.OffsetCheckpoints
|
||||
import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache
|
||||
import org.apache.kafka.storage.internals.log.{AppendOrigin, CleanerConfig, EpochEntry, LocalLog, LogAppendInfo, LogDirFailureChannel, LogLoader, LogOffsetMetadata, LogOffsetsListener, LogReadInfo, LogSegments, LogStartOffsetIncrementReason, ProducerStateManager, ProducerStateManagerConfig, UnifiedLog, VerificationGuard}
|
||||
import org.apache.kafka.storage.internals.log.{AppendOrigin, CleanerConfig, EpochEntry, LocalLog, LogAppendInfo, LogConfig, LogDirFailureChannel, LogLoader, LogOffsetMetadata, LogOffsetsListener, LogReadInfo, LogSegments, LogStartOffsetIncrementReason, ProducerStateManager, ProducerStateManagerConfig, UnifiedLog, VerificationGuard}
|
||||
import org.apache.kafka.storage.log.metrics.BrokerTopicStats
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
import org.junit.jupiter.params.provider.ValueSource
|
||||
|
@ -4030,4 +4030,46 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
alterPartitionManager)
|
||||
partition.tryCompleteDelayedRequests()
|
||||
}
|
||||
|
||||
@Test
|
||||
def testDeleteRecordsOnLeaderWithEmptyPolicy(): Unit = {
|
||||
val leaderEpoch = 5
|
||||
val partition = setupPartitionWithMocks(leaderEpoch, isLeader = true)
|
||||
|
||||
val emptyPolicyConfig = new LogConfig(util.Map.of(
|
||||
TopicConfig.CLEANUP_POLICY_CONFIG, ""
|
||||
))
|
||||
|
||||
val mockLog = mock(classOf[UnifiedLog])
|
||||
when(mockLog.config).thenReturn(emptyPolicyConfig)
|
||||
when(mockLog.logEndOffset).thenReturn(2L)
|
||||
when(mockLog.logStartOffset).thenReturn(0L)
|
||||
when(mockLog.highWatermark).thenReturn(2L)
|
||||
when(mockLog.maybeIncrementLogStartOffset(any(), any())).thenReturn(true)
|
||||
|
||||
partition.setLog(mockLog, false)
|
||||
|
||||
val result = partition.deleteRecordsOnLeader(1L)
|
||||
assertEquals(1L, result.requestedOffset)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testDeleteRecordsOnLeaderWithCompactPolicy(): Unit = {
|
||||
val leaderEpoch = 5
|
||||
val partition = setupPartitionWithMocks(leaderEpoch, isLeader = true)
|
||||
|
||||
val emptyPolicyConfig = new LogConfig(util.Map.of(
|
||||
TopicConfig.CLEANUP_POLICY_CONFIG, "compact"
|
||||
))
|
||||
|
||||
val mockLog = mock(classOf[UnifiedLog])
|
||||
when(mockLog.config).thenReturn(emptyPolicyConfig)
|
||||
when(mockLog.logEndOffset).thenReturn(2L)
|
||||
when(mockLog.logStartOffset).thenReturn(0L)
|
||||
when(mockLog.highWatermark).thenReturn(2L)
|
||||
when(mockLog.maybeIncrementLogStartOffset(any(), any())).thenReturn(true)
|
||||
|
||||
partition.setLog(mockLog, false)
|
||||
assertThrows(classOf[PolicyViolationException], () => partition.deleteRecordsOnLeader(1L))
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,7 +23,6 @@ import org.apache.kafka.common.config.ConfigDef.Importance.MEDIUM
|
|||
import org.apache.kafka.common.config.ConfigDef.Type.INT
|
||||
import org.apache.kafka.common.config.{ConfigException, SslConfigs, TopicConfig}
|
||||
import org.apache.kafka.common.errors.InvalidConfigurationException
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
|
@ -429,21 +428,4 @@ class LogConfigTest {
|
|||
logProps.put(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, deleteOnDisable.toString)
|
||||
LogConfig.validate(logProps)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testValidateWithMetadataVersionJbodSupport(): Unit = {
|
||||
def validate(metadataVersion: MetadataVersion, jbodConfig: Boolean): Unit =
|
||||
KafkaConfig.fromProps(
|
||||
TestUtils.createBrokerConfig(nodeId = 0, logDirCount = if (jbodConfig) 2 else 1)
|
||||
).validateWithMetadataVersion(metadataVersion)
|
||||
|
||||
validate(MetadataVersion.IBP_3_6_IV2, jbodConfig = false)
|
||||
validate(MetadataVersion.IBP_3_7_IV0, jbodConfig = false)
|
||||
validate(MetadataVersion.IBP_3_7_IV2, jbodConfig = false)
|
||||
assertThrows(classOf[IllegalArgumentException], () =>
|
||||
validate(MetadataVersion.IBP_3_6_IV2, jbodConfig = true))
|
||||
assertThrows(classOf[IllegalArgumentException], () =>
|
||||
validate(MetadataVersion.IBP_3_7_IV0, jbodConfig = true))
|
||||
validate(MetadataVersion.IBP_3_7_IV2, jbodConfig = true)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,9 +26,9 @@ import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity
|
|||
import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse
|
||||
import org.apache.kafka.common.message.SyncGroupRequestData.SyncGroupRequestAssignment
|
||||
import org.apache.kafka.common.message.WriteTxnMarkersRequestData.{WritableTxnMarker, WritableTxnMarkerTopic}
|
||||
import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AddOffsetsToTxnResponseData, ConsumerGroupDescribeRequestData, ConsumerGroupDescribeResponseData, ConsumerGroupHeartbeatRequestData, ConsumerGroupHeartbeatResponseData, DeleteGroupsRequestData, DeleteGroupsResponseData, DescribeGroupsRequestData, DescribeGroupsResponseData, EndTxnRequestData, HeartbeatRequestData, HeartbeatResponseData, InitProducerIdRequestData, JoinGroupRequestData, JoinGroupResponseData, LeaveGroupResponseData, ListGroupsRequestData, ListGroupsResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteRequestData, OffsetDeleteResponseData, OffsetFetchRequestData, OffsetFetchResponseData, ShareGroupDescribeRequestData, ShareGroupDescribeResponseData, ShareGroupHeartbeatRequestData, ShareGroupHeartbeatResponseData, SyncGroupRequestData, SyncGroupResponseData, TxnOffsetCommitRequestData, TxnOffsetCommitResponseData, WriteTxnMarkersRequestData}
|
||||
import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AddOffsetsToTxnResponseData, ConsumerGroupDescribeRequestData, ConsumerGroupDescribeResponseData, ConsumerGroupHeartbeatRequestData, ConsumerGroupHeartbeatResponseData, DeleteGroupsRequestData, DeleteGroupsResponseData, DescribeGroupsRequestData, DescribeGroupsResponseData, EndTxnRequestData, HeartbeatRequestData, HeartbeatResponseData, InitProducerIdRequestData, JoinGroupRequestData, JoinGroupResponseData, LeaveGroupResponseData, ListGroupsRequestData, ListGroupsResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteRequestData, OffsetDeleteResponseData, OffsetFetchRequestData, OffsetFetchResponseData, ShareGroupDescribeRequestData, ShareGroupDescribeResponseData, ShareGroupHeartbeatRequestData, ShareGroupHeartbeatResponseData, StreamsGroupDescribeRequestData, StreamsGroupDescribeResponseData, StreamsGroupHeartbeatRequestData, StreamsGroupHeartbeatResponseData, SyncGroupRequestData, SyncGroupResponseData, TxnOffsetCommitRequestData, TxnOffsetCommitResponseData, WriteTxnMarkersRequestData}
|
||||
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
|
||||
import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, AddOffsetsToTxnRequest, AddOffsetsToTxnResponse, ConsumerGroupDescribeRequest, ConsumerGroupDescribeResponse, ConsumerGroupHeartbeatRequest, ConsumerGroupHeartbeatResponse, DeleteGroupsRequest, DeleteGroupsResponse, DescribeGroupsRequest, DescribeGroupsResponse, EndTxnRequest, EndTxnResponse, HeartbeatRequest, HeartbeatResponse, InitProducerIdRequest, InitProducerIdResponse, JoinGroupRequest, JoinGroupResponse, LeaveGroupRequest, LeaveGroupResponse, ListGroupsRequest, ListGroupsResponse, OffsetCommitRequest, OffsetCommitResponse, OffsetDeleteRequest, OffsetDeleteResponse, OffsetFetchRequest, OffsetFetchResponse, ShareGroupDescribeRequest, ShareGroupDescribeResponse, ShareGroupHeartbeatRequest, ShareGroupHeartbeatResponse, SyncGroupRequest, SyncGroupResponse, TxnOffsetCommitRequest, TxnOffsetCommitResponse, WriteTxnMarkersRequest, WriteTxnMarkersResponse}
|
||||
import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, AddOffsetsToTxnRequest, AddOffsetsToTxnResponse, ConsumerGroupDescribeRequest, ConsumerGroupDescribeResponse, ConsumerGroupHeartbeatRequest, ConsumerGroupHeartbeatResponse, DeleteGroupsRequest, DeleteGroupsResponse, DescribeGroupsRequest, DescribeGroupsResponse, EndTxnRequest, EndTxnResponse, HeartbeatRequest, HeartbeatResponse, InitProducerIdRequest, InitProducerIdResponse, JoinGroupRequest, JoinGroupResponse, LeaveGroupRequest, LeaveGroupResponse, ListGroupsRequest, ListGroupsResponse, OffsetCommitRequest, OffsetCommitResponse, OffsetDeleteRequest, OffsetDeleteResponse, OffsetFetchRequest, OffsetFetchResponse, ShareGroupDescribeRequest, ShareGroupDescribeResponse, ShareGroupHeartbeatRequest, ShareGroupHeartbeatResponse, StreamsGroupDescribeRequest, StreamsGroupDescribeResponse, StreamsGroupHeartbeatRequest, StreamsGroupHeartbeatResponse, SyncGroupRequest, SyncGroupResponse, TxnOffsetCommitRequest, TxnOffsetCommitResponse, WriteTxnMarkersRequest, WriteTxnMarkersResponse}
|
||||
import org.apache.kafka.common.serialization.StringSerializer
|
||||
import org.apache.kafka.common.test.ClusterInstance
|
||||
import org.apache.kafka.common.utils.ProducerIdAndEpoch
|
||||
|
@ -768,6 +768,21 @@ class GroupCoordinatorBaseRequestTest(cluster: ClusterInstance) {
|
|||
shareGroupDescribeResponse.data.groups.asScala.toList
|
||||
}
|
||||
|
||||
protected def streamsGroupDescribe(
|
||||
groupIds: List[String],
|
||||
includeAuthorizedOperations: Boolean = false,
|
||||
version: Short = ApiKeys.STREAMS_GROUP_DESCRIBE.latestVersion(isUnstableApiEnabled)
|
||||
): List[StreamsGroupDescribeResponseData.DescribedGroup] = {
|
||||
val streamsGroupDescribeRequest = new StreamsGroupDescribeRequest.Builder(
|
||||
new StreamsGroupDescribeRequestData()
|
||||
.setGroupIds(groupIds.asJava)
|
||||
.setIncludeAuthorizedOperations(includeAuthorizedOperations)
|
||||
).build(version)
|
||||
|
||||
val streamsGroupDescribeResponse = connectAndReceive[StreamsGroupDescribeResponse](streamsGroupDescribeRequest)
|
||||
streamsGroupDescribeResponse.data.groups.asScala.toList
|
||||
}
|
||||
|
||||
protected def heartbeat(
|
||||
groupId: String,
|
||||
generationId: Int,
|
||||
|
@ -855,6 +870,41 @@ class GroupCoordinatorBaseRequestTest(cluster: ClusterInstance) {
|
|||
shareGroupHeartbeatResponse.data
|
||||
}
|
||||
|
||||
protected def streamsGroupHeartbeat(
|
||||
groupId: String,
|
||||
memberId: String = "",
|
||||
memberEpoch: Int = 0,
|
||||
rebalanceTimeoutMs: Int = -1,
|
||||
activeTasks: List[StreamsGroupHeartbeatRequestData.TaskIds] = null,
|
||||
standbyTasks: List[StreamsGroupHeartbeatRequestData.TaskIds] = null,
|
||||
warmupTasks: List[StreamsGroupHeartbeatRequestData.TaskIds] = null,
|
||||
topology: StreamsGroupHeartbeatRequestData.Topology = null,
|
||||
expectedError: Errors = Errors.NONE,
|
||||
version: Short = ApiKeys.STREAMS_GROUP_HEARTBEAT.latestVersion(isUnstableApiEnabled)
|
||||
): StreamsGroupHeartbeatResponseData = {
|
||||
val streamsGroupHeartbeatRequest = new StreamsGroupHeartbeatRequest.Builder(
|
||||
new StreamsGroupHeartbeatRequestData()
|
||||
.setGroupId(groupId)
|
||||
.setMemberId(memberId)
|
||||
.setMemberEpoch(memberEpoch)
|
||||
.setRebalanceTimeoutMs(rebalanceTimeoutMs)
|
||||
.setActiveTasks(activeTasks.asJava)
|
||||
.setStandbyTasks(standbyTasks.asJava)
|
||||
.setWarmupTasks(warmupTasks.asJava)
|
||||
.setTopology(topology)
|
||||
).build(version)
|
||||
|
||||
// Send the request until receiving a successful response. There is a delay
|
||||
// here because the group coordinator is loaded in the background.
|
||||
var streamsGroupHeartbeatResponse: StreamsGroupHeartbeatResponse = null
|
||||
TestUtils.waitUntilTrue(() => {
|
||||
streamsGroupHeartbeatResponse = connectAndReceive[StreamsGroupHeartbeatResponse](streamsGroupHeartbeatRequest)
|
||||
streamsGroupHeartbeatResponse.data.errorCode == expectedError.code
|
||||
}, msg = s"Could not heartbeat successfully. Last response $streamsGroupHeartbeatResponse.")
|
||||
|
||||
streamsGroupHeartbeatResponse.data
|
||||
}
|
||||
|
||||
protected def leaveGroupWithNewProtocol(
|
||||
groupId: String,
|
||||
memberId: String
|
||||
|
|
|
@ -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
|
||||
}
|
||||
}
|
||||
}
|
|
@ -4315,7 +4315,7 @@ $ bin/kafka-topics.sh --create --topic tieredTopic --bootstrap-server localhost:
|
|||
|
||||
<p>Lastly, we can try to consume some data from the beginning and print offset number, to make sure it will successfully fetch offset 0 from the remote storage.</p>
|
||||
|
||||
<pre><code class="language-bash">$ bin/kafka-console-consumer.sh --topic tieredTopic --from-beginning --max-messages 1 --bootstrap-server localhost:9092 --property print.offset=true</code></pre>
|
||||
<pre><code class="language-bash">$ bin/kafka-console-consumer.sh --topic tieredTopic --from-beginning --max-messages 1 --bootstrap-server localhost:9092 --formatter-property print.offset=true</code></pre>
|
||||
|
||||
<p>In KRaft mode, you can disable tiered storage at the topic level, to make the remote logs as read-only logs, or completely delete all remote logs.</p>
|
||||
|
||||
|
|
|
@ -1222,7 +1222,7 @@ streamsConfig.put(StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG, CustomRocksD
|
|||
These optimizations include moving/reducing repartition topics and reusing the source topic as the changelog for source KTables. These optimizations will save on network traffic and storage in Kafka without changing the semantics of your applications. Enabling them is recommended.
|
||||
</p>
|
||||
<p>
|
||||
Note that as of 2.3, you need to do two things to enable optimizations. In addition to setting this config to <code>StreamsConfig.OPTIMIZE</code>, you'll need to pass in your
|
||||
Note that you need to do two things to enable optimizations. In addition to setting this config to <code>StreamsConfig.OPTIMIZE</code>, you'll need to pass in your
|
||||
configuration properties when building your topology by using the overloaded <code>StreamsBuilder.build(Properties)</code> method.
|
||||
For example <code>KafkaStreams myStream = new KafkaStreams(streamsBuilder.build(properties), properties)</code>.
|
||||
</p>
|
||||
|
@ -1235,7 +1235,7 @@ streamsConfig.put(StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG, CustomRocksD
|
|||
The version you are upgrading from. It is important to set this config when performing a rolling upgrade to certain versions, as described in the upgrade guide.
|
||||
You should set this config to the appropriate version before bouncing your instances and upgrading them to the newer version. Once everyone is on the
|
||||
newer version, you should remove this config and do a second rolling bounce. It is only necessary to set this config and follow the two-bounce upgrade path
|
||||
when upgrading from below version 2.0, or when upgrading to 2.4+ from any version lower than 2.4.
|
||||
when upgrading to 3.4+ from any version lower than 3.4.
|
||||
</div>
|
||||
</blockquote>
|
||||
</div>
|
||||
|
|
|
@ -217,15 +217,15 @@ TimeWindowedDeserializer<String> deserializer = new TimeWindowedDeserializ
|
|||
<h4>Usage in Command Line</h4>
|
||||
<p>When using command-line tools (like <code>bin/kafka-console-consumer.sh</code>), you can configure windowed deserializers by passing the inner class and window size via configuration properties. The property names use a prefix pattern:</p>
|
||||
<pre class="line-numbers"><code class="language-bash"># Time windowed deserializer configuration
|
||||
--property print.key=true \
|
||||
--property key.deserializer=org.apache.kafka.streams.kstream.TimeWindowedDeserializer \
|
||||
--property key.deserializer.windowed.inner.deserializer.class=org.apache.kafka.common.serialization.StringDeserializer \
|
||||
--property key.deserializer.window.size.ms=500
|
||||
--formatter-property print.key=true \
|
||||
--formatter-property key.deserializer=org.apache.kafka.streams.kstream.TimeWindowedDeserializer \
|
||||
--formatter-property key.deserializer.windowed.inner.deserializer.class=org.apache.kafka.common.serialization.StringDeserializer \
|
||||
--formatter-property key.deserializer.window.size.ms=500
|
||||
|
||||
# Session windowed deserializer configuration
|
||||
--property print.key=true \
|
||||
--property key.deserializer=org.apache.kafka.streams.kstream.SessionWindowedDeserializer \
|
||||
--property key.deserializer.windowed.inner.deserializer.class=org.apache.kafka.common.serialization.StringDeserializer</code></pre>
|
||||
--formatter-property print.key=true \
|
||||
--formatter-property key.deserializer=org.apache.kafka.streams.kstream.SessionWindowedDeserializer \
|
||||
--formatter-property key.deserializer.windowed.inner.deserializer.class=org.apache.kafka.common.serialization.StringDeserializer</code></pre>
|
||||
|
||||
<h4>Deprecated Configs</h4>
|
||||
<p>The following <code>StreamsConfig</code> parameters are deprecated in favor of passing parameters directly to serializer/deserializer constructors:</p>
|
||||
|
|
|
@ -175,10 +175,10 @@ and inspect the output of the WordCount demo application by reading from its out
|
|||
<pre><code class="language-bash">$ bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 \
|
||||
--topic streams-wordcount-output \
|
||||
--from-beginning \
|
||||
--property print.key=true \
|
||||
--property print.value=true \
|
||||
--property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \
|
||||
--property value.deserializer=org.apache.kafka.common.serialization.LongDeserializer</code></pre>
|
||||
--formatter-property print.key=true \
|
||||
--formatter-property print.value=true \
|
||||
--formatter-property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \
|
||||
--formatter-property value.deserializer=org.apache.kafka.common.serialization.LongDeserializer</code></pre>
|
||||
|
||||
|
||||
<h4><a id="quickstart_streams_process" href="#quickstart_streams_process">Step 5: Process some data</a></h4>
|
||||
|
@ -197,10 +197,10 @@ This message will be processed by the Wordcount application and the following ou
|
|||
<pre><code class="language-bash">$ bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 \
|
||||
--topic streams-wordcount-output \
|
||||
--from-beginning \
|
||||
--property print.key=true \
|
||||
--property print.value=true \
|
||||
--property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \
|
||||
--property value.deserializer=org.apache.kafka.common.serialization.LongDeserializer
|
||||
--formatter-property print.key=true \
|
||||
--formatter-property print.value=true \
|
||||
--formatter-property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \
|
||||
--formatter-property value.deserializer=org.apache.kafka.common.serialization.LongDeserializer
|
||||
|
||||
all 1
|
||||
streams 1
|
||||
|
@ -225,10 +225,10 @@ In your other terminal in which the console consumer is running, you will observ
|
|||
<pre><code class="language-bash">$ bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 \
|
||||
--topic streams-wordcount-output \
|
||||
--from-beginning \
|
||||
--property print.key=true \
|
||||
--property print.value=true \
|
||||
--property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \
|
||||
--property value.deserializer=org.apache.kafka.common.serialization.LongDeserializer
|
||||
--formatter-property print.key=true \
|
||||
--formatter-property print.value=true \
|
||||
--formatter-property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \
|
||||
--formatter-property value.deserializer=org.apache.kafka.common.serialization.LongDeserializer
|
||||
|
||||
all 1
|
||||
streams 1
|
||||
|
@ -255,10 +255,10 @@ The <b>streams-wordcount-output</b> topic will subsequently show the correspondi
|
|||
<pre><code class="language-bash">$ bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 \
|
||||
--topic streams-wordcount-output \
|
||||
--from-beginning \
|
||||
--property print.key=true \
|
||||
--property print.value=true \
|
||||
--property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \
|
||||
--property value.deserializer=org.apache.kafka.common.serialization.LongDeserializer
|
||||
--formatter-property print.key=true \
|
||||
--formatter-property print.value=true \
|
||||
--formatter-property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \
|
||||
--formatter-property value.deserializer=org.apache.kafka.common.serialization.LongDeserializer
|
||||
|
||||
all 1
|
||||
streams 1
|
||||
|
|
|
@ -35,9 +35,8 @@
|
|||
|
||||
<p>
|
||||
Upgrading from any older version to {{fullDotVersion}} is possible: if upgrading from 3.4 or below, you will need to do two rolling bounces, where during the first rolling bounce phase you set the config <code>upgrade.from="older version"</code>
|
||||
(possible values are <code>"0.10.0" - "3.4"</code>) and during the second you remove it. This is required to safely handle 3 changes. The first is introduction of the new cooperative rebalancing protocol of the embedded consumer. The second is a change in foreign-key join serialization format.
|
||||
Note that you will remain using the old eager rebalancing protocol if you skip or delay the second rolling bounce, but you can safely switch over to cooperative at any time once the entire group is on 2.4+ by removing the config value and bouncing. For more details please refer to
|
||||
<a href="https://cwiki.apache.org/confluence/x/vAclBg">KIP-429</a>. The third is a change in the serialization format for an internal repartition topic. For more details, please refer to <a href="https://cwiki.apache.org/confluence/x/P5VbDg">KIP-904</a>:
|
||||
(possible values are <code>"2.4" - "3.4"</code>) and during the second you remove it. This is required to safely handle 2 changes. The first is a change in foreign-key join serialization format.
|
||||
The second is a change in the serialization format for an internal repartition topic. For more details, please refer to <a href="https://cwiki.apache.org/confluence/x/P5VbDg">KIP-904</a>:
|
||||
</p>
|
||||
<ul>
|
||||
<li> prepare your application instances for a rolling bounce and make sure that config <code>upgrade.from</code> is set to the version from which it is being upgrade.</li>
|
||||
|
@ -45,24 +44,12 @@
|
|||
<li> prepare your newly deployed {{fullDotVersion}} application instances for a second round of rolling bounces; make sure to remove the value for config <code>upgrade.from</code> </li>
|
||||
<li> bounce each instance of your application once more to complete the upgrade </li>
|
||||
</ul>
|
||||
<p> As an alternative, an offline upgrade is also possible. Upgrading from any versions as old as 0.10.0.x to {{fullDotVersion}} in offline mode require the following steps: </p>
|
||||
<p> As an alternative, an offline upgrade is also possible. Upgrading from any versions as old as 0.11.0.x to {{fullDotVersion}} in offline mode require the following steps: </p>
|
||||
<ul>
|
||||
<li> stop all old (e.g., 0.10.0.x) application instances </li>
|
||||
<li> stop all old (e.g., 0.11.0.x) application instances </li>
|
||||
<li> update your code and swap old code and jar file with new code and new jar file </li>
|
||||
<li> restart all new ({{fullDotVersion}}) application instances </li>
|
||||
</ul>
|
||||
<p>
|
||||
Note: The cooperative rebalancing protocol has been the default since 2.4, but we have continued to support the
|
||||
eager rebalancing protocol to provide users an upgrade path. This support will be dropped in a future release,
|
||||
so any users still on the eager protocol should prepare to finish upgrading their applications to the cooperative protocol in version 3.1.
|
||||
This only affects users who are still on a version older than 2.4, and users who have upgraded already but have not yet
|
||||
removed the <code>upgrade.from</code> config that they set when upgrading from a version below 2.4.
|
||||
Users fitting into the latter case will simply need to unset this config when upgrading beyond 3.1,
|
||||
while users in the former case will need to follow a slightly different upgrade path if they attempt to upgrade from 2.3 or below to a version above 3.1.
|
||||
Those applications will need to go through a bridge release, by first upgrading to a version between 2.4 - 3.1 and setting the <code>upgrade.from</code> config,
|
||||
then removing that config and upgrading to the final version above 3.1. See <a href="https://issues.apache.org/jira/browse/KAFKA-8575">KAFKA-8575</a>
|
||||
for more details.
|
||||
</p>
|
||||
|
||||
<p>For a table that shows Streams API compatibility with Kafka broker versions, see <a href="#streams_api_broker_compat">Broker Compatibility</a>.</p>
|
||||
|
||||
|
@ -121,26 +108,10 @@
|
|||
|
||||
<p>Since 2.6.0 release, Kafka Streams depends on a RocksDB version that requires MacOS 10.14 or higher.</p>
|
||||
|
||||
<p>
|
||||
To run a Kafka Streams application version 2.2.1, 2.3.0, or higher a broker version 0.11.0 or higher is required
|
||||
and the on-disk message format must be 0.11 or higher.
|
||||
Brokers must be on version 0.10.1 or higher to run a Kafka Streams application version 0.10.1 to 2.2.0.
|
||||
Additionally, on-disk message format must be 0.10 or higher to run a Kafka Streams application version 1.0 to 2.2.0.
|
||||
For Kafka Streams 0.10.0, broker version 0.10.0 or higher is required.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
In deprecated <code>KStreamBuilder</code> class, when a <code>KTable</code> is created from a source topic via <code>KStreamBuilder.table()</code>, its materialized state store
|
||||
will reuse the source topic as its changelog topic for restoring, and will disable logging to avoid appending new updates to the source topic; in the <code>StreamsBuilder</code> class introduced in 1.0, this behavior was changed
|
||||
accidentally: we still reuse the source topic as the changelog topic for restoring, but will also create a separate changelog topic to append the update records from source topic to. In the 2.0 release, we have fixed this issue and now users
|
||||
can choose whether or not to reuse the source topic based on the <code>StreamsConfig#TOPOLOGY_OPTIMIZATION_CONFIG</code>: if you are upgrading from the old <code>KStreamBuilder</code> class and hence you need to change your code to use
|
||||
the new <code>StreamsBuilder</code>, you should set this config value to <code>StreamsConfig#OPTIMIZE</code> to continue reusing the source topic; if you are upgrading from 1.0 or 1.1 where you are already using <code>StreamsBuilder</code> and hence have already
|
||||
created a separate changelog topic, you should set this config value to <code>StreamsConfig#NO_OPTIMIZATION</code> when upgrading to {{fullDotVersion}} in order to use that changelog topic for restoring the state store.
|
||||
More details about the new config <code>StreamsConfig#TOPOLOGY_OPTIMIZATION_CONFIG</code> can be found in <a href="https://cwiki.apache.org/confluence/x/V53LB">KIP-295</a>.
|
||||
</p>
|
||||
|
||||
<h3><a id="streams_api_changes_410" href="#streams_api_changes_410">Streams API changes in 4.1.0</a></h3>
|
||||
|
||||
<p><b>Note:</b> Kafka Streams 4.1.0 contains a critical memory leak bug (<a href="https://issues.apache.org/jira/browse/KAFKA-19748">KAFKA-19748</a>) that affects users of range scans and certain DSL operators (session windows, sliding windows, stream-stream joins, foreign-key joins). Users running Kafka Streams should consider upgrading directly to 4.1.1 when available.</p>
|
||||
|
||||
<h4>Early Access of the Streams Rebalance Protocol</h4>
|
||||
|
||||
<p>
|
||||
|
@ -1150,705 +1121,9 @@
|
|||
Hence, this feature won't be supported in the future any longer and you need to updated your code accordingly.
|
||||
If you use a custom <code>PartitionGrouper</code> and stop to use it, the created tasks might change.
|
||||
Hence, you will need to reset your application to upgrade it.
|
||||
|
||||
|
||||
<h3 class="anchor-heading"><a id="streams_api_changes_230" class="anchor-link"></a><a href="#streams_api_changes_230">Streams API changes in 2.3.0</a></h3>
|
||||
|
||||
<p>Version 2.3.0 adds the Suppress operator to the <code>kafka-streams-scala</code> Ktable API.</p>
|
||||
|
||||
<p>
|
||||
As of 2.3.0 Streams now offers an in-memory version of the window (<a href="https://cwiki.apache.org/confluence/x/6AQlBg">KIP-428</a>)
|
||||
and the session (<a href="https://cwiki.apache.org/confluence/x/DiqGBg">KIP-445</a>) store, in addition to the persistent ones based on RocksDB.
|
||||
The new public interfaces <code>inMemoryWindowStore()</code> and <code>inMemorySessionStore()</code> are added to <code>Stores</code> and provide the built-in in-memory window or session store.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
As of 2.3.0 we've updated how to turn on optimizations. Now to enable optimizations, you need to do two things.
|
||||
First add this line to your properties <code>properties.setProperty(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, StreamsConfig.OPTIMIZE);</code>, as you have done before.
|
||||
Second, when constructing your <code>KafkaStreams</code> instance, you'll need to pass your configuration properties when building your
|
||||
topology by using the overloaded <code>StreamsBuilder.build(Properties)</code> method.
|
||||
For example <code>KafkaStreams myStream = new KafkaStreams(streamsBuilder.build(properties), properties)</code>.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
In 2.3.0 we have added default implementation to <code>close()</code> and <code>configure()</code> for <code>Serializer</code>,
|
||||
<code>Deserializer</code> and <code>Serde</code> so that they can be implemented by lambda expression.
|
||||
For more details please read <a href="https://cwiki.apache.org/confluence/x/fgw0BQ">KIP-331</a>.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
To improve operator semantics, new store types are added that allow storing an additional timestamp per key-value pair or window.
|
||||
Some DSL operators (for example KTables) are using those new stores.
|
||||
Hence, you can now retrieve the last update timestamp via Interactive Queries if you specify
|
||||
<code>TimestampedKeyValueStoreType</code> or <code>TimestampedWindowStoreType</code> as your <code>QueryableStoreType</code>.
|
||||
While this change is mainly transparent, there are some corner cases that may require code changes:
|
||||
<strong>Caution: If you receive an untyped store and use a cast, you might need to update your code to cast to the correct type.
|
||||
Otherwise, you might get an exception similar to
|
||||
<code>java.lang.ClassCastException: class org.apache.kafka.streams.state.ValueAndTimestamp cannot be cast to class YOUR-VALUE-TYPE</code>
|
||||
upon getting a value from the store.</strong>
|
||||
Additionally, <code>TopologyTestDriver#getStateStore()</code> only returns non-built-in stores and throws an exception if a built-in store is accessed.
|
||||
For more details please read <a href="https://cwiki.apache.org/confluence/x/0j6HB">KIP-258</a>.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
To improve type safety, a new operator <code>KStream#flatTransformValues</code> is added.
|
||||
For more details please read <a href="https://cwiki.apache.org/confluence/x/bUgYBQ">KIP-313</a>.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
Kafka Streams used to set the configuration parameter <code>max.poll.interval.ms</code> to <code>Integer.MAX_VALUE</code>.
|
||||
This default value is removed and Kafka Streams uses the consumer default value now.
|
||||
For more details please read <a href="https://cwiki.apache.org/confluence/x/1COGBg">KIP-442</a>.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
Default configuration for repartition topic was changed:
|
||||
The segment size for index files (<code>segment.index.bytes</code>) is no longer 50MB, but uses the cluster default.
|
||||
Similarly, the configuration <code>segment.ms</code> in no longer 10 minutes, but uses the cluster default configuration.
|
||||
Lastly, the retention period (<code>retention.ms</code>) is changed from <code>Long.MAX_VALUE</code> to <code>-1</code> (infinite).
|
||||
For more details please read <a href="https://cwiki.apache.org/confluence/x/4iOGBg">KIP-443</a>.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
To avoid memory leaks, <code>RocksDBConfigSetter</code> has a new <code>close()</code> method that is called on shutdown.
|
||||
Users should implement this method to release any memory used by RocksDB config objects, by closing those objects.
|
||||
For more details please read <a href="https://cwiki.apache.org/confluence/x/QhaZBg">KIP-453</a>.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
RocksDB dependency was updated to version <code>5.18.3</code>.
|
||||
The new version allows to specify more RocksDB configurations, including <code>WriteBufferManager</code> which helps to limit RocksDB off-heap memory usage.
|
||||
For more details please read <a href="https://issues.apache.org/jira/browse/KAFKA-8215">KAFKA-8215</a>.
|
||||
</p>
|
||||
|
||||
<h3 class="anchor-heading"><a id="streams_api_changes_220" class="anchor-link"></a><a href="#streams_api_changes_220">Streams API changes in 2.2.0</a></h3>
|
||||
<p>
|
||||
We've simplified the <code>KafkaStreams#state</code> transition diagram during the starting up phase a bit in 2.2.0: in older versions the state will transit from <code>CREATED</code> to <code>RUNNING</code>, and then to <code>REBALANCING</code> to get the first
|
||||
stream task assignment, and then back to <code>RUNNING</code>; starting in 2.2.0 it will transit from <code>CREATED</code> directly to <code>REBALANCING</code> and then to <code>RUNNING</code>.
|
||||
If you have registered a <code>StateListener</code> that captures state transition events, you may need to adjust your listener implementation accordingly for this simplification (in practice, your listener logic should be very unlikely to be affected at all).
|
||||
</p>
|
||||
|
||||
<p>
|
||||
In <code>WindowedSerdes</code>, we've added a new static constructor to return a <code>TimeWindowSerde</code> with configurable window size. This is to help users to construct time window serdes to read directly from a time-windowed store's changelog.
|
||||
More details can be found in <a href="https://cwiki.apache.org/confluence/x/WYTQBQ">KIP-393</a>.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
In 2.2.0 we have extended a few public interfaces including <code>KafkaStreams</code> to extend <code>AutoCloseable</code> so that they can be
|
||||
used in a try-with-resource statement. For a full list of public interfaces that get impacted please read <a href="https://cwiki.apache.org/confluence/x/-AeQBQ">KIP-376</a>.
|
||||
</p>
|
||||
|
||||
<h3 class="anchor-heading"><a id="streams_api_changes_210" class="anchor-link"></a><a href="#streams_api_changes_210">Streams API changes in 2.1.0</a></h3>
|
||||
<p>
|
||||
We updated <code>TopologyDescription</code> API to allow for better runtime checking.
|
||||
Users are encouraged to use <code>#topicSet()</code> and <code>#topicPattern()</code> accordingly on <code>TopologyDescription.Source</code> nodes,
|
||||
instead of using <code>#topics()</code>, which has since been deprecated. Similarly, use <code>#topic()</code> and <code>#topicNameExtractor()</code>
|
||||
to get descriptions of <code>TopologyDescription.Sink</code> nodes. For more details, see
|
||||
<a href="https://cwiki.apache.org/confluence/x/NQU0BQ">KIP-321</a>.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
We've added a new class <code>Grouped</code> and deprecated <code>Serialized</code>. The intent of adding <code>Grouped</code> is the ability to
|
||||
name repartition topics created when performing aggregation operations. Users can name the potential repartition topic using the
|
||||
<code>Grouped#as()</code> method which takes a <code>String</code> and is used as part of the repartition topic name. The resulting repartition
|
||||
topic name will still follow the pattern of <code>${application-id}->name<-repartition</code>. The <code>Grouped</code> class is now favored over
|
||||
<code>Serialized</code> in <code>KStream#groupByKey()</code>, <code>KStream#groupBy()</code>, and <code>KTable#groupBy()</code>.
|
||||
Note that Kafka Streams does not automatically create repartition topics for aggregation operations.
|
||||
|
||||
Additionally, we've updated the <code>Joined</code> class with a new method <code>Joined#withName</code>
|
||||
enabling users to name any repartition topics required for performing Stream/Stream or Stream/Table join. For more details repartition
|
||||
topic naming, see <a href="https://cwiki.apache.org/confluence/x/mgJ1BQ">KIP-372</a>.
|
||||
|
||||
As a result we've updated the Kafka Streams Scala API and removed the <code>Serialized</code> class in favor of adding <code>Grouped</code>.
|
||||
If you just rely on the implicit <code>Serialized</code>, you just need to recompile; if you pass in <code>Serialized</code> explicitly, sorry you'll have to make code changes.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
We've added a new config named <code>max.task.idle.ms</code> to allow users specify how to handle out-of-order data within a task that may be processing multiple
|
||||
topic-partitions (see <a href="/{{version}}/documentation/streams/core-concepts.html#streams_out_of_ordering">Out-of-Order Handling</a> section for more details).
|
||||
The default value is set to <code>0</code>, to favor minimized latency over synchronization between multiple input streams from topic-partitions.
|
||||
If users would like to wait for longer time when some of the topic-partitions do not have data available to process and hence cannot determine its corresponding stream time,
|
||||
they can override this config to a larger value.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
We've added the missing <code>SessionBytesStoreSupplier#retentionPeriod()</code> to be consistent with the <code>WindowBytesStoreSupplier</code> which allows users to get the specified retention period for session-windowed stores.
|
||||
We've also added the missing <code>StoreBuilder#withCachingDisabled()</code> to allow users to turn off caching for their customized stores.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
We added a new serde for UUIDs (<code>Serdes.UUIDSerde</code>) that you can use via <code>Serdes.UUID()</code>
|
||||
(cf. <a href="https://cwiki.apache.org/confluence/x/26hjB">KIP-206</a>).
|
||||
</p>
|
||||
|
||||
<p>
|
||||
We updated a list of methods that take <code>long</code> arguments as either timestamp (fix point) or duration (time period)
|
||||
and replaced them with <code>Instant</code> and <code>Duration</code> parameters for improved semantics.
|
||||
Some old methods base on <code>long</code> are deprecated and users are encouraged to update their code.
|
||||
<br />
|
||||
In particular, aggregation windows (hopping/tumbling/unlimited time windows and session windows) as well as join windows now take <code>Duration</code>
|
||||
arguments to specify window size, hop, and gap parameters.
|
||||
Also, window sizes and retention times are now specified as <code>Duration</code> type in <code>Stores</code> class.
|
||||
The <code>Window</code> class has new methods <code>#startTime()</code> and <code>#endTime()</code> that return window start/end timestamp as <code>Instant</code>.
|
||||
For interactive queries, there are new <code>#fetch(...)</code> overloads taking <code>Instant</code> arguments.
|
||||
Additionally, punctuations are now registered via <code>ProcessorContext#schedule(Duration interval, ...)</code>.
|
||||
For more details, see <a href="https://cwiki.apache.org/confluence/x/IBNPBQ">KIP-358</a>.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
We deprecated <code>KafkaStreams#close(...)</code> and replaced it with <code>KafkaStreams#close(Duration)</code> that accepts a single timeout argument
|
||||
Note: the new <code>#close(Duration)</code> method has improved (but slightly different) semantics.
|
||||
For more details, see <a href="https://cwiki.apache.org/confluence/x/IBNPBQ">KIP-358</a>.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
The newly exposed <code>AdminClient</code> metrics are now available when calling the <code>KafkaStream#metrics()</code> method.
|
||||
For more details on exposing <code>AdminClients</code> metrics
|
||||
see <a href="https://cwiki.apache.org/confluence/x/lQg0BQ">KIP-324</a>
|
||||
</p>
|
||||
|
||||
<p>
|
||||
We deprecated the notion of segments in window stores as those are intended to be an implementation details.
|
||||
Thus, method <code>Windows#segments()</code> and variable <code>Windows#segments</code> were deprecated.
|
||||
If you implement custom windows, you should update your code accordingly.
|
||||
Similarly, <code>WindowBytesStoreSupplier#segments()</code> was deprecated and replaced with <code>WindowBytesStoreSupplier#segmentInterval()</code>.
|
||||
If you implement custom window store, you need to update your code accordingly.
|
||||
Finally, <code>Stores#persistentWindowStore(...)</code> were deprecated and replaced with a new overload that does not allow to specify the number of segments any longer.
|
||||
For more details, see <a href="https://cwiki.apache.org/confluence/x/mQU0BQ">KIP-319</a>
|
||||
(note: <a href="https://cwiki.apache.org/confluence/x/sQU0BQ">KIP-328</a> and
|
||||
<a href="https://cwiki.apache.org/confluence/x/IBNPBQ">KIP-358</a> 'overlap' with KIP-319).
|
||||
</p>
|
||||
|
||||
<p>
|
||||
We've added an overloaded <code>StreamsBuilder#build</code> method that accepts an instance of <code>java.util.Properties</code> with the intent of using the
|
||||
<code>StreamsConfig#TOPOLOGY_OPTIMIZATION_CONFIG</code> config added in Kafka Streams 2.0. Before 2.1, when building a topology with
|
||||
the DSL, Kafka Streams writes the physical plan as the user makes calls on the DSL. Now by providing a <code>java.util.Properties</code> instance when
|
||||
executing a <code>StreamsBuilder#build</code> call, Kafka Streams can optimize the physical plan of the topology, provided the <code>StreamsConfig#TOPOLOGY_OPTIMIZATION_CONFIG</code>
|
||||
config is set to <code>StreamsConfig#OPTIMIZE</code>. By setting <code>StreamsConfig#OPTIMIZE</code> in addition to the <code>KTable</code> optimization of
|
||||
reusing the source topic as the changelog topic, the topology may be optimized to merge redundant repartition topics into one
|
||||
repartition topic. The original no parameter version of <code>StreamsBuilder#build</code> is still available for those who wish to not
|
||||
optimize their topology. Note that enabling optimization of the topology may require you to do an application reset when redeploying the application. For more
|
||||
details, see <a href="https://cwiki.apache.org/confluence/x/CkcYBQ">KIP-312</a>
|
||||
</p>
|
||||
|
||||
<p>
|
||||
We are introducing static membership towards Kafka Streams user. This feature reduces unnecessary rebalances during normal application upgrades or rolling bounces.
|
||||
For more details on how to use it, checkout <a href="/{{version}}/documentation/#static_membership">static membership design</a>.
|
||||
Note, Kafka Streams uses the same <code>ConsumerConfig#GROUP_INSTANCE_ID_CONFIG</code>, and you only need to make sure it is uniquely defined across
|
||||
different stream instances in one application.
|
||||
</p>
|
||||
|
||||
<h3 class="anchor-heading"><a id="streams_api_changes_200" class="anchor-link"></a><a href="#streams_api_changes_200">Streams API changes in 2.0.0</a></h3>
|
||||
<p>
|
||||
In 2.0.0 we have added a few new APIs on the <code>ReadOnlyWindowStore</code> interface (for details please read <a href="#streams_api_changes_200">Streams API changes</a> below).
|
||||
If you have customized window store implementations that extends the <code>ReadOnlyWindowStore</code> interface you need to make code changes.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
In addition, if you using Java 8 method references in your Kafka Streams code you might need to update your code to resolve method ambiguities.
|
||||
Hot-swapping the jar-file only might not work for this case.
|
||||
See below a complete list of <a href="#streams_api_changes_200">2.0.0</a>
|
||||
API and semantic changes that allow you to advance your application and/or simplify your code base.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
We moved <code>Consumed</code> interface from <code>org.apache.kafka.streams</code> to <code>org.apache.kafka.streams.kstream</code>
|
||||
as it was mistakenly placed in the previous release. If your code has already used it there is a simple one-liner change needed in your import statement.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
We have also removed some public APIs that are deprecated prior to 1.0.x in 2.0.0.
|
||||
See below for a detailed list of removed APIs.
|
||||
</p>
|
||||
<p>
|
||||
We have removed the <code>skippedDueToDeserializationError-rate</code> and <code>skippedDueToDeserializationError-total</code> metrics.
|
||||
Deserialization errors, and all other causes of record skipping, are now accounted for in the pre-existing metrics
|
||||
<code>skipped-records-rate</code> and <code>skipped-records-total</code>. When a record is skipped, the event is
|
||||
now logged at WARN level. If these warnings become burdensome, we recommend explicitly filtering out unprocessable
|
||||
records instead of depending on record skipping semantics. For more details, see
|
||||
<a href="https://cwiki.apache.org/confluence/x/gFOHB">KIP-274</a>.
|
||||
As of right now, the potential causes of skipped records are:
|
||||
</p>
|
||||
<ul>
|
||||
<li><code>null</code> keys in table sources</li>
|
||||
<li><code>null</code> keys in table-table inner/left/outer/right joins</li>
|
||||
<li><code>null</code> keys or values in stream-table joins</li>
|
||||
<li><code>null</code> keys or values in stream-stream joins</li>
|
||||
<li><code>null</code> keys or values in aggregations on grouped streams</li>
|
||||
<li><code>null</code> keys or values in reductions on grouped streams</li>
|
||||
<li><code>null</code> keys in aggregations on windowed streams</li>
|
||||
<li><code>null</code> keys in reductions on windowed streams</li>
|
||||
<li><code>null</code> keys in aggregations on session-windowed streams</li>
|
||||
<li>
|
||||
Errors producing results, when the configured <code>default.production.exception.handler</code> decides to
|
||||
<code>CONTINUE</code> (the default is to <code>FAIL</code> and throw an exception).
|
||||
</li>
|
||||
<li>
|
||||
Errors deserializing records, when the configured <code>default.deserialization.exception.handler</code>
|
||||
decides to <code>CONTINUE</code> (the default is to <code>FAIL</code> and throw an exception).
|
||||
This was the case previously captured in the <code>skippedDueToDeserializationError</code> metrics.
|
||||
</li>
|
||||
<li>Fetched records having a negative timestamp.</li>
|
||||
</ul>
|
||||
|
||||
<p>
|
||||
We've also fixed the metrics name for time and session windowed store operations in 2.0. As a result, our current built-in stores
|
||||
will have their store types in the metric names as <code>in-memory-state</code>, <code>in-memory-lru-state</code>,
|
||||
<code>rocksdb-state</code>, <code>rocksdb-window-state</code>, and <code>rocksdb-session-state</code>. For example, a RocksDB time windowed store's
|
||||
put operation metrics would now be
|
||||
<code>kafka.streams:type=stream-rocksdb-window-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),rocksdb-window-state-id=([-.\w]+)</code>.
|
||||
Users need to update their metrics collecting and reporting systems for their time and session windowed stores accordingly.
|
||||
For more details, please read the <a href="/{{version}}/documentation/#kafka_streams_store_monitoring">State Store Metrics</a> section.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
We have added support for methods in <code>ReadOnlyWindowStore</code> which allows for querying a single window's key-value pair.
|
||||
For users who have customized window store implementations on the above interface, they'd need to update their code to implement the newly added method as well.
|
||||
For more details, see <a href="https://cwiki.apache.org/confluence/x/UUSHB">KIP-261</a>.
|
||||
</p>
|
||||
<p>
|
||||
We have added public <code>WindowedSerdes</code> to allow users to read from / write to a topic storing windowed table changelogs directly.
|
||||
In addition, in <code>StreamsConfig</code> we have also added <code>default.windowed.key.serde.inner</code> and <code>default.windowed.value.serde.inner</code>
|
||||
to let users specify inner serdes if the default serde classes are windowed serdes.
|
||||
For more details, see <a href="https://cwiki.apache.org/confluence/x/_keHB">KIP-265</a>.
|
||||
</p>
|
||||
<p>
|
||||
We've added message header support in the <code>Processor API</code> in Kafka 2.0.0. In particular, we have added a new API <code>ProcessorContext#headers()</code>
|
||||
which returns a <code>Headers</code> object that keeps track of the headers of the source topic's message that is being processed. Through this object, users can manipulate
|
||||
the headers map that is being propagated throughout the processor topology as well. For more details please feel free to read
|
||||
the <a href="/{{version}}/documentation/streams/developer-guide/processor-api.html#accessing-processor-context">Developer Guide</a> section.
|
||||
</p>
|
||||
<p>
|
||||
We have deprecated constructors of <code>KafkaStreams</code> that take a <code>StreamsConfig</code> as parameter.
|
||||
Please use the other corresponding constructors that accept <code>java.util.Properties</code> instead.
|
||||
For more details, see <a href="https://cwiki.apache.org/confluence/x/KLRzB">KIP-245</a>.
|
||||
</p>
|
||||
<p>
|
||||
Kafka 2.0.0 allows to manipulate timestamps of output records using the Processor API (<a href="https://cwiki.apache.org/confluence/x/Ih6HB">KIP-251</a>).
|
||||
To enable this new feature, <code>ProcessorContext#forward(...)</code> was modified.
|
||||
The two existing overloads <code>#forward(Object key, Object value, String childName)</code> and <code>#forward(Object key, Object value, int childIndex)</code> were deprecated and a new overload <code>#forward(Object key, Object value, To to)</code> was added.
|
||||
The new class <code>To</code> allows you to send records to all or specific downstream processors by name and to set the timestamp for the output record.
|
||||
Forwarding based on child index is not supported in the new API any longer.
|
||||
</p>
|
||||
<p>
|
||||
We have added support to allow routing records dynamically to Kafka topics. More specifically, in both the lower-level <code>Topology#addSink</code> and higher-level <code>KStream#to</code> APIs, we have added variants that
|
||||
take a <code>TopicNameExtractor</code> instance instead of a specific <code>String</code> typed topic name, such that for each received record from the upstream processor, the library will dynamically determine which Kafka topic to write to
|
||||
based on the record's key and value, as well as record context. Note that all the Kafka topics that may possibly be used are still considered as user topics and hence required to be pre-created. In addition to that, we have modified the
|
||||
<code>StreamPartitioner</code> interface to add the topic name parameter since the topic name now may not be known beforehand; users who have customized implementations of this interface would need to update their code while upgrading their application
|
||||
to use Kafka Streams 2.0.0.
|
||||
</p>
|
||||
<p>
|
||||
<a href="https://cwiki.apache.org/confluence/x/DVyHB">KIP-284</a> changed the retention time for repartition topics by setting its default value to <code>Long.MAX_VALUE</code>.
|
||||
Instead of relying on data retention Kafka Streams uses the new purge data API to delete consumed data from those topics and to keep used storage small now.
|
||||
</p>
|
||||
<p>
|
||||
We have modified the <code>ProcessorStateManger#register(...)</code> signature and removed the deprecated <code>loggingEnabled</code> boolean parameter as it is specified in the <code>StoreBuilder</code>.
|
||||
Users who used this function to register their state stores into the processor topology need to simply update their code and remove this parameter from the caller.
|
||||
</p>
|
||||
<p>
|
||||
Kafka Streams DSL for Scala is a new Kafka Streams client library available for developers authoring Kafka Streams applications in Scala. It wraps core Kafka Streams DSL types to make it easier to call when
|
||||
interoperating with Scala code. For example, it includes higher order functions as parameters for transformations avoiding the need anonymous classes in Java 7 or experimental SAM type conversions in Scala 2.11,
|
||||
automatic conversion between Java and Scala collection types, a way
|
||||
to implicitly provide Serdes to reduce boilerplate from your application and make it more typesafe, and more! For more information see the
|
||||
<a href="/{{version}}/documentation/streams/developer-guide/dsl-api.html#scala-dsl">Kafka Streams DSL for Scala documentation</a> and
|
||||
<a href="https://cwiki.apache.org/confluence/x/c06HB">KIP-270</a>.
|
||||
</p>
|
||||
<p>
|
||||
We have removed these deprecated APIs:
|
||||
</p>
|
||||
<ul>
|
||||
<li><code>KafkaStreams#toString</code> no longer returns the topology and runtime metadata; to get topology metadata users can call <code>Topology#describe()</code> and to get thread runtime metadata users can call <code>KafkaStreams#localThreadsMetadata</code> (they are deprecated since 1.0.0).
|
||||
For detailed guidance on how to update your code please read <a href="#streams_api_changes_100">here</a></li>
|
||||
<li><code>TopologyBuilder</code> and <code>KStreamBuilder</code> are removed and replaced by <code>Topology</code> and <code>StreamsBuidler</code> respectively (they are deprecated since 1.0.0).
|
||||
For detailed guidance on how to update your code please read <a href="#streams_api_changes_100">here</a></li>
|
||||
<li><code>StateStoreSupplier</code> are removed and replaced with <code>StoreBuilder</code> (they are deprecated since 1.0.0);
|
||||
and the corresponding <code>Stores#create</code> and <code>KStream, KTable, KGroupedStream</code> overloaded functions that use it have also been removed.
|
||||
For detailed guidance on how to update your code please read <a href="#streams_api_changes_100">here</a></li>
|
||||
<li><code>KStream, KTable, KGroupedStream</code> overloaded functions that requires serde and other specifications explicitly are removed and replaced with simpler overloaded functions that use <code>Consumed, Produced, Serialized, Materialized, Joined</code> (they are deprecated since 1.0.0).
|
||||
For detailed guidance on how to update your code please read <a href="#streams_api_changes_100">here</a></li>
|
||||
<li><code>Processor#punctuate</code>, <code>ValueTransformer#punctuate</code>, <code>ValueTransformer#punctuate</code> and <code>ProcessorContext#schedule(long)</code> are removed and replaced by <code>ProcessorContext#schedule(long, PunctuationType, Punctuator)</code> (they are deprecated in 1.0.0). </li>
|
||||
<li>The second <code>boolean</code> typed parameter "loggingEnabled" in <code>ProcessorContext#register</code> has been removed; users can now use <code>StoreBuilder#withLoggingEnabled, withLoggingDisabled</code> to specify the behavior when they create the state store. </li>
|
||||
<li><code>KTable#writeAs, print, foreach, to, through</code> are removed, users can call <code>KTable#tostream()#writeAs</code> instead for the same purpose (they are deprecated since 0.11.0.0).
|
||||
For detailed list of removed APIs please read <a href="#streams_api_changes_0110">here</a></li>
|
||||
<li><code>StreamsConfig#KEY_SERDE_CLASS_CONFIG, VALUE_SERDE_CLASS_CONFIG, TIMESTAMP_EXTRACTOR_CLASS_CONFIG</code> are removed and replaced with <code>StreamsConfig#DEFAULT_KEY_SERDE_CLASS_CONFIG, DEFAULT_VALUE_SERDE_CLASS_CONFIG, DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG</code> respectively (they are deprecated since 0.11.0.0). </li>
|
||||
<li><code>StreamsConfig#ZOOKEEPER_CONNECT_CONFIG</code> are removed as we do not need ZooKeeper dependency in Streams any more (it is deprecated since 0.10.2.0). </li>
|
||||
</ul>
|
||||
|
||||
<h3 class="anchor-heading"><a id="streams_api_changes_110" class="anchor-link"></a><a href="#streams_api_changes_110">Streams API changes in 1.1.0</a></h3>
|
||||
<p>
|
||||
We have added support for methods in <code>ReadOnlyWindowStore</code> which allows for querying <code>WindowStore</code>s without the necessity of providing keys.
|
||||
For users who have customized window store implementations on the above interface, they'd need to update their code to implement the newly added method as well.
|
||||
For more details, see <a href="https://cwiki.apache.org/confluence/x/6qdjB">KIP-205</a>.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
There is a new artifact <code>kafka-streams-test-utils</code> providing a <code>TopologyTestDriver</code>, <code>ConsumerRecordFactory</code>, and <code>OutputVerifier</code> class.
|
||||
You can include the new artifact as a regular dependency to your unit tests and use the test driver to test your business logic of your Kafka Streams application.
|
||||
For more details, see <a href="https://cwiki.apache.org/confluence/x/EQOHB">KIP-247</a>.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
The introduction of <a href="https://cwiki.apache.org/confluence/x/QJ5zB">KIP-220</a>
|
||||
enables you to provide configuration parameters for the embedded admin client created by Kafka Streams, similar to the embedded producer and consumer clients.
|
||||
You can provide the configs via <code>StreamsConfig</code> by adding the configs with the prefix <code>admin.</code> as defined by <code>StreamsConfig#adminClientPrefix(String)</code>
|
||||
to distinguish them from configurations of other clients that share the same config names.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
New method in <code>KTable</code>
|
||||
</p>
|
||||
<ul>
|
||||
<li> <code>transformValues</code> methods have been added to <code>KTable</code>. Similar to those on <code>KStream</code>, these methods allow for richer, stateful, value transformation similar to the Processor API.</li>
|
||||
</ul>
|
||||
|
||||
<p>
|
||||
New method in <code>GlobalKTable</code>
|
||||
</p>
|
||||
<ul>
|
||||
<li> A method has been provided such that it will return the store name associated with the <code>GlobalKTable</code> or <code>null</code> if the store name is non-queryable. </li>
|
||||
</ul>
|
||||
|
||||
<p>
|
||||
New methods in <code>KafkaStreams</code>:
|
||||
</p>
|
||||
<ul>
|
||||
<li> added overload for the constructor that allows overriding the <code>Time</code> object used for tracking system wall-clock time; this is useful for unit testing your application code. </li>
|
||||
</ul>
|
||||
|
||||
<p> New methods in <code>KafkaClientSupplier</code>: </p>
|
||||
<ul>
|
||||
<li> added <code>getAdminClient(config)</code> that allows to override an <code>AdminClient</code> used for administrative requests such as internal topic creations, etc. </li>
|
||||
</ul>
|
||||
|
||||
<p>New error handling for exceptions during production:</p>
|
||||
<ul>
|
||||
<li>added interface <code>ProductionExceptionHandler</code> that allows implementors to decide whether or not Streams should <code>FAIL</code> or <code>CONTINUE</code> when certain exception occur while trying to produce.</li>
|
||||
<li>provided an implementation, <code>DefaultProductionExceptionHandler</code> that always fails, preserving the existing behavior by default.</li>
|
||||
<li>changing which implementation is used can be done by settings <code>default.production.exception.handler</code> to the fully qualified name of a class implementing this interface.</li>
|
||||
</ul>
|
||||
|
||||
<p> Changes in <code>StreamsResetter</code>: </p>
|
||||
<ul>
|
||||
<li> added options to specify input topics offsets to reset according to <a href="https://cwiki.apache.org/confluence/x/ApI7B">KIP-171</a></li>
|
||||
</ul>
|
||||
|
||||
<h3 class="anchor-heading"><a id="streams_api_changes_100" class="anchor-link"></a><a href="#streams_api_changes_100">Streams API changes in 1.0.0</a></h3>
|
||||
|
||||
<p>
|
||||
With 1.0 a major API refactoring was accomplished and the new API is cleaner and easier to use.
|
||||
This change includes the five main classes <code>KafkaStreams</code>, <code>KStreamBuilder</code>,
|
||||
<code>KStream</code>, <code>KTable</code>, and <code>TopologyBuilder</code> (and some more others).
|
||||
All changes are fully backward compatible as old API is only deprecated but not removed.
|
||||
We recommend to move to the new API as soon as you can.
|
||||
We will summarize all API changes in the next paragraphs.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
The two main classes to specify a topology via the DSL (<code>KStreamBuilder</code>)
|
||||
or the Processor API (<code>TopologyBuilder</code>) were deprecated and replaced by
|
||||
<code>StreamsBuilder</code> and <code>Topology</code> (both new classes are located in
|
||||
package <code>org.apache.kafka.streams</code>).
|
||||
Note, that <code>StreamsBuilder</code> does not extend <code>Topology</code>, i.e.,
|
||||
the class hierarchy is different now.
|
||||
The new classes have basically the same methods as the old ones to build a topology via DSL or Processor API.
|
||||
However, some internal methods that were public in <code>KStreamBuilder</code>
|
||||
and <code>TopologyBuilder</code> but not part of the actual API are not present
|
||||
in the new classes any longer.
|
||||
Furthermore, some overloads were simplified compared to the original classes.
|
||||
See <a href="https://cwiki.apache.org/confluence/x/uR8IB">KIP-120</a>
|
||||
and <a href="https://cwiki.apache.org/confluence/x/TYZjB">KIP-182</a>
|
||||
for full details.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
Changing how a topology is specified also affects <code>KafkaStreams</code> constructors,
|
||||
that now only accept a <code>Topology</code>.
|
||||
Using the DSL builder class <code>StreamsBuilder</code> one can get the constructed
|
||||
<code>Topology</code> via <code>StreamsBuilder#build()</code>.
|
||||
Additionally, a new class <code>org.apache.kafka.streams.TopologyDescription</code>
|
||||
(and some more dependent classes) were added.
|
||||
Those can be used to get a detailed description of the specified topology
|
||||
and can be obtained by calling <code>Topology#describe()</code>.
|
||||
An example using this new API is shown in the <a href="/{{version}}/documentation/streams/quickstart">quickstart section</a>.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
New methods in <code>KStream</code>:
|
||||
</p>
|
||||
<ul>
|
||||
<li>With the introduction of <a href="https://cwiki.apache.org/confluence/x/66JjB">KIP-202</a>
|
||||
a new method <code>merge()</code> has been created in <code>KStream</code> as the StreamsBuilder class's <code>StreamsBuilder#merge()</code> has been removed.
|
||||
The method signature was also changed, too: instead of providing multiple <code>KStream</code>s into the method at the once, only a single <code>KStream</code> is accepted.
|
||||
</li>
|
||||
</ul>
|
||||
|
||||
<p>
|
||||
New methods in <code>KafkaStreams</code>:
|
||||
</p>
|
||||
<ul>
|
||||
<li>retrieve the current runtime information about the local threads via <code>localThreadsMetadata()</code> </li>
|
||||
<li>observe the restoration of all state stores via <code>setGlobalStateRestoreListener()</code>, in which users can provide their customized implementation of the <code>org.apache.kafka.streams.processor.StateRestoreListener</code> interface</li>
|
||||
</ul>
|
||||
|
||||
<p>
|
||||
Deprecated / modified methods in <code>KafkaStreams</code>:
|
||||
</p>
|
||||
<ul>
|
||||
<li>
|
||||
<code>toString()</code>, <code>toString(final String indent)</code> were previously used to return static and runtime information.
|
||||
They have been deprecated in favor of using the new classes/methods <code>localThreadsMetadata()</code> / <code>ThreadMetadata</code> (returning runtime information) and
|
||||
<code>TopologyDescription</code> / <code>Topology#describe()</code> (returning static information).
|
||||
</li>
|
||||
<li>
|
||||
With the introduction of <a href="https://cwiki.apache.org/confluence/x/TYZjB">KIP-182</a>
|
||||
you should no longer pass in <code>Serde</code> to <code>KStream#print</code> operations.
|
||||
If you can't rely on using <code>toString</code> to print your keys an values, you should instead you provide a custom <code>KeyValueMapper</code> via the <code>Printed#withKeyValueMapper</code> call.
|
||||
</li>
|
||||
<li>
|
||||
<code>setStateListener()</code> now can only be set before the application start running, i.e. before <code>KafkaStreams.start()</code> is called.
|
||||
</li>
|
||||
</ul>
|
||||
|
||||
<p>
|
||||
Deprecated methods in <code>KGroupedStream</code>
|
||||
</p>
|
||||
<ul>
|
||||
<li>
|
||||
Windowed aggregations have been deprecated from <code>KGroupedStream</code> and moved to <code>WindowedKStream</code>.
|
||||
You can now perform a windowed aggregation by, for example, using <code>KGroupedStream#windowedBy(Windows)#reduce(Reducer)</code>.
|
||||
</li>
|
||||
</ul>
|
||||
|
||||
<p>
|
||||
Modified methods in <code>Processor</code>:
|
||||
</p>
|
||||
<ul>
|
||||
<li>
|
||||
<p>
|
||||
The Processor API was extended to allow users to schedule <code>punctuate</code> functions either based on data-driven <b>stream time</b> or wall-clock time.
|
||||
As a result, the original <code>ProcessorContext#schedule</code> is deprecated with a new overloaded function that accepts a user customizable <code>Punctuator</code> callback interface, which triggers its <code>punctuate</code> API method periodically based on the <code>PunctuationType</code>.
|
||||
The <code>PunctuationType</code> determines what notion of time is used for the punctuation scheduling: either <a href="/{{version}}/documentation/streams/core-concepts#streams_time">stream time</a> or wall-clock time (by default, <b>stream time</b> is configured to represent event time via <code>TimestampExtractor</code>).
|
||||
In addition, the <code>punctuate</code> function inside <code>Processor</code> is also deprecated.
|
||||
</p>
|
||||
<p>
|
||||
Before this, users could only schedule based on stream time (i.e. <code>PunctuationType.STREAM_TIME</code>) and hence the <code>punctuate</code> function was data-driven only because stream time is determined (and advanced forward) by the timestamps derived from the input data.
|
||||
If there is no data arriving at the processor, the stream time would not advance and hence punctuation will not be triggered.
|
||||
On the other hand, When wall-clock time (i.e. <code>PunctuationType.WALL_CLOCK_TIME</code>) is used, <code>punctuate</code> will be triggered purely based on wall-clock time.
|
||||
So for example if the <code>Punctuator</code> function is scheduled based on <code>PunctuationType.WALL_CLOCK_TIME</code>, if these 60 records were processed within 20 seconds,
|
||||
<code>punctuate</code> would be called 2 times (one time every 10 seconds);
|
||||
if these 60 records were processed within 5 seconds, then no <code>punctuate</code> would be called at all.
|
||||
Users can schedule multiple <code>Punctuator</code> callbacks with different <code>PunctuationType</code>s within the same processor by simply calling <code>ProcessorContext#schedule</code> multiple times inside processor's <code>init()</code> method.
|
||||
</p>
|
||||
</li>
|
||||
</ul>
|
||||
|
||||
<p>
|
||||
If you are monitoring on task level or processor-node / state store level Streams metrics, please note that the metrics sensor name and hierarchy was changed:
|
||||
The task ids, store names and processor names are no longer in the sensor metrics names, but instead are added as tags of the sensors to achieve consistent metrics hierarchy.
|
||||
As a result you may need to make corresponding code changes on your metrics reporting and monitoring tools when upgrading to 1.0.0.
|
||||
Detailed metrics sensor can be found in the <a href="/{{version}}/documentation/#kafka_streams_monitoring">Streams Monitoring</a> section.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
The introduction of <a href="https://cwiki.apache.org/confluence/x/WQgwB">KIP-161</a>
|
||||
enables you to provide a default exception handler for deserialization errors when reading data from Kafka rather than throwing the exception all the way out of your streams application.
|
||||
You can provide the configs via the <code>StreamsConfig</code> as <code>StreamsConfig#DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG</code>.
|
||||
The specified handler must implement the <code>org.apache.kafka.streams.errors.DeserializationExceptionHandler</code> interface.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
The introduction of <a href="https://cwiki.apache.org/confluence/x/aZM7B">KIP-173</a>
|
||||
enables you to provide topic configuration parameters for any topics created by Kafka Streams.
|
||||
This includes repartition and changelog topics.
|
||||
You can provide the configs via the <code>StreamsConfig</code> by adding the configs with the prefix as defined by <code>StreamsConfig#topicPrefix(String)</code>.
|
||||
Any properties in the <code>StreamsConfig</code> with the prefix will be applied when creating internal topics.
|
||||
Any configs that aren't topic configs will be ignored.
|
||||
If you already use <code>StateStoreSupplier</code> or <code>Materialized</code> to provide configs for changelogs, then they will take precedence over those supplied in the config.
|
||||
</p>
|
||||
|
||||
<h3 class="anchor-heading"><a id="streams_api_changes_0110" class="anchor-link"></a><a href="#streams_api_changes_0110">Streams API changes in 0.11.0.0</a></h3>
|
||||
|
||||
<p> Updates in <code>StreamsConfig</code>: </p>
|
||||
<ul>
|
||||
<li> new configuration parameter <code>processing.guarantee</code> is added </li>
|
||||
<li> configuration parameter <code>key.serde</code> was deprecated and replaced by <code>default.key.serde</code> </li>
|
||||
<li> configuration parameter <code>value.serde</code> was deprecated and replaced by <code>default.value.serde</code> </li>
|
||||
<li> configuration parameter <code>timestamp.extractor</code> was deprecated and replaced by <code>default.timestamp.extractor</code> </li>
|
||||
<li> method <code>keySerde()</code> was deprecated and replaced by <code>defaultKeySerde()</code> </li>
|
||||
<li> method <code>valueSerde()</code> was deprecated and replaced by <code>defaultValueSerde()</code> </li>
|
||||
<li> new method <code>defaultTimestampExtractor()</code> was added </li>
|
||||
</ul>
|
||||
|
||||
<p> New methods in <code>TopologyBuilder</code>: </p>
|
||||
<ul>
|
||||
<li> added overloads for <code>addSource()</code> that allow to define a <code>TimestampExtractor</code> per source node </li>
|
||||
<li> added overloads for <code>addGlobalStore()</code> that allow to define a <code>TimestampExtractor</code> per source node associated with the global store </li>
|
||||
</ul>
|
||||
|
||||
<p> New methods in <code>KStreamBuilder</code>: </p>
|
||||
<ul>
|
||||
<li> added overloads for <code>stream()</code> that allow to define a <code>TimestampExtractor</code> per input stream </li>
|
||||
<li> added overloads for <code>table()</code> that allow to define a <code>TimestampExtractor</code> per input table </li>
|
||||
<li> added overloads for <code>globalKTable()</code> that allow to define a <code>TimestampExtractor</code> per global table </li>
|
||||
</ul>
|
||||
|
||||
<p> Deprecated methods in <code>KTable</code>: </p>
|
||||
<ul>
|
||||
<li> <code>void foreach(final ForeachAction<? super K, ? super V> action)</code> </li>
|
||||
<li> <code>void print()</code> </li>
|
||||
<li> <code>void print(final String streamName)</code> </li>
|
||||
<li> <code>void print(final Serde<K> keySerde, final Serde<V> valSerde)</code> </li>
|
||||
<li> <code>void print(final Serde<K> keySerde, final Serde<V> valSerde, final String streamName)</code> </li>
|
||||
<li> <code>void writeAsText(final String filePath)</code> </li>
|
||||
<li> <code>void writeAsText(final String filePath, final String streamName)</code> </li>
|
||||
<li> <code>void writeAsText(final String filePath, final Serde<K> keySerde, final Serde<V> valSerde)</code> </li>
|
||||
<li> <code>void writeAsText(final String filePath, final String streamName, final Serde<K> keySerde, final Serde<V> valSerde)</code> </li>
|
||||
</ul>
|
||||
|
||||
<p>
|
||||
The above methods have been deprecated in favor of using the Interactive Queries API.
|
||||
If you want to query the current content of the state store backing the KTable, use the following approach:
|
||||
</p>
|
||||
<ul>
|
||||
<li> Make a call to <code>KafkaStreams.store(final String storeName, final QueryableStoreType<T> queryableStoreType)</code> </li>
|
||||
<li> Then make a call to <code>ReadOnlyKeyValueStore.all()</code> to iterate over the keys of a <code>KTable</code>. </li>
|
||||
</ul>
|
||||
<p>
|
||||
If you want to view the changelog stream of the <code>KTable</code> then you could call <code>KTable.toStream().print(Printed.toSysOut)</code>.
|
||||
</p>
|
||||
|
||||
<p> Metrics using exactly-once semantics: </p>
|
||||
<p>
|
||||
If <code>"exactly_once"</code> processing (EOS version 1) is enabled via the <code>processing.guarantee</code> parameter,
|
||||
internally Streams switches from a producer-per-thread to a producer-per-task runtime model.
|
||||
Using <code>"exactly_once_beta"</code> (EOS version 2) does use a producer-per-thread, so <code>client.id</code> doesn't change,
|
||||
compared with <code>"at_least_once"</code> for this case).
|
||||
In order to distinguish the different producers, the producer's <code>client.id</code> additionally encodes the task-ID for this case.
|
||||
Because the producer's <code>client.id</code> is used to report JMX metrics, it might be required to update tools that receive those metrics.
|
||||
</p>
|
||||
|
||||
<p> Producer's <code>client.id</code> naming schema: </p>
|
||||
<ul>
|
||||
<li> at-least-once (default): <code>[client.Id]-StreamThread-[sequence-number]</code> </li>
|
||||
<li> exactly-once: <code>[client.Id]-StreamThread-[sequence-number]-[taskId]</code> </li>
|
||||
<li> exactly-once-beta: <code>[client.Id]-StreamThread-[sequence-number]</code> </li>
|
||||
</ul>
|
||||
<p> <code>[client.Id]</code> is either set via Streams configuration parameter <code>client.id</code> or defaults to <code>[application.id]-[processId]</code> (<code>[processId]</code> is a random UUID). </p>
|
||||
|
||||
<h3 class="anchor-heading"><a id="streams_api_changes_01021" class="anchor-link"></a><a href="#streams_api_changes_01021">Notable changes in 0.10.2.1</a></h3>
|
||||
|
||||
<p>
|
||||
Parameter updates in <code>StreamsConfig</code>:
|
||||
</p>
|
||||
<ul>
|
||||
<li> The default config values of embedded producer's <code>retries</code> and consumer's <code>max.poll.interval.ms</code> have been changed to improve the resiliency of a Kafka Streams application </li>
|
||||
</ul>
|
||||
|
||||
<h3 class="anchor-heading"><a id="streams_api_changes_0102" class="anchor-link"></a><a href="#streams_api_changes_0102">Streams API changes in 0.10.2.0</a></h3>
|
||||
|
||||
<p>
|
||||
New methods in <code>KafkaStreams</code>:
|
||||
</p>
|
||||
<ul>
|
||||
<li> set a listener to react on application state change via <code>setStateListener(StateListener listener)</code> </li>
|
||||
<li> retrieve the current application state via <code>state()</code> </li>
|
||||
<li> retrieve the global metrics registry via <code>metrics()</code> </li>
|
||||
<li> apply a timeout when closing an application via <code>close(long timeout, TimeUnit timeUnit)</code> </li>
|
||||
<li> specify a custom indent when retrieving Kafka Streams information via <code>toString(String indent)</code> </li>
|
||||
</ul>
|
||||
|
||||
<p>
|
||||
Parameter updates in <code>StreamsConfig</code>:
|
||||
</p>
|
||||
<ul>
|
||||
<li> parameter <code>zookeeper.connect</code> was deprecated; a Kafka Streams application does no longer interact with ZooKeeper for topic management but uses the new broker admin protocol
|
||||
(cf. <a href="https://cwiki.apache.org/confluence/x/vBEIAw">KIP-4, Section "Topic Admin Schema"</a>) </li>
|
||||
<li> added many new parameters for metrics, security, and client configurations </li>
|
||||
</ul>
|
||||
|
||||
<p> Changes in <code>StreamsMetrics</code> interface: </p>
|
||||
<ul>
|
||||
<li> removed methods: <code>addLatencySensor()</code> </li>
|
||||
<li> added methods: <code>addLatencyAndThroughputSensor()</code>, <code>addThroughputSensor()</code>, <code>recordThroughput()</code>,
|
||||
<code>addSensor()</code>, <code>removeSensor()</code> </li>
|
||||
</ul>
|
||||
|
||||
<p> New methods in <code>TopologyBuilder</code>: </p>
|
||||
<ul>
|
||||
<li> added overloads for <code>addSource()</code> that allow to define a <code>auto.offset.reset</code> policy per source node </li>
|
||||
<li> added methods <code>addGlobalStore()</code> to add global <code>StateStore</code>s </li>
|
||||
</ul>
|
||||
|
||||
<p> New methods in <code>KStreamBuilder</code>: </p>
|
||||
<ul>
|
||||
<li> added overloads for <code>stream()</code> and <code>table()</code> that allow to define a <code>auto.offset.reset</code> policy per input stream/table </li>
|
||||
<li> added method <code>globalKTable()</code> to create a <code>GlobalKTable</code> </li>
|
||||
</ul>
|
||||
|
||||
<p> New joins for <code>KStream</code>: </p>
|
||||
<ul>
|
||||
<li> added overloads for <code>join()</code> to join with <code>KTable</code> </li>
|
||||
<li> added overloads for <code>join()</code> and <code>leftJoin()</code> to join with <code>GlobalKTable</code> </li>
|
||||
<li> note, join semantics in 0.10.2 were improved and thus you might see different result compared to 0.10.0.x and 0.10.1.x
|
||||
(cf. <a href="https://cwiki.apache.org/confluence/x/EzPtAw">Kafka Streams Join Semantics</a> in the Apache Kafka wiki)
|
||||
</ul>
|
||||
|
||||
<p> Aligned <code>null</code>-key handling for <code>KTable</code> joins: </p>
|
||||
<ul>
|
||||
<li> like all other KTable operations, <code>KTable-KTable</code> joins do not throw an exception on <code>null</code> key records anymore, but drop those records silently </li>
|
||||
</ul>
|
||||
|
||||
<p> New window type <em>Session Windows</em>: </p>
|
||||
<ul>
|
||||
<li> added class <code>SessionWindows</code> to specify session windows </li>
|
||||
<li> added overloads for <code>KGroupedStream</code> methods <code>count()</code>, <code>reduce()</code>, and <code>aggregate()</code>
|
||||
to allow session window aggregations </li>
|
||||
</ul>
|
||||
|
||||
<p> Changes to <code>TimestampExtractor</code>: </p>
|
||||
<ul>
|
||||
<li> method <code>extract()</code> has a second parameter now </li>
|
||||
<li> new default timestamp extractor class <code>FailOnInvalidTimestamp</code>
|
||||
(it gives the same behavior as old (and removed) default extractor <code>ConsumerRecordTimestampExtractor</code>) </li>
|
||||
<li> new alternative timestamp extractor classes <code>LogAndSkipOnInvalidTimestamp</code> and <code>UsePreviousTimeOnInvalidTimestamps</code> </li>
|
||||
</ul>
|
||||
|
||||
<p> Relaxed type constraints of many DSL interfaces, classes, and methods (cf. <a href="https://cwiki.apache.org/confluence/x/dQMIB">KIP-100</a>). </p>
|
||||
|
||||
<h3 class="anchor-heading"><a id="streams_api_changes_0101" class="anchor-link"></a><a href="#streams_api_changes_0101">Streams API changes in 0.10.1.0</a></h3>
|
||||
|
||||
<p> Stream grouping and aggregation split into two methods: </p>
|
||||
<ul>
|
||||
<li> old: KStream #aggregateByKey(), #reduceByKey(), and #countByKey() </li>
|
||||
<li> new: KStream#groupByKey() plus KGroupedStream #aggregate(), #reduce(), and #count() </li>
|
||||
<li> Example: stream.countByKey() changes to stream.groupByKey().count() </li>
|
||||
</ul>
|
||||
|
||||
<p> Auto Repartitioning: </p>
|
||||
<ul>
|
||||
<li> a call to through() after a key-changing operator and before an aggregation/join is no longer required </li>
|
||||
<li> Example: stream.selectKey(...).through(...).countByKey() changes to stream.selectKey().groupByKey().count() </li>
|
||||
</ul>
|
||||
|
||||
<p> TopologyBuilder: </p>
|
||||
<ul>
|
||||
<li> methods #sourceTopics(String applicationId) and #topicGroups(String applicationId) got simplified to #sourceTopics() and #topicGroups() </li>
|
||||
</ul>
|
||||
|
||||
<p> DSL: new parameter to specify state store names: </p>
|
||||
<ul>
|
||||
<li> The new Interactive Queries feature requires to specify a store name for all source KTables and window aggregation result KTables (previous parameter "operator/window name" is now the storeName) </li>
|
||||
<li> KStreamBuilder#table(String topic) changes to #topic(String topic, String storeName) </li>
|
||||
<li> KTable#through(String topic) changes to #through(String topic, String storeName) </li>
|
||||
<li> KGroupedStream #aggregate(), #reduce(), and #count() require additional parameter "String storeName"</li>
|
||||
<li> Example: stream.countByKey(TimeWindows.of("windowName", 1000)) changes to stream.groupByKey().count(TimeWindows.of(1000), "countStoreName") </li>
|
||||
</ul>
|
||||
|
||||
<p> Windowing: </p>
|
||||
<ul>
|
||||
<li> Windows are not named anymore: TimeWindows.of("name", 1000) changes to TimeWindows.of(1000) (cf. DSL: new parameter to specify state store names) </li>
|
||||
<li> JoinWindows has no default size anymore: JoinWindows.of("name").within(1000) changes to JoinWindows.of(1000) </li>
|
||||
</ul>
|
||||
<p>For Streams API changes in version older than 2.4.x, please check <a href="/39/documentation/streams/upgrade-guide">3.9 upgrade document</a>.</p>
|
||||
|
||||
<h3 class="anchor-heading"><a id="streams_api_broker_compat" class="anchor-link"></a><a href="#streams_api_broker_compat">Streams API broker compatibility</a></h3>
|
||||
|
||||
|
@ -1864,7 +1139,7 @@
|
|||
<tbody>
|
||||
<tr>
|
||||
<td>Kafka Streams API (rows)</td>
|
||||
<td>2.1.x and<br>2.2.x and<br>2.3.x and<br>2.4.x and<br>2.5.x and<br>2.6.x and<br>2.7.x and<br>2.8.x and<br>3.0.x and<br>3.1.x and<br>3.2.x and<br>3.3.x and<br>3.4.x and<br>3.5.x and<br>3.6.x and<br>3.7.x and<br>3.8.x and<br>3.9.x and<br>4.0.x</td>
|
||||
<td>2.4.x and<br>2.5.x and<br>2.6.x and<br>2.7.x and<br>2.8.x and<br>3.0.x and<br>3.1.x and<br>3.2.x and<br>3.3.x and<br>3.4.x and<br>3.5.x and<br>3.6.x and<br>3.7.x and<br>3.8.x and<br>3.9.x and<br>4.0.x</td>
|
||||
<td>4.1.x</td>
|
||||
</tr>
|
||||
<tr>
|
||||
|
|
|
@ -36,6 +36,9 @@
|
|||
The <code>KafkaPrincipalBuilder</code> now extends <code>KafkaPrincipalSerde</code>. Force developer to implement <code>KafkaPrincipalSerde</code> interface for custom <code>KafkaPrincipalBuilder</code>.
|
||||
For further details, please refer to <a href="https://cwiki.apache.org/confluence/x/1gq9F">KIP-1157</a>.
|
||||
</li>
|
||||
<li>The behavior of <code>org.apache.kafka.streams.KafkaStreams#removeStreamThread</code> has been changed. The consumer has no longer remove once <code>removeStreamThread</code> finished.
|
||||
Instead, consumer would be kicked off from the group after <code>org.apache.kafka.streams.processor.internals.StreamThread</code> completes its <code>run</code> function.
|
||||
</li>
|
||||
<li>
|
||||
The support for MX4J library, enabled through <code>kafka_mx4jenable</code> system property, was deprecated and will be removed in Kafka 5.0.
|
||||
</li>
|
||||
|
@ -136,9 +139,17 @@
|
|||
settings.
|
||||
</li>
|
||||
<li>
|
||||
The <code>cleanup.policy</code> is empty and <code>remote.storage.enable</code> is set to true, the
|
||||
<code>cleanup.policy</code> now supports empty values, which means infinite retention.
|
||||
This is equivalent to setting <code>retention.ms=-1</code> and <code>retention.bytes=-1</code>
|
||||
<br>
|
||||
If <code>cleanup.policy</code> is empty and <code>remote.storage.enable</code> is set to true, the
|
||||
local log segments will be cleaned based on the values of <code>log.local.retention.bytes</code> and
|
||||
<code>log.local.retention.ms</code>.
|
||||
<br>
|
||||
If <code>cleanup.policy</code> is empty and <code>remote.storage.enable</code> is set to false,
|
||||
local log segments will not be deleted automatically. However, records can still be deleted
|
||||
explicitly through <code>deleteRecords</code> API calls, which will advance the log start offset
|
||||
and remove the corresponding log segments.
|
||||
</li>
|
||||
</ul>
|
||||
</li>
|
||||
|
@ -162,10 +173,35 @@
|
|||
</ul>
|
||||
For further details, please refer to <a href="https://cwiki.apache.org/confluence/x/3gn0Ew">KIP-1120</a>.
|
||||
</li>
|
||||
<li>
|
||||
The metrics <code>org.apache.kafka.server:type=AssignmentsManager.QueuedReplicaToDirAssignments</code>,
|
||||
<code>org.apache.kafka.storage.internals.log:type=RemoteStorageThreadPool.RemoteLogReaderTaskQueueSize</code>, and
|
||||
<code>org.apache.kafka.storage.internals.log:type=RemoteStorageThreadPool.RemoteLogReaderAvgIdlePercent</code>
|
||||
have been deprecated and will be removed in Kafka 5.0.
|
||||
|
||||
As replacements, the following metrics have been introduced, which report the same information:
|
||||
<code>kafka.server:type=AssignmentsManager.QueuedReplicaToDirAssignments</code>,
|
||||
<code>kafka.log.remote:type=RemoteStorageThreadPool.RemoteLogReaderTaskQueueSize</code>, and
|
||||
<code>kafka.log.remote:type=RemoteStorageThreadPool.RemoteLogReaderAvgIdlePercent</code>.
|
||||
For further details, please refer to <a href="https://cwiki.apache.org/confluence/x/6oqMEw">KIP-1100</a>.
|
||||
</li>
|
||||
<li>
|
||||
A new metric <code>AvgIdleRatio</code> has been added to the <code>ControllerEventManager</code> group. This metric measures the average idle ratio of the controller event queue thread,
|
||||
providing visibility into how much time the controller spends waiting for events versus processing them. The metric value ranges from 0.0 (always busy) to 1.0 (always idle).
|
||||
</li>
|
||||
<li>
|
||||
Deprecated <code>org.apache.kafka.streams.KafkaStreams$CloseOptions</code> and its related methods, such as
|
||||
<code>KafkaStreams#close(org.apache.kafka.streams.KafkaStreams$CloseOptions)</code>.
|
||||
As a replacement, please use <code>org.apache.kafka.streams.CloseOptions</code> and
|
||||
<code>KafkaStreams#close(org.apache.kafka.streams.CloseOptions)</code>.
|
||||
For further details, please refer to <a href="https://cwiki.apache.org/confluence/x/QAq9F">KIP-1153</a>.
|
||||
</li>
|
||||
</ul>
|
||||
|
||||
<h4><a id="upgrade_4_1_0" href="#upgrade_4_1_0">Upgrading to 4.1.0</a></h4>
|
||||
|
||||
<p><b>Note:</b> Kafka Streams 4.1.0 contains a critical memory leak bug (<a href="https://issues.apache.org/jira/browse/KAFKA-19748">KAFKA-19748</a>) that affects users of range scans and certain DSL operators (session windows, sliding windows, stream-stream joins, foreign-key joins). Users running Kafka Streams should consider upgrading directly to 4.1.1 when available.</p>
|
||||
|
||||
<h5><a id="upgrade_4_1_0_from" href="#upgrade_4_1_0_from">Upgrading Servers to 4.1.0 from any version 3.3.x through 4.0.x</a></h5>
|
||||
<h5><a id="upgrade_410_notable" href="#upgrade_410_notable">Notable changes in 4.1.0</a></h5>
|
||||
<ul>
|
||||
|
|
|
@ -57,11 +57,10 @@ versions += [
|
|||
caffeine: "3.2.0",
|
||||
bndlib: "7.1.0",
|
||||
checkstyle: project.hasProperty('checkstyleVersion') ? checkstyleVersion : "10.20.2",
|
||||
commonsBeanutils: "1.11.0",
|
||||
commonsLang: "3.18.0",
|
||||
commonsValidator: "1.9.0",
|
||||
commonsValidator: "1.10.0",
|
||||
classgraph: "4.8.179",
|
||||
gradle: "8.14.3",
|
||||
gradle: "9.1.0",
|
||||
grgit: "4.1.1",
|
||||
httpclient: "4.5.14",
|
||||
jackson: "2.19.0",
|
||||
|
@ -126,7 +125,7 @@ versions += [
|
|||
snappy: "1.1.10.7",
|
||||
spotbugs: "4.9.4",
|
||||
mockOAuth2Server: "2.2.1",
|
||||
zinc: "1.9.2",
|
||||
zinc: "1.10.8",
|
||||
// When updating the zstd version, please do as well in docker/native/native-image-configs/resource-config.json
|
||||
// Also make sure the compression levels in org.apache.kafka.common.record.CompressionType are still valid
|
||||
zstd: "1.5.6-10",
|
||||
|
@ -151,7 +150,6 @@ libs += [
|
|||
bndlib:"biz.aQute.bnd:biz.aQute.bndlib:$versions.bndlib",
|
||||
caffeine: "com.github.ben-manes.caffeine:caffeine:$versions.caffeine",
|
||||
classgraph: "io.github.classgraph:classgraph:$versions.classgraph",
|
||||
commonsBeanutils: "commons-beanutils:commons-beanutils:$versions.commonsBeanutils",
|
||||
commonsLang: "org.apache.commons:commons-lang3:$versions.commonsLang",
|
||||
commonsValidator: "commons-validator:commons-validator:$versions.commonsValidator",
|
||||
jacksonAnnotations: "com.fasterxml.jackson.core:jackson-annotations:$versions.jackson",
|
||||
|
|
|
@ -1,7 +1,7 @@
|
|||
distributionBase=GRADLE_USER_HOME
|
||||
distributionPath=wrapper/dists
|
||||
distributionSha256Sum=bd71102213493060956ec229d946beee57158dbd89d0e62b91bca0fa2c5f3531
|
||||
distributionUrl=https\://services.gradle.org/distributions/gradle-8.14.3-bin.zip
|
||||
distributionSha256Sum=a17ddd85a26b6a7f5ddb71ff8b05fc5104c0202c6e64782429790c933686c806
|
||||
distributionUrl=https\://services.gradle.org/distributions/gradle-9.1.0-bin.zip
|
||||
networkTimeout=10000
|
||||
validateDistributionUrl=true
|
||||
zipStoreBase=GRADLE_USER_HOME
|
||||
|
|
|
@ -1,7 +1,7 @@
|
|||
#!/bin/sh
|
||||
|
||||
#
|
||||
# Copyright © 2015-2021 the original authors.
|
||||
# Copyright © 2015 the original authors.
|
||||
#
|
||||
# Licensed under the Apache License, Version 2.0 (the "License");
|
||||
# you may not use this file except in compliance with the License.
|
||||
|
@ -15,6 +15,8 @@
|
|||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
#
|
||||
# SPDX-License-Identifier: Apache-2.0
|
||||
#
|
||||
|
||||
##############################################################################
|
||||
#
|
||||
|
@ -55,7 +57,7 @@
|
|||
# Darwin, MinGW, and NonStop.
|
||||
#
|
||||
# (3) This script is generated from the Groovy template
|
||||
# https://github.com/gradle/gradle/blob/HEAD/subprojects/plugins/src/main/resources/org/gradle/api/internal/plugins/unixStartScript.txt
|
||||
# https://github.com/gradle/gradle/blob/HEAD/platforms/jvm/plugins-application/src/main/resources/org/gradle/api/internal/plugins/unixStartScript.txt
|
||||
# within the Gradle project.
|
||||
#
|
||||
# You can find Gradle at https://github.com/gradle/gradle/.
|
||||
|
@ -84,7 +86,7 @@ done
|
|||
# shellcheck disable=SC2034
|
||||
APP_BASE_NAME=${0##*/}
|
||||
# Discard cd standard output in case $CDPATH is set (https://github.com/gradle/gradle/issues/25036)
|
||||
APP_HOME=$( cd "${APP_HOME:-./}" > /dev/null && pwd -P ) || exit
|
||||
APP_HOME=$( cd -P "${APP_HOME:-./}" > /dev/null && printf '%s\n' "$PWD" ) || exit
|
||||
|
||||
# Use the maximum available, or set MAX_FD != -1 to use that value.
|
||||
MAX_FD=maximum
|
||||
|
@ -113,20 +115,6 @@ case "$( uname )" in #(
|
|||
esac
|
||||
|
||||
|
||||
# Loop in case we encounter an error.
|
||||
for attempt in 1 2 3; do
|
||||
if [ ! -e "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" ]; then
|
||||
if ! curl -s -S --retry 3 -L -o "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" "https://raw.githubusercontent.com/gradle/gradle/v8.14.3/gradle/wrapper/gradle-wrapper.jar"; then
|
||||
rm -f "$APP_HOME/gradle/wrapper/gradle-wrapper.jar"
|
||||
# Pause for a bit before looping in case the server throttled us.
|
||||
sleep 5
|
||||
continue
|
||||
fi
|
||||
fi
|
||||
done
|
||||
|
||||
CLASSPATH=$APP_HOME/gradle/wrapper/gradle-wrapper.jar
|
||||
|
||||
|
||||
# Determine the Java command to use to start the JVM.
|
||||
if [ -n "$JAVA_HOME" ] ; then
|
||||
|
@ -183,7 +171,6 @@ fi
|
|||
# For Cygwin or MSYS, switch paths to Windows format before running java
|
||||
if "$cygwin" || "$msys" ; then
|
||||
APP_HOME=$( cygpath --path --mixed "$APP_HOME" )
|
||||
CLASSPATH=$( cygpath --path --mixed "$CLASSPATH" )
|
||||
|
||||
JAVACMD=$( cygpath --unix "$JAVACMD" )
|
||||
|
||||
|
@ -212,19 +199,31 @@ if "$cygwin" || "$msys" ; then
|
|||
fi
|
||||
|
||||
|
||||
|
||||
# Loop in case we encounter an error.
|
||||
for attempt in 1 2 3; do
|
||||
if [ ! -e "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" ]; then
|
||||
if ! curl -s -S --retry 3 -L -o "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" "https://raw.githubusercontent.com/gradle/gradle/v9.1.0/gradle/wrapper/gradle-wrapper.jar"; then
|
||||
rm -f "$APP_HOME/gradle/wrapper/gradle-wrapper.jar"
|
||||
# Pause for a bit before looping in case the server throttled us.
|
||||
sleep 5
|
||||
continue
|
||||
fi
|
||||
fi
|
||||
done
|
||||
|
||||
# Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script.
|
||||
DEFAULT_JVM_OPTS='"-Xmx64m" "-Xms64m"'
|
||||
|
||||
# Collect all arguments for the java command:
|
||||
# * DEFAULT_JVM_OPTS, JAVA_OPTS, JAVA_OPTS, and optsEnvironmentVar are not allowed to contain shell fragments,
|
||||
# * DEFAULT_JVM_OPTS, JAVA_OPTS, and optsEnvironmentVar are not allowed to contain shell fragments,
|
||||
# and any embedded shellness will be escaped.
|
||||
# * For example: A user cannot expect ${Hostname} to be expanded, as it is an environment variable and will be
|
||||
# treated as '${Hostname}' itself on the command line.
|
||||
|
||||
set -- \
|
||||
"-Dorg.gradle.appname=$APP_BASE_NAME" \
|
||||
-classpath "$CLASSPATH" \
|
||||
org.gradle.wrapper.GradleWrapperMain \
|
||||
-jar "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" \
|
||||
"$@"
|
||||
|
||||
# Stop when "xargs" is not available.
|
||||
|
|
|
@ -2256,18 +2256,13 @@ public class GroupMetadataManager {
|
|||
.setClassicMemberMetadata(null)
|
||||
.build();
|
||||
|
||||
// If the group is newly created, we must ensure that it moves away from
|
||||
// epoch 0 and that it is fully initialized.
|
||||
boolean bumpGroupEpoch = group.groupEpoch() == 0;
|
||||
|
||||
bumpGroupEpoch |= hasMemberSubscriptionChanged(
|
||||
boolean subscribedTopicNamesChanged = hasMemberSubscriptionChanged(
|
||||
groupId,
|
||||
member,
|
||||
updatedMember,
|
||||
records
|
||||
);
|
||||
|
||||
bumpGroupEpoch |= maybeUpdateRegularExpressions(
|
||||
UpdateRegularExpressionsResult updateRegularExpressionsResult = maybeUpdateRegularExpressions(
|
||||
context,
|
||||
group,
|
||||
member,
|
||||
|
@ -2275,9 +2270,24 @@ public class GroupMetadataManager {
|
|||
records
|
||||
);
|
||||
|
||||
// The subscription has changed when either the subscribed topic names or subscribed topic
|
||||
// regex has changed.
|
||||
boolean hasSubscriptionChanged = subscribedTopicNamesChanged || updateRegularExpressionsResult.regexUpdated();
|
||||
int groupEpoch = group.groupEpoch();
|
||||
SubscriptionType subscriptionType = group.subscriptionType();
|
||||
|
||||
boolean bumpGroupEpoch =
|
||||
// If the group is newly created, we must ensure that it moves away from
|
||||
// epoch 0 and that it is fully initialized.
|
||||
groupEpoch == 0 ||
|
||||
// Bumping the group epoch signals that the target assignment should be updated. We bump
|
||||
// the group epoch when the member has changed its subscribed topic names or the member
|
||||
// has changed its subscribed topic regex to a regex that is already resolved. We avoid
|
||||
// bumping the group epoch when the new subscribed topic regex has not been resolved
|
||||
// yet, since we will have to update the target assignment again later.
|
||||
subscribedTopicNamesChanged ||
|
||||
updateRegularExpressionsResult == UpdateRegularExpressionsResult.REGEX_UPDATED_AND_RESOLVED;
|
||||
|
||||
if (bumpGroupEpoch || group.hasMetadataExpired(currentTimeMs)) {
|
||||
// The subscription metadata is updated in two cases:
|
||||
// 1) The member has updated its subscriptions;
|
||||
|
@ -2322,6 +2332,9 @@ public class GroupMetadataManager {
|
|||
group::currentPartitionEpoch,
|
||||
targetAssignmentEpoch,
|
||||
targetAssignment,
|
||||
group.resolvedRegularExpressions(),
|
||||
// Force consistency with the subscription when the subscription has changed.
|
||||
hasSubscriptionChanged,
|
||||
ownedTopicPartitions,
|
||||
records
|
||||
);
|
||||
|
@ -2475,6 +2488,8 @@ public class GroupMetadataManager {
|
|||
group::currentPartitionEpoch,
|
||||
group.assignmentEpoch(),
|
||||
group.targetAssignment(updatedMember.memberId(), updatedMember.instanceId()),
|
||||
group.resolvedRegularExpressions(),
|
||||
bumpGroupEpoch,
|
||||
toTopicPartitions(subscription.ownedPartitions(), metadataImage),
|
||||
records
|
||||
);
|
||||
|
@ -2518,6 +2533,9 @@ public class GroupMetadataManager {
|
|||
group::currentPartitionEpoch,
|
||||
targetAssignmentEpoch,
|
||||
targetAssignment,
|
||||
group.resolvedRegularExpressions(),
|
||||
// Force consistency with the subscription when the subscription has changed.
|
||||
bumpGroupEpoch,
|
||||
toTopicPartitions(subscription.ownedPartitions(), metadataImage),
|
||||
records
|
||||
);
|
||||
|
@ -2676,6 +2694,8 @@ public class GroupMetadataManager {
|
|||
updatedMember,
|
||||
targetAssignmentEpoch,
|
||||
targetAssignment,
|
||||
// Force consistency with the subscription when the subscription has changed.
|
||||
bumpGroupEpoch,
|
||||
records
|
||||
);
|
||||
|
||||
|
@ -3115,6 +3135,16 @@ public class GroupMetadataManager {
|
|||
return value != null && !value.isEmpty();
|
||||
}
|
||||
|
||||
private enum UpdateRegularExpressionsResult {
|
||||
NO_CHANGE,
|
||||
REGEX_UPDATED,
|
||||
REGEX_UPDATED_AND_RESOLVED;
|
||||
|
||||
public boolean regexUpdated() {
|
||||
return this == REGEX_UPDATED || this == REGEX_UPDATED_AND_RESOLVED;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Check whether the member has updated its subscribed topic regular expression and
|
||||
* may trigger the resolution/the refresh of all the regular expressions in the
|
||||
|
@ -3126,9 +3156,9 @@ public class GroupMetadataManager {
|
|||
* @param member The old member.
|
||||
* @param updatedMember The new member.
|
||||
* @param records The records accumulator.
|
||||
* @return Whether a rebalance must be triggered.
|
||||
* @return The result of the update.
|
||||
*/
|
||||
private boolean maybeUpdateRegularExpressions(
|
||||
private UpdateRegularExpressionsResult maybeUpdateRegularExpressions(
|
||||
AuthorizableRequestContext context,
|
||||
ConsumerGroup group,
|
||||
ConsumerGroupMember member,
|
||||
|
@ -3141,14 +3171,17 @@ public class GroupMetadataManager {
|
|||
String oldSubscribedTopicRegex = member.subscribedTopicRegex();
|
||||
String newSubscribedTopicRegex = updatedMember.subscribedTopicRegex();
|
||||
|
||||
boolean bumpGroupEpoch = false;
|
||||
boolean requireRefresh = false;
|
||||
UpdateRegularExpressionsResult updateRegularExpressionsResult = UpdateRegularExpressionsResult.NO_CHANGE;
|
||||
|
||||
// Check whether the member has changed its subscribed regex.
|
||||
if (!Objects.equals(oldSubscribedTopicRegex, newSubscribedTopicRegex)) {
|
||||
boolean subscribedTopicRegexChanged = !Objects.equals(oldSubscribedTopicRegex, newSubscribedTopicRegex);
|
||||
if (subscribedTopicRegexChanged) {
|
||||
log.debug("[GroupId {}] Member {} updated its subscribed regex to: {}.",
|
||||
groupId, memberId, newSubscribedTopicRegex);
|
||||
|
||||
updateRegularExpressionsResult = UpdateRegularExpressionsResult.REGEX_UPDATED;
|
||||
|
||||
if (isNotEmpty(oldSubscribedTopicRegex) && group.numSubscribedMembers(oldSubscribedTopicRegex) == 1) {
|
||||
// If the member was the last one subscribed to the regex, we delete the
|
||||
// resolved regular expression.
|
||||
|
@ -3167,7 +3200,9 @@ public class GroupMetadataManager {
|
|||
} else {
|
||||
// If the new regex is already resolved, we trigger a rebalance
|
||||
// by bumping the group epoch.
|
||||
bumpGroupEpoch = group.resolvedRegularExpression(newSubscribedTopicRegex).isPresent();
|
||||
if (group.resolvedRegularExpression(newSubscribedTopicRegex).isPresent()) {
|
||||
updateRegularExpressionsResult = UpdateRegularExpressionsResult.REGEX_UPDATED_AND_RESOLVED;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -3183,20 +3218,20 @@ public class GroupMetadataManager {
|
|||
// 0. The group is subscribed to regular expressions. We also take the one
|
||||
// that the current may have just introduced.
|
||||
if (!requireRefresh && group.subscribedRegularExpressions().isEmpty()) {
|
||||
return bumpGroupEpoch;
|
||||
return updateRegularExpressionsResult;
|
||||
}
|
||||
|
||||
// 1. There is no ongoing refresh for the group.
|
||||
String key = group.groupId() + "-regex";
|
||||
if (executor.isScheduled(key)) {
|
||||
return bumpGroupEpoch;
|
||||
return updateRegularExpressionsResult;
|
||||
}
|
||||
|
||||
// 2. The last refresh is older than 10s. If the group does not have any regular
|
||||
// expressions but the current member just brought a new one, we should continue.
|
||||
long lastRefreshTimeMs = group.lastResolvedRegularExpressionRefreshTimeMs();
|
||||
if (currentTimeMs <= lastRefreshTimeMs + REGEX_BATCH_REFRESH_MIN_INTERVAL_MS) {
|
||||
return bumpGroupEpoch;
|
||||
return updateRegularExpressionsResult;
|
||||
}
|
||||
|
||||
// 3.1 The group has unresolved regular expressions.
|
||||
|
@ -3225,7 +3260,7 @@ public class GroupMetadataManager {
|
|||
);
|
||||
}
|
||||
|
||||
return bumpGroupEpoch;
|
||||
return updateRegularExpressionsResult;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -3499,16 +3534,18 @@ public class GroupMetadataManager {
|
|||
/**
|
||||
* Reconciles the current assignment of the member towards the target assignment if needed.
|
||||
*
|
||||
* @param groupId The group id.
|
||||
* @param member The member to reconcile.
|
||||
* @param currentPartitionEpoch The function returning the current epoch of
|
||||
* a given partition.
|
||||
* @param targetAssignmentEpoch The target assignment epoch.
|
||||
* @param targetAssignment The target assignment.
|
||||
* @param ownedTopicPartitions The list of partitions owned by the member. This
|
||||
* is reported in the ConsumerGroupHeartbeat API and
|
||||
* it could be null if not provided.
|
||||
* @param records The list to accumulate any new records.
|
||||
* @param groupId The group id.
|
||||
* @param member The member to reconcile.
|
||||
* @param currentPartitionEpoch The function returning the current epoch of
|
||||
* a given partition.
|
||||
* @param targetAssignmentEpoch The target assignment epoch.
|
||||
* @param targetAssignment The target assignment.
|
||||
* @param resolvedRegularExpressions The resolved regular expressions.
|
||||
* @param hasSubscriptionChanged Whether the member has changed its subscription on the current heartbeat.
|
||||
* @param ownedTopicPartitions The list of partitions owned by the member. This
|
||||
* is reported in the ConsumerGroupHeartbeat API and
|
||||
* it could be null if not provided.
|
||||
* @param records The list to accumulate any new records.
|
||||
* @return The received member if no changes have been made; or a new
|
||||
* member containing the new assignment.
|
||||
*/
|
||||
|
@ -3518,15 +3555,20 @@ public class GroupMetadataManager {
|
|||
BiFunction<Uuid, Integer, Integer> currentPartitionEpoch,
|
||||
int targetAssignmentEpoch,
|
||||
Assignment targetAssignment,
|
||||
Map<String, ResolvedRegularExpression> resolvedRegularExpressions,
|
||||
boolean hasSubscriptionChanged,
|
||||
List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions,
|
||||
List<CoordinatorRecord> records
|
||||
) {
|
||||
if (member.isReconciledTo(targetAssignmentEpoch)) {
|
||||
if (!hasSubscriptionChanged && member.isReconciledTo(targetAssignmentEpoch)) {
|
||||
return member;
|
||||
}
|
||||
|
||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||
.withMetadataImage(metadataImage)
|
||||
.withTargetAssignment(targetAssignmentEpoch, targetAssignment)
|
||||
.withHasSubscriptionChanged(hasSubscriptionChanged)
|
||||
.withResolvedRegularExpressions(resolvedRegularExpressions)
|
||||
.withCurrentPartitionEpoch(currentPartitionEpoch)
|
||||
.withOwnedTopicPartitions(ownedTopicPartitions)
|
||||
.build();
|
||||
|
@ -3563,11 +3605,12 @@ public class GroupMetadataManager {
|
|||
/**
|
||||
* Reconciles the current assignment of the member towards the target assignment if needed.
|
||||
*
|
||||
* @param groupId The group id.
|
||||
* @param member The member to reconcile.
|
||||
* @param targetAssignmentEpoch The target assignment epoch.
|
||||
* @param targetAssignment The target assignment.
|
||||
* @param records The list to accumulate any new records.
|
||||
* @param groupId The group id.
|
||||
* @param member The member to reconcile.
|
||||
* @param targetAssignmentEpoch The target assignment epoch.
|
||||
* @param targetAssignment The target assignment.
|
||||
* @param hasSubscriptionChanged Whether the member has changed its subscription on the current heartbeat.
|
||||
* @param records The list to accumulate any new records.
|
||||
* @return The received member if no changes have been made; or a new
|
||||
* member containing the new assignment.
|
||||
*/
|
||||
|
@ -3576,14 +3619,17 @@ public class GroupMetadataManager {
|
|||
ShareGroupMember member,
|
||||
int targetAssignmentEpoch,
|
||||
Assignment targetAssignment,
|
||||
boolean hasSubscriptionChanged,
|
||||
List<CoordinatorRecord> records
|
||||
) {
|
||||
if (member.isReconciledTo(targetAssignmentEpoch)) {
|
||||
if (!hasSubscriptionChanged && member.isReconciledTo(targetAssignmentEpoch)) {
|
||||
return member;
|
||||
}
|
||||
|
||||
ShareGroupMember updatedMember = new ShareGroupAssignmentBuilder(member)
|
||||
.withMetadataImage(metadataImage)
|
||||
.withTargetAssignment(targetAssignmentEpoch, targetAssignment)
|
||||
.withHasSubscriptionChanged(hasSubscriptionChanged)
|
||||
.build();
|
||||
|
||||
if (!updatedMember.equals(member)) {
|
||||
|
|
|
@ -19,8 +19,11 @@ package org.apache.kafka.coordinator.group.modern.consumer;
|
|||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.errors.FencedMemberEpochException;
|
||||
import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
|
||||
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
|
||||
import org.apache.kafka.coordinator.group.modern.Assignment;
|
||||
import org.apache.kafka.coordinator.group.modern.MemberState;
|
||||
import org.apache.kafka.coordinator.group.modern.TopicIds;
|
||||
import org.apache.kafka.coordinator.group.modern.UnionSet;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
|
@ -41,6 +44,11 @@ public class CurrentAssignmentBuilder {
|
|||
*/
|
||||
private final ConsumerGroupMember member;
|
||||
|
||||
/**
|
||||
* The metadata image.
|
||||
*/
|
||||
private CoordinatorMetadataImage metadataImage = CoordinatorMetadataImage.EMPTY;
|
||||
|
||||
/**
|
||||
* The target assignment epoch.
|
||||
*/
|
||||
|
@ -51,6 +59,16 @@ public class CurrentAssignmentBuilder {
|
|||
*/
|
||||
private Assignment targetAssignment;
|
||||
|
||||
/**
|
||||
* Whether the member has changed its subscription on the current heartbeat.
|
||||
*/
|
||||
private boolean hasSubscriptionChanged;
|
||||
|
||||
/**
|
||||
* The resolved regular expressions.
|
||||
*/
|
||||
private Map<String, ResolvedRegularExpression> resolvedRegularExpressions = Map.of();
|
||||
|
||||
/**
|
||||
* A function which returns the current epoch of a topic-partition or -1 if the
|
||||
* topic-partition is not assigned. The current epoch is the epoch of the current owner.
|
||||
|
@ -73,6 +91,19 @@ public class CurrentAssignmentBuilder {
|
|||
this.member = Objects.requireNonNull(member);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the metadata image.
|
||||
*
|
||||
* @param metadataImage The metadata image.
|
||||
* @return This object.
|
||||
*/
|
||||
public CurrentAssignmentBuilder withMetadataImage(
|
||||
CoordinatorMetadataImage metadataImage
|
||||
) {
|
||||
this.metadataImage = metadataImage;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the target assignment epoch and the target assignment that the
|
||||
* consumer group member must be reconciled to.
|
||||
|
@ -90,6 +121,32 @@ public class CurrentAssignmentBuilder {
|
|||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets whether the member has changed its subscription on the current heartbeat.
|
||||
*
|
||||
* @param hasSubscriptionChanged If true, always removes unsubscribed topics from the current assignment.
|
||||
* @return This object.
|
||||
*/
|
||||
public CurrentAssignmentBuilder withHasSubscriptionChanged(
|
||||
boolean hasSubscriptionChanged
|
||||
) {
|
||||
this.hasSubscriptionChanged = hasSubscriptionChanged;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the resolved regular expressions.
|
||||
*
|
||||
* @param resolvedRegularExpressions The resolved regular expressions.
|
||||
* @return This object.
|
||||
*/
|
||||
public CurrentAssignmentBuilder withResolvedRegularExpressions(
|
||||
Map<String, ResolvedRegularExpression> resolvedRegularExpressions
|
||||
) {
|
||||
this.resolvedRegularExpressions = resolvedRegularExpressions;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets a BiFunction which allows to retrieve the current epoch of a
|
||||
* partition. This is used by the state machine to determine if a
|
||||
|
@ -132,12 +189,15 @@ public class CurrentAssignmentBuilder {
|
|||
case STABLE:
|
||||
// When the member is in the STABLE state, we verify if a newer
|
||||
// epoch (or target assignment) is available. If it is, we can
|
||||
// reconcile the member towards it. Otherwise, we return.
|
||||
// reconcile the member towards it. Otherwise, we ensure the
|
||||
// assignment is consistent with the subscribed topics, if changed.
|
||||
if (member.memberEpoch() != targetAssignmentEpoch) {
|
||||
return computeNextAssignment(
|
||||
member.memberEpoch(),
|
||||
member.assignedPartitions()
|
||||
);
|
||||
} else if (hasSubscriptionChanged) {
|
||||
return updateCurrentAssignment(member.assignedPartitions());
|
||||
} else {
|
||||
return member;
|
||||
}
|
||||
|
@ -147,18 +207,27 @@ public class CurrentAssignmentBuilder {
|
|||
// until the member has revoked the necessary partitions. They are
|
||||
// considered revoked when they are not anymore reported in the
|
||||
// owned partitions set in the ConsumerGroupHeartbeat API.
|
||||
// Additional partitions may need revoking when the member's
|
||||
// subscription changes.
|
||||
|
||||
// If the member provides its owned partitions. We verify if it still
|
||||
// owns any of the revoked partitions. If it does, we cannot progress.
|
||||
if (ownsRevokedPartitions(member.partitionsPendingRevocation())) {
|
||||
return member;
|
||||
if (hasSubscriptionChanged) {
|
||||
return updateCurrentAssignment(member.assignedPartitions());
|
||||
} else {
|
||||
return member;
|
||||
}
|
||||
}
|
||||
|
||||
// When the member has revoked all the pending partitions, it can
|
||||
// transition to the next epoch (current + 1) and we can reconcile
|
||||
// its state towards the latest target assignment.
|
||||
return computeNextAssignment(
|
||||
member.memberEpoch() + 1,
|
||||
// When we enter UNREVOKED_PARTITIONS due to a subscription change,
|
||||
// we must not advance the member epoch when the new target
|
||||
// assignment is not available yet.
|
||||
Math.min(member.memberEpoch() + 1, targetAssignmentEpoch),
|
||||
member.assignedPartitions()
|
||||
);
|
||||
|
||||
|
@ -215,6 +284,71 @@ public class CurrentAssignmentBuilder {
|
|||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates the current assignment, removing any partitions that are not part of the subscribed topics.
|
||||
* This method is a lot faster than running the full reconciliation logic in computeNextAssignment.
|
||||
*
|
||||
* @param memberAssignedPartitions The assigned partitions of the member to use.
|
||||
* @return A new ConsumerGroupMember.
|
||||
*/
|
||||
private ConsumerGroupMember updateCurrentAssignment(
|
||||
Map<Uuid, Set<Integer>> memberAssignedPartitions
|
||||
) {
|
||||
Set<Uuid> subscribedTopicIds = subscribedTopicIds();
|
||||
|
||||
// Reuse the original map if no topics need to be removed.
|
||||
Map<Uuid, Set<Integer>> newAssignedPartitions;
|
||||
Map<Uuid, Set<Integer>> newPartitionsPendingRevocation;
|
||||
if (subscribedTopicIds.isEmpty() && member.partitionsPendingRevocation().isEmpty()) {
|
||||
newAssignedPartitions = Map.of();
|
||||
newPartitionsPendingRevocation = memberAssignedPartitions;
|
||||
} else {
|
||||
newAssignedPartitions = memberAssignedPartitions;
|
||||
newPartitionsPendingRevocation = new HashMap<>(member.partitionsPendingRevocation());
|
||||
for (Map.Entry<Uuid, Set<Integer>> entry : memberAssignedPartitions.entrySet()) {
|
||||
if (!subscribedTopicIds.contains(entry.getKey())) {
|
||||
if (newAssignedPartitions == memberAssignedPartitions) {
|
||||
newAssignedPartitions = new HashMap<>(memberAssignedPartitions);
|
||||
newPartitionsPendingRevocation = new HashMap<>(member.partitionsPendingRevocation());
|
||||
}
|
||||
newAssignedPartitions.remove(entry.getKey());
|
||||
newPartitionsPendingRevocation.merge(
|
||||
entry.getKey(),
|
||||
entry.getValue(),
|
||||
(existing, additional) -> {
|
||||
existing = new HashSet<>(existing);
|
||||
existing.addAll(additional);
|
||||
return existing;
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (newAssignedPartitions == memberAssignedPartitions) {
|
||||
// If no partitions were removed, we can return the member as is.
|
||||
return member;
|
||||
}
|
||||
|
||||
if (!newPartitionsPendingRevocation.isEmpty() && ownsRevokedPartitions(newPartitionsPendingRevocation)) {
|
||||
return new ConsumerGroupMember.Builder(member)
|
||||
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||
.setAssignedPartitions(newAssignedPartitions)
|
||||
.setPartitionsPendingRevocation(newPartitionsPendingRevocation)
|
||||
.build();
|
||||
} else {
|
||||
// There were partitions removed, but they were already revoked.
|
||||
// Keep the member in the current state and shrink the assigned partitions.
|
||||
|
||||
// We do not expect to be in the UNREVOKED_PARTITIONS state here. The full
|
||||
// reconciliation logic should handle the case where the member has revoked all its
|
||||
// partitions pending revocation.
|
||||
return new ConsumerGroupMember.Builder(member)
|
||||
.setAssignedPartitions(newAssignedPartitions)
|
||||
.build();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Computes the next assignment.
|
||||
*
|
||||
|
@ -227,6 +361,8 @@ public class CurrentAssignmentBuilder {
|
|||
int memberEpoch,
|
||||
Map<Uuid, Set<Integer>> memberAssignedPartitions
|
||||
) {
|
||||
Set<Uuid> subscribedTopicIds = subscribedTopicIds();
|
||||
|
||||
boolean hasUnreleasedPartitions = false;
|
||||
Map<Uuid, Set<Integer>> newAssignedPartitions = new HashMap<>();
|
||||
Map<Uuid, Set<Integer>> newPartitionsPendingRevocation = new HashMap<>();
|
||||
|
@ -241,6 +377,11 @@ public class CurrentAssignmentBuilder {
|
|||
Set<Integer> currentAssignedPartitions = memberAssignedPartitions
|
||||
.getOrDefault(topicId, Set.of());
|
||||
|
||||
// If the member is no longer subscribed to the topic, treat its target assignment as empty.
|
||||
if (!subscribedTopicIds.contains(topicId)) {
|
||||
target = Set.of();
|
||||
}
|
||||
|
||||
// New Assigned Partitions = Previous Assigned Partitions ∩ Target
|
||||
Set<Integer> assignedPartitions = new HashSet<>(currentAssignedPartitions);
|
||||
assignedPartitions.retainAll(target);
|
||||
|
@ -317,4 +458,28 @@ public class CurrentAssignmentBuilder {
|
|||
.build();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the set of topic IDs that the member is subscribed to.
|
||||
*
|
||||
* @return The set of topic IDs that the member is subscribed to.
|
||||
*/
|
||||
private Set<Uuid> subscribedTopicIds() {
|
||||
Set<String> subscriptions = member.subscribedTopicNames();
|
||||
String subscribedTopicRegex = member.subscribedTopicRegex();
|
||||
if (subscribedTopicRegex != null && !subscribedTopicRegex.isEmpty()) {
|
||||
ResolvedRegularExpression resolvedRegularExpression = resolvedRegularExpressions.get(subscribedTopicRegex);
|
||||
if (resolvedRegularExpression != null) {
|
||||
if (subscriptions.isEmpty()) {
|
||||
subscriptions = resolvedRegularExpression.topics();
|
||||
} else if (!resolvedRegularExpression.topics().isEmpty()) {
|
||||
subscriptions = new UnionSet<>(subscriptions, resolvedRegularExpression.topics());
|
||||
}
|
||||
} else {
|
||||
// Treat an unresolved regex as matching no topics, to be conservative.
|
||||
}
|
||||
}
|
||||
|
||||
return new TopicIds(subscriptions, metadataImage);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -16,10 +16,16 @@
|
|||
*/
|
||||
package org.apache.kafka.coordinator.group.modern.share;
|
||||
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
|
||||
import org.apache.kafka.coordinator.group.modern.Assignment;
|
||||
import org.apache.kafka.coordinator.group.modern.MemberState;
|
||||
import org.apache.kafka.coordinator.group.modern.TopicIds;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* The ShareGroupAssignmentBuilder class encapsulates the reconciliation engine of the
|
||||
|
@ -32,6 +38,11 @@ public class ShareGroupAssignmentBuilder {
|
|||
*/
|
||||
private final ShareGroupMember member;
|
||||
|
||||
/**
|
||||
* The metadata image.
|
||||
*/
|
||||
private CoordinatorMetadataImage metadataImage = CoordinatorMetadataImage.EMPTY;
|
||||
|
||||
/**
|
||||
* The target assignment epoch.
|
||||
*/
|
||||
|
@ -42,6 +53,11 @@ public class ShareGroupAssignmentBuilder {
|
|||
*/
|
||||
private Assignment targetAssignment;
|
||||
|
||||
/**
|
||||
* Whether the member has changed its subscription on the current heartbeat.
|
||||
*/
|
||||
private boolean hasSubscriptionChanged;
|
||||
|
||||
/**
|
||||
* Constructs the ShareGroupAssignmentBuilder based on the current state of the
|
||||
* provided share group member.
|
||||
|
@ -52,6 +68,19 @@ public class ShareGroupAssignmentBuilder {
|
|||
this.member = Objects.requireNonNull(member);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the metadata image.
|
||||
*
|
||||
* @param metadataImage The metadata image.
|
||||
* @return This object.
|
||||
*/
|
||||
public ShareGroupAssignmentBuilder withMetadataImage(
|
||||
CoordinatorMetadataImage metadataImage
|
||||
) {
|
||||
this.metadataImage = metadataImage;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the target assignment epoch and the target assignment that the
|
||||
* share group member must be reconciled to.
|
||||
|
@ -69,6 +98,19 @@ public class ShareGroupAssignmentBuilder {
|
|||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets whether the member has changed its subscription on the current heartbeat.
|
||||
*
|
||||
* @param hasSubscriptionChanged If true, always removes unsubscribed topics from the current assignment.
|
||||
* @return This object.
|
||||
*/
|
||||
public ShareGroupAssignmentBuilder withHasSubscriptionChanged(
|
||||
boolean hasSubscriptionChanged
|
||||
) {
|
||||
this.hasSubscriptionChanged = hasSubscriptionChanged;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Builds the next state for the member or keep the current one if it
|
||||
* is not possible to move forward with the current state.
|
||||
|
@ -83,11 +125,38 @@ public class ShareGroupAssignmentBuilder {
|
|||
// when the member is updated.
|
||||
return new ShareGroupMember.Builder(member)
|
||||
.setState(MemberState.STABLE)
|
||||
.setAssignedPartitions(targetAssignment.partitions())
|
||||
// If we have client-side assignors, the latest target assignment may not
|
||||
// be consistent with the latest subscribed topics, so we must always
|
||||
// filter the assigned partitions to ensure they are consistent with the
|
||||
// subscribed topics.
|
||||
.setAssignedPartitions(filterAssignedPartitions(targetAssignment.partitions(), member.subscribedTopicNames()))
|
||||
.updateMemberEpoch(targetAssignmentEpoch)
|
||||
.build();
|
||||
} else if (hasSubscriptionChanged) {
|
||||
return new ShareGroupMember.Builder(member)
|
||||
.setAssignedPartitions(filterAssignedPartitions(targetAssignment.partitions(), member.subscribedTopicNames()))
|
||||
.build();
|
||||
} else {
|
||||
return member;
|
||||
}
|
||||
}
|
||||
|
||||
return member;
|
||||
private Map<Uuid, Set<Integer>> filterAssignedPartitions(
|
||||
Map<Uuid, Set<Integer>> partitions,
|
||||
Set<String> subscribedTopicNames
|
||||
) {
|
||||
TopicIds subscribedTopicIds = new TopicIds(member.subscribedTopicNames(), metadataImage);
|
||||
|
||||
// Reuse the original map if no topics need to be removed.
|
||||
Map<Uuid, Set<Integer>> filteredPartitions = partitions;
|
||||
for (Map.Entry<Uuid, Set<Integer>> entry : partitions.entrySet()) {
|
||||
if (!subscribedTopicIds.contains(entry.getKey())) {
|
||||
if (filteredPartitions == partitions) {
|
||||
filteredPartitions = new HashMap<>(partitions);
|
||||
}
|
||||
filteredPartitions.remove(entry.getKey());
|
||||
}
|
||||
}
|
||||
return filteredPartitions;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -74,6 +74,7 @@ public class ShareGroupMember extends ModernGroupMember {
|
|||
this.memberId = Objects.requireNonNull(newMemberId);
|
||||
this.memberEpoch = member.memberEpoch;
|
||||
this.previousMemberEpoch = member.previousMemberEpoch;
|
||||
this.state = member.state;
|
||||
this.rackId = member.rackId;
|
||||
this.clientId = member.clientId;
|
||||
this.clientHost = member.clientHost;
|
||||
|
|
|
@ -113,7 +113,6 @@ public class StreamsGroup implements Group {
|
|||
}
|
||||
}
|
||||
|
||||
private final LogContext logContext;
|
||||
private final Logger log;
|
||||
|
||||
/**
|
||||
|
@ -217,7 +216,6 @@ public class StreamsGroup implements Group {
|
|||
String groupId
|
||||
) {
|
||||
this.log = logContext.logger(StreamsGroup.class);
|
||||
this.logContext = logContext;
|
||||
this.snapshotRegistry = Objects.requireNonNull(snapshotRegistry);
|
||||
this.groupId = Objects.requireNonNull(groupId);
|
||||
this.state = new TimelineObject<>(snapshotRegistry, EMPTY);
|
||||
|
|
|
@ -20711,7 +20711,7 @@ public class GroupMetadataManagerTest {
|
|||
.build();
|
||||
|
||||
// Member 1 updates its new regular expression.
|
||||
CoordinatorResult<ConsumerGroupHeartbeatResponseData, CoordinatorRecord> result = context.consumerGroupHeartbeat(
|
||||
CoordinatorResult<ConsumerGroupHeartbeatResponseData, CoordinatorRecord> result1 = context.consumerGroupHeartbeat(
|
||||
new ConsumerGroupHeartbeatRequestData()
|
||||
.setGroupId(groupId)
|
||||
.setMemberId(memberId1)
|
||||
|
@ -20727,19 +20727,15 @@ public class GroupMetadataManagerTest {
|
|||
.setMemberEpoch(10)
|
||||
.setHeartbeatIntervalMs(5000)
|
||||
.setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()
|
||||
.setTopicPartitions(List.of(
|
||||
new ConsumerGroupHeartbeatResponseData.TopicPartitions()
|
||||
.setTopicId(fooTopicId)
|
||||
.setPartitions(List.of(0, 1, 2, 3, 4, 5))
|
||||
))
|
||||
.setTopicPartitions(List.of())
|
||||
),
|
||||
result.response()
|
||||
result1.response()
|
||||
);
|
||||
|
||||
ConsumerGroupMember expectedMember1 = new ConsumerGroupMember.Builder(memberId1)
|
||||
.setState(MemberState.STABLE)
|
||||
.setMemberEpoch(10)
|
||||
.setPreviousMemberEpoch(0)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setClientId(DEFAULT_CLIENT_ID)
|
||||
.setClientHost(DEFAULT_CLIENT_ADDRESS.toString())
|
||||
.setRebalanceTimeoutMs(5000)
|
||||
|
@ -20751,10 +20747,12 @@ public class GroupMetadataManagerTest {
|
|||
// The member subscription is updated.
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember1),
|
||||
// The previous regular expression is deleted.
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionTombstone(groupId, "foo*")
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionTombstone(groupId, "foo*"),
|
||||
// The member assignment is updated.
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedMember1)
|
||||
);
|
||||
|
||||
assertRecordsEquals(expectedRecords, result.records());
|
||||
assertRecordsEquals(expectedRecords, result1.records());
|
||||
|
||||
// Execute pending tasks.
|
||||
List<MockCoordinatorExecutor.ExecutorResult<CoordinatorRecord>> tasks = context.processTasks();
|
||||
|
@ -20782,6 +20780,65 @@ public class GroupMetadataManagerTest {
|
|||
),
|
||||
task.result().records()
|
||||
);
|
||||
|
||||
assignor.prepareGroupAssignment(new GroupAssignment(Map.of(
|
||||
memberId1, new MemberAssignmentImpl(mkAssignment(
|
||||
mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5),
|
||||
mkTopicAssignment(barTopicId, 0, 1, 2)
|
||||
))
|
||||
)));
|
||||
|
||||
// Member heartbeats again with the same regex.
|
||||
CoordinatorResult<ConsumerGroupHeartbeatResponseData, CoordinatorRecord> result2 = context.consumerGroupHeartbeat(
|
||||
new ConsumerGroupHeartbeatRequestData()
|
||||
.setGroupId(groupId)
|
||||
.setMemberId(memberId1)
|
||||
.setMemberEpoch(10)
|
||||
.setRebalanceTimeoutMs(5000)
|
||||
.setSubscribedTopicRegex("foo*|bar*")
|
||||
.setServerAssignor("range")
|
||||
.setTopicPartitions(List.of()));
|
||||
|
||||
assertResponseEquals(
|
||||
new ConsumerGroupHeartbeatResponseData()
|
||||
.setMemberId(memberId1)
|
||||
.setMemberEpoch(11)
|
||||
.setHeartbeatIntervalMs(5000)
|
||||
.setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()
|
||||
.setTopicPartitions(List.of(
|
||||
new ConsumerGroupHeartbeatResponseData.TopicPartitions()
|
||||
.setTopicId(fooTopicId)
|
||||
.setPartitions(List.of(0, 1, 2, 3, 4, 5)),
|
||||
new ConsumerGroupHeartbeatResponseData.TopicPartitions()
|
||||
.setTopicId(barTopicId)
|
||||
.setPartitions(List.of(0, 1, 2))))),
|
||||
result2.response()
|
||||
);
|
||||
|
||||
ConsumerGroupMember expectedMember2 = new ConsumerGroupMember.Builder(memberId1)
|
||||
.setState(MemberState.STABLE)
|
||||
.setMemberEpoch(11)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setClientId(DEFAULT_CLIENT_ID)
|
||||
.setClientHost(DEFAULT_CLIENT_ADDRESS.toString())
|
||||
.setRebalanceTimeoutMs(5000)
|
||||
.setSubscribedTopicRegex("foo*|bar*")
|
||||
.setServerAssignorName("range")
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5),
|
||||
mkTopicAssignment(barTopicId, 0, 1, 2)))
|
||||
.build();
|
||||
|
||||
expectedRecords = List.of(
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId1, mkAssignment(
|
||||
mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5),
|
||||
mkTopicAssignment(barTopicId, 0, 1, 2)
|
||||
)),
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochRecord(groupId, 11),
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedMember2)
|
||||
);
|
||||
|
||||
assertRecordsEquals(expectedRecords, result2.records());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -21184,10 +21241,7 @@ public class GroupMetadataManagerTest {
|
|||
.setMemberEpoch(10)
|
||||
.setHeartbeatIntervalMs(5000)
|
||||
.setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()
|
||||
.setTopicPartitions(List.of(
|
||||
new ConsumerGroupHeartbeatResponseData.TopicPartitions()
|
||||
.setTopicId(fooTopicId)
|
||||
.setPartitions(List.of(3, 4, 5))))),
|
||||
.setTopicPartitions(List.of())),
|
||||
result1.response()
|
||||
);
|
||||
|
||||
|
@ -21205,7 +21259,8 @@ public class GroupMetadataManagerTest {
|
|||
assertRecordsEquals(
|
||||
List.of(
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember2),
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionTombstone(groupId, "foo*")
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionTombstone(groupId, "foo*"),
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedMember2)
|
||||
),
|
||||
result1.records()
|
||||
);
|
||||
|
@ -21271,8 +21326,7 @@ public class GroupMetadataManagerTest {
|
|||
.setRebalanceTimeoutMs(5000)
|
||||
.setSubscribedTopicRegex("foo|bar*")
|
||||
.setServerAssignorName("range")
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(fooTopicId, 3, 4, 5)))
|
||||
.setAssignedPartitions(mkAssignment())
|
||||
.build();
|
||||
|
||||
assertResponseEquals(
|
||||
|
@ -21281,10 +21335,7 @@ public class GroupMetadataManagerTest {
|
|||
.setMemberEpoch(11)
|
||||
.setHeartbeatIntervalMs(5000)
|
||||
.setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()
|
||||
.setTopicPartitions(List.of(
|
||||
new ConsumerGroupHeartbeatResponseData.TopicPartitions()
|
||||
.setTopicId(fooTopicId)
|
||||
.setPartitions(List.of(3, 4, 5))))),
|
||||
.setTopicPartitions(List.of())),
|
||||
result2.response()
|
||||
);
|
||||
|
||||
|
@ -21413,10 +21464,7 @@ public class GroupMetadataManagerTest {
|
|||
.setMemberEpoch(10)
|
||||
.setHeartbeatIntervalMs(5000)
|
||||
.setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()
|
||||
.setTopicPartitions(List.of(
|
||||
new ConsumerGroupHeartbeatResponseData.TopicPartitions()
|
||||
.setTopicId(fooTopicId)
|
||||
.setPartitions(List.of(3, 4, 5))))),
|
||||
.setTopicPartitions(List.of())),
|
||||
result1.response()
|
||||
);
|
||||
|
||||
|
@ -21434,7 +21482,8 @@ public class GroupMetadataManagerTest {
|
|||
assertRecordsEquals(
|
||||
List.of(
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember2),
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionTombstone(groupId, "foo*")
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionTombstone(groupId, "foo*"),
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedMember2)
|
||||
),
|
||||
result1.records()
|
||||
);
|
||||
|
@ -21547,6 +21596,219 @@ public class GroupMetadataManagerTest {
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStaticConsumerGroupMemberJoinsWithUpdatedRegex() {
|
||||
String groupId = "fooup";
|
||||
String memberId1 = Uuid.randomUuid().toString();
|
||||
String memberId2 = Uuid.randomUuid().toString();
|
||||
String instanceId = "instance-id";
|
||||
|
||||
Uuid fooTopicId = Uuid.randomUuid();
|
||||
String fooTopicName = "foo";
|
||||
Uuid barTopicId = Uuid.randomUuid();
|
||||
String barTopicName = "bar";
|
||||
|
||||
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
|
||||
.addTopic(fooTopicId, fooTopicName, 6)
|
||||
.addTopic(barTopicId, barTopicName, 3)
|
||||
.buildCoordinatorMetadataImage(12345L);
|
||||
|
||||
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
|
||||
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
|
||||
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor))
|
||||
.withMetadataImage(metadataImage)
|
||||
.withConsumerGroup(new ConsumerGroupBuilder(groupId, 10)
|
||||
.withMember(new ConsumerGroupMember.Builder(memberId1)
|
||||
.setInstanceId(instanceId)
|
||||
.setState(MemberState.STABLE)
|
||||
.setMemberEpoch(10)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setClientId(DEFAULT_CLIENT_ID)
|
||||
.setClientHost(DEFAULT_CLIENT_ADDRESS.toString())
|
||||
.setRebalanceTimeoutMs(5000)
|
||||
.setSubscribedTopicRegex("foo*|bar*")
|
||||
.setServerAssignorName("range")
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5),
|
||||
mkTopicAssignment(barTopicId, 0, 1, 2)))
|
||||
.build())
|
||||
.withAssignment(memberId1, mkAssignment(
|
||||
mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5),
|
||||
mkTopicAssignment(barTopicId, 0, 1, 2)))
|
||||
.withAssignmentEpoch(10))
|
||||
.build();
|
||||
|
||||
// Static member temporarily leaves the group.
|
||||
CoordinatorResult<ConsumerGroupHeartbeatResponseData, CoordinatorRecord> result1 = context.consumerGroupHeartbeat(
|
||||
new ConsumerGroupHeartbeatRequestData()
|
||||
.setGroupId(groupId)
|
||||
.setInstanceId(instanceId)
|
||||
.setMemberId(memberId1)
|
||||
.setMemberEpoch(LEAVE_GROUP_STATIC_MEMBER_EPOCH)
|
||||
);
|
||||
|
||||
assertResponseEquals(
|
||||
new ConsumerGroupHeartbeatResponseData()
|
||||
.setMemberId(memberId1)
|
||||
.setMemberEpoch(LEAVE_GROUP_STATIC_MEMBER_EPOCH),
|
||||
result1.response()
|
||||
);
|
||||
|
||||
// Static member joins the group with an updated regular expression.
|
||||
CoordinatorResult<ConsumerGroupHeartbeatResponseData, CoordinatorRecord> result2 = context.consumerGroupHeartbeat(
|
||||
new ConsumerGroupHeartbeatRequestData()
|
||||
.setGroupId(groupId)
|
||||
.setInstanceId(instanceId)
|
||||
.setMemberId(memberId2)
|
||||
.setMemberEpoch(0)
|
||||
.setRebalanceTimeoutMs(5000)
|
||||
.setSubscribedTopicRegex("foo*")
|
||||
.setServerAssignor("range")
|
||||
.setTopicPartitions(List.of()));
|
||||
|
||||
// The returned assignment does not contain topics not in the current regular expression.
|
||||
assertResponseEquals(
|
||||
new ConsumerGroupHeartbeatResponseData()
|
||||
.setMemberId(memberId2)
|
||||
.setMemberEpoch(10)
|
||||
.setHeartbeatIntervalMs(5000)
|
||||
.setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()
|
||||
.setTopicPartitions(List.of())
|
||||
),
|
||||
result2.response()
|
||||
);
|
||||
|
||||
ConsumerGroupMember expectedCopiedMember = new ConsumerGroupMember.Builder(memberId2)
|
||||
.setState(MemberState.STABLE)
|
||||
.setInstanceId(instanceId)
|
||||
.setMemberEpoch(0)
|
||||
.setPreviousMemberEpoch(0)
|
||||
.setClientId(DEFAULT_CLIENT_ID)
|
||||
.setClientHost(DEFAULT_CLIENT_ADDRESS.toString())
|
||||
.setRebalanceTimeoutMs(5000)
|
||||
.setSubscribedTopicRegex("foo*|bar*")
|
||||
.setServerAssignorName("range")
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5),
|
||||
mkTopicAssignment(barTopicId, 0, 1, 2)))
|
||||
.build();
|
||||
|
||||
ConsumerGroupMember expectedMember1 = new ConsumerGroupMember.Builder(memberId2)
|
||||
.setState(MemberState.STABLE)
|
||||
.setInstanceId(instanceId)
|
||||
.setMemberEpoch(10)
|
||||
.setPreviousMemberEpoch(0)
|
||||
.setClientId(DEFAULT_CLIENT_ID)
|
||||
.setClientHost(DEFAULT_CLIENT_ADDRESS.toString())
|
||||
.setRebalanceTimeoutMs(5000)
|
||||
.setSubscribedTopicRegex("foo*")
|
||||
.setServerAssignorName("range")
|
||||
.build();
|
||||
|
||||
List<CoordinatorRecord> expectedRecords = List.of(
|
||||
// The previous member is deleted.
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentTombstoneRecord(groupId, memberId1),
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentTombstoneRecord(groupId, memberId1),
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionTombstoneRecord(groupId, memberId1),
|
||||
// The previous member is replaced by the new one.
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedCopiedMember),
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId2, mkAssignment(
|
||||
mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5),
|
||||
mkTopicAssignment(barTopicId, 0, 1, 2)
|
||||
)),
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedCopiedMember),
|
||||
// The member subscription is updated.
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember1),
|
||||
// The previous regular expression is deleted.
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionTombstone(groupId, "foo*|bar*"),
|
||||
// The member assignment is updated.
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedMember1)
|
||||
);
|
||||
|
||||
assertRecordsEquals(expectedRecords, result2.records());
|
||||
|
||||
// Execute pending tasks.
|
||||
List<MockCoordinatorExecutor.ExecutorResult<CoordinatorRecord>> tasks = context.processTasks();
|
||||
assertEquals(1, tasks.size());
|
||||
|
||||
MockCoordinatorExecutor.ExecutorResult<CoordinatorRecord> task = tasks.get(0);
|
||||
assertEquals(groupId + "-regex", task.key());
|
||||
assertRecordsEquals(
|
||||
List.of(
|
||||
// The resolution of the new regex is persisted.
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionRecord(
|
||||
groupId,
|
||||
"foo*",
|
||||
new ResolvedRegularExpression(
|
||||
Set.of("foo"),
|
||||
12345L,
|
||||
context.time.milliseconds()
|
||||
)
|
||||
),
|
||||
// The group epoch is bumped.
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, computeGroupHash(Map.of(
|
||||
fooTopicName, computeTopicHash(fooTopicName, metadataImage)
|
||||
)))
|
||||
),
|
||||
task.result().records()
|
||||
);
|
||||
|
||||
assignor.prepareGroupAssignment(new GroupAssignment(Map.of(
|
||||
memberId2, new MemberAssignmentImpl(mkAssignment(
|
||||
mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5)
|
||||
))
|
||||
)));
|
||||
|
||||
// Member heartbeats again with the same regex.
|
||||
CoordinatorResult<ConsumerGroupHeartbeatResponseData, CoordinatorRecord> result3 = context.consumerGroupHeartbeat(
|
||||
new ConsumerGroupHeartbeatRequestData()
|
||||
.setGroupId(groupId)
|
||||
.setInstanceId(instanceId)
|
||||
.setMemberId(memberId2)
|
||||
.setMemberEpoch(10)
|
||||
.setRebalanceTimeoutMs(5000)
|
||||
.setSubscribedTopicRegex("foo*")
|
||||
.setServerAssignor("range")
|
||||
.setTopicPartitions(List.of()));
|
||||
|
||||
assertResponseEquals(
|
||||
new ConsumerGroupHeartbeatResponseData()
|
||||
.setMemberId(memberId2)
|
||||
.setMemberEpoch(11)
|
||||
.setHeartbeatIntervalMs(5000)
|
||||
.setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()
|
||||
.setTopicPartitions(List.of(
|
||||
new ConsumerGroupHeartbeatResponseData.TopicPartitions()
|
||||
.setTopicId(fooTopicId)
|
||||
.setPartitions(List.of(0, 1, 2, 3, 4, 5))))),
|
||||
result3.response()
|
||||
);
|
||||
|
||||
ConsumerGroupMember expectedMember2 = new ConsumerGroupMember.Builder(memberId2)
|
||||
.setState(MemberState.STABLE)
|
||||
.setInstanceId(instanceId)
|
||||
.setMemberEpoch(11)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setClientId(DEFAULT_CLIENT_ID)
|
||||
.setClientHost(DEFAULT_CLIENT_ADDRESS.toString())
|
||||
.setRebalanceTimeoutMs(5000)
|
||||
.setSubscribedTopicRegex("foo*|bar*")
|
||||
.setServerAssignorName("range")
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5)))
|
||||
.build();
|
||||
|
||||
expectedRecords = List.of(
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId2, mkAssignment(
|
||||
mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5)
|
||||
)),
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochRecord(groupId, 11),
|
||||
GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedMember2)
|
||||
);
|
||||
|
||||
assertRecordsEquals(expectedRecords, result3.records());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testResolvedRegularExpressionsRemovedWhenMembersLeaveOrFenced() {
|
||||
String groupId = "fooup";
|
||||
|
|
|
@ -19,13 +19,19 @@ package org.apache.kafka.coordinator.group.modern.consumer;
|
|||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.errors.FencedMemberEpochException;
|
||||
import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
|
||||
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
|
||||
import org.apache.kafka.coordinator.common.runtime.MetadataImageBuilder;
|
||||
import org.apache.kafka.coordinator.group.modern.Assignment;
|
||||
import org.apache.kafka.coordinator.group.modern.MemberState;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.CsvSource;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
|
||||
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
|
||||
|
@ -36,19 +42,28 @@ public class CurrentAssignmentBuilderTest {
|
|||
|
||||
@Test
|
||||
public void testStableToStable() {
|
||||
String topic1 = "topic1";
|
||||
String topic2 = "topic2";
|
||||
Uuid topicId1 = Uuid.randomUuid();
|
||||
Uuid topicId2 = Uuid.randomUuid();
|
||||
|
||||
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
|
||||
.addTopic(topicId1, topic1, 10)
|
||||
.addTopic(topicId2, topic2, 10)
|
||||
.buildCoordinatorMetadataImage();
|
||||
|
||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||
.setState(MemberState.STABLE)
|
||||
.setMemberEpoch(10)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||
.build();
|
||||
|
||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||
.withMetadataImage(metadataImage)
|
||||
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 4, 5, 6))))
|
||||
|
@ -60,6 +75,7 @@ public class CurrentAssignmentBuilderTest {
|
|||
.setState(MemberState.STABLE)
|
||||
.setMemberEpoch(11)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||
|
@ -70,19 +86,28 @@ public class CurrentAssignmentBuilderTest {
|
|||
|
||||
@Test
|
||||
public void testStableToStableWithNewPartitions() {
|
||||
String topic1 = "topic1";
|
||||
String topic2 = "topic2";
|
||||
Uuid topicId1 = Uuid.randomUuid();
|
||||
Uuid topicId2 = Uuid.randomUuid();
|
||||
|
||||
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
|
||||
.addTopic(topicId1, topic1, 10)
|
||||
.addTopic(topicId2, topic2, 10)
|
||||
.buildCoordinatorMetadataImage();
|
||||
|
||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||
.setState(MemberState.STABLE)
|
||||
.setMemberEpoch(10)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||
.build();
|
||||
|
||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||
.withMetadataImage(metadataImage)
|
||||
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 2, 3, 4),
|
||||
mkTopicAssignment(topicId2, 4, 5, 6, 7))))
|
||||
|
@ -94,6 +119,7 @@ public class CurrentAssignmentBuilderTest {
|
|||
.setState(MemberState.STABLE)
|
||||
.setMemberEpoch(11)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 2, 3, 4),
|
||||
mkTopicAssignment(topicId2, 4, 5, 6, 7)))
|
||||
|
@ -104,19 +130,28 @@ public class CurrentAssignmentBuilderTest {
|
|||
|
||||
@Test
|
||||
public void testStableToUnrevokedPartitions() {
|
||||
String topic1 = "topic1";
|
||||
String topic2 = "topic2";
|
||||
Uuid topicId1 = Uuid.randomUuid();
|
||||
Uuid topicId2 = Uuid.randomUuid();
|
||||
|
||||
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
|
||||
.addTopic(topicId1, topic1, 10)
|
||||
.addTopic(topicId2, topic2, 10)
|
||||
.buildCoordinatorMetadataImage();
|
||||
|
||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||
.setState(MemberState.STABLE)
|
||||
.setMemberEpoch(10)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||
.build();
|
||||
|
||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||
.withMetadataImage(metadataImage)
|
||||
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 2, 3, 4),
|
||||
mkTopicAssignment(topicId2, 5, 6, 7))))
|
||||
|
@ -128,6 +163,7 @@ public class CurrentAssignmentBuilderTest {
|
|||
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||
.setMemberEpoch(10)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 5, 6)))
|
||||
|
@ -141,19 +177,28 @@ public class CurrentAssignmentBuilderTest {
|
|||
|
||||
@Test
|
||||
public void testStableToUnreleasedPartitions() {
|
||||
String topic1 = "topic1";
|
||||
String topic2 = "topic2";
|
||||
Uuid topicId1 = Uuid.randomUuid();
|
||||
Uuid topicId2 = Uuid.randomUuid();
|
||||
|
||||
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
|
||||
.addTopic(topicId1, topic1, 10)
|
||||
.addTopic(topicId2, topic2, 10)
|
||||
.buildCoordinatorMetadataImage();
|
||||
|
||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||
.setState(MemberState.STABLE)
|
||||
.setMemberEpoch(10)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||
.build();
|
||||
|
||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||
.withMetadataImage(metadataImage)
|
||||
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 2, 3, 4),
|
||||
mkTopicAssignment(topicId2, 4, 5, 6, 7))))
|
||||
|
@ -165,6 +210,7 @@ public class CurrentAssignmentBuilderTest {
|
|||
.setState(MemberState.UNRELEASED_PARTITIONS)
|
||||
.setMemberEpoch(11)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||
|
@ -175,19 +221,28 @@ public class CurrentAssignmentBuilderTest {
|
|||
|
||||
@Test
|
||||
public void testStableToUnreleasedPartitionsWithOwnedPartitionsNotHavingRevokedPartitions() {
|
||||
String topic1 = "topic1";
|
||||
String topic2 = "topic2";
|
||||
Uuid topicId1 = Uuid.randomUuid();
|
||||
Uuid topicId2 = Uuid.randomUuid();
|
||||
|
||||
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
|
||||
.addTopic(topicId1, topic1, 10)
|
||||
.addTopic(topicId2, topic2, 10)
|
||||
.buildCoordinatorMetadataImage();
|
||||
|
||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||
.setState(MemberState.STABLE)
|
||||
.setMemberEpoch(10)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||
.build();
|
||||
|
||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||
.withMetadataImage(metadataImage)
|
||||
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 4, 5, 7))))
|
||||
|
@ -202,6 +257,7 @@ public class CurrentAssignmentBuilderTest {
|
|||
.setState(MemberState.UNRELEASED_PARTITIONS)
|
||||
.setMemberEpoch(11)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 4, 5)))
|
||||
|
@ -212,13 +268,21 @@ public class CurrentAssignmentBuilderTest {
|
|||
|
||||
@Test
|
||||
public void testUnrevokedPartitionsToStable() {
|
||||
String topic1 = "topic1";
|
||||
String topic2 = "topic2";
|
||||
Uuid topicId1 = Uuid.randomUuid();
|
||||
Uuid topicId2 = Uuid.randomUuid();
|
||||
|
||||
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
|
||||
.addTopic(topicId1, topic1, 10)
|
||||
.addTopic(topicId2, topic2, 10)
|
||||
.buildCoordinatorMetadataImage();
|
||||
|
||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||
.setMemberEpoch(10)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 5, 6)))
|
||||
|
@ -228,6 +292,7 @@ public class CurrentAssignmentBuilderTest {
|
|||
.build();
|
||||
|
||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||
.withMetadataImage(metadataImage)
|
||||
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 5, 6))))
|
||||
|
@ -246,6 +311,7 @@ public class CurrentAssignmentBuilderTest {
|
|||
.setState(MemberState.STABLE)
|
||||
.setMemberEpoch(11)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 5, 6)))
|
||||
|
@ -256,13 +322,21 @@ public class CurrentAssignmentBuilderTest {
|
|||
|
||||
@Test
|
||||
public void testRemainsInUnrevokedPartitions() {
|
||||
String topic1 = "topic1";
|
||||
String topic2 = "topic2";
|
||||
Uuid topicId1 = Uuid.randomUuid();
|
||||
Uuid topicId2 = Uuid.randomUuid();
|
||||
|
||||
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
|
||||
.addTopic(topicId1, topic1, 10)
|
||||
.addTopic(topicId2, topic2, 10)
|
||||
.buildCoordinatorMetadataImage();
|
||||
|
||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||
.setMemberEpoch(10)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 5, 6)))
|
||||
|
@ -272,6 +346,7 @@ public class CurrentAssignmentBuilderTest {
|
|||
.build();
|
||||
|
||||
CurrentAssignmentBuilder currentAssignmentBuilder = new CurrentAssignmentBuilder(member)
|
||||
.withMetadataImage(metadataImage)
|
||||
.withTargetAssignment(12, new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 3),
|
||||
mkTopicAssignment(topicId2, 6))))
|
||||
|
@ -311,15 +386,27 @@ public class CurrentAssignmentBuilderTest {
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnrevokedPartitionsToUnrevokedPartitions() {
|
||||
@ParameterizedTest
|
||||
@CsvSource({
|
||||
"10, 12, 11",
|
||||
"10, 10, 10", // The member epoch must not advance past the target assignment epoch.
|
||||
})
|
||||
public void testUnrevokedPartitionsToUnrevokedPartitions(int memberEpoch, int targetAssignmentEpoch, int expectedMemberEpoch) {
|
||||
String topic1 = "topic1";
|
||||
String topic2 = "topic2";
|
||||
Uuid topicId1 = Uuid.randomUuid();
|
||||
Uuid topicId2 = Uuid.randomUuid();
|
||||
|
||||
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
|
||||
.addTopic(topicId1, topic1, 10)
|
||||
.addTopic(topicId2, topic2, 10)
|
||||
.buildCoordinatorMetadataImage();
|
||||
|
||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||
.setMemberEpoch(10)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setMemberEpoch(memberEpoch)
|
||||
.setPreviousMemberEpoch(memberEpoch)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 5, 6)))
|
||||
|
@ -329,7 +416,8 @@ public class CurrentAssignmentBuilderTest {
|
|||
.build();
|
||||
|
||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||
.withTargetAssignment(12, new Assignment(mkAssignment(
|
||||
.withMetadataImage(metadataImage)
|
||||
.withTargetAssignment(targetAssignmentEpoch, new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 3),
|
||||
mkTopicAssignment(topicId2, 6))))
|
||||
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
|
||||
|
@ -345,8 +433,9 @@ public class CurrentAssignmentBuilderTest {
|
|||
assertEquals(
|
||||
new ConsumerGroupMember.Builder("member")
|
||||
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||
.setMemberEpoch(11)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setMemberEpoch(expectedMemberEpoch)
|
||||
.setPreviousMemberEpoch(memberEpoch)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 3),
|
||||
mkTopicAssignment(topicId2, 6)))
|
||||
|
@ -360,19 +449,28 @@ public class CurrentAssignmentBuilderTest {
|
|||
|
||||
@Test
|
||||
public void testUnrevokedPartitionsToUnreleasedPartitions() {
|
||||
String topic1 = "topic1";
|
||||
String topic2 = "topic2";
|
||||
Uuid topicId1 = Uuid.randomUuid();
|
||||
Uuid topicId2 = Uuid.randomUuid();
|
||||
|
||||
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
|
||||
.addTopic(topicId1, topic1, 10)
|
||||
.addTopic(topicId2, topic2, 10)
|
||||
.buildCoordinatorMetadataImage();
|
||||
|
||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||
.setMemberEpoch(11)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 5, 6)))
|
||||
.build();
|
||||
|
||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||
.withMetadataImage(metadataImage)
|
||||
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 2, 3, 4),
|
||||
mkTopicAssignment(topicId2, 5, 6, 7))))
|
||||
|
@ -391,6 +489,7 @@ public class CurrentAssignmentBuilderTest {
|
|||
.setState(MemberState.UNRELEASED_PARTITIONS)
|
||||
.setMemberEpoch(11)
|
||||
.setPreviousMemberEpoch(11)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 5, 6)))
|
||||
|
@ -401,19 +500,28 @@ public class CurrentAssignmentBuilderTest {
|
|||
|
||||
@Test
|
||||
public void testUnreleasedPartitionsToStable() {
|
||||
String topic1 = "topic1";
|
||||
String topic2 = "topic2";
|
||||
Uuid topicId1 = Uuid.randomUuid();
|
||||
Uuid topicId2 = Uuid.randomUuid();
|
||||
|
||||
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
|
||||
.addTopic(topicId1, topic1, 10)
|
||||
.addTopic(topicId2, topic2, 10)
|
||||
.buildCoordinatorMetadataImage();
|
||||
|
||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||
.setState(MemberState.UNRELEASED_PARTITIONS)
|
||||
.setMemberEpoch(11)
|
||||
.setPreviousMemberEpoch(11)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 5, 6)))
|
||||
.build();
|
||||
|
||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||
.withMetadataImage(metadataImage)
|
||||
.withTargetAssignment(12, new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 5, 6))))
|
||||
|
@ -425,6 +533,7 @@ public class CurrentAssignmentBuilderTest {
|
|||
.setState(MemberState.STABLE)
|
||||
.setMemberEpoch(12)
|
||||
.setPreviousMemberEpoch(11)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 5, 6)))
|
||||
|
@ -435,19 +544,28 @@ public class CurrentAssignmentBuilderTest {
|
|||
|
||||
@Test
|
||||
public void testUnreleasedPartitionsToStableWithNewPartitions() {
|
||||
String topic1 = "topic1";
|
||||
String topic2 = "topic2";
|
||||
Uuid topicId1 = Uuid.randomUuid();
|
||||
Uuid topicId2 = Uuid.randomUuid();
|
||||
|
||||
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
|
||||
.addTopic(topicId1, topic1, 10)
|
||||
.addTopic(topicId2, topic2, 10)
|
||||
.buildCoordinatorMetadataImage();
|
||||
|
||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||
.setState(MemberState.UNRELEASED_PARTITIONS)
|
||||
.setMemberEpoch(11)
|
||||
.setPreviousMemberEpoch(11)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 5, 6)))
|
||||
.build();
|
||||
|
||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||
.withMetadataImage(metadataImage)
|
||||
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 2, 3, 4),
|
||||
mkTopicAssignment(topicId2, 5, 6, 7))))
|
||||
|
@ -459,6 +577,7 @@ public class CurrentAssignmentBuilderTest {
|
|||
.setState(MemberState.STABLE)
|
||||
.setMemberEpoch(11)
|
||||
.setPreviousMemberEpoch(11)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 2, 3, 4),
|
||||
mkTopicAssignment(topicId2, 5, 6, 7)))
|
||||
|
@ -469,19 +588,28 @@ public class CurrentAssignmentBuilderTest {
|
|||
|
||||
@Test
|
||||
public void testUnreleasedPartitionsToUnreleasedPartitions() {
|
||||
String topic1 = "topic1";
|
||||
String topic2 = "topic2";
|
||||
Uuid topicId1 = Uuid.randomUuid();
|
||||
Uuid topicId2 = Uuid.randomUuid();
|
||||
|
||||
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
|
||||
.addTopic(topicId1, topic1, 10)
|
||||
.addTopic(topicId2, topic2, 10)
|
||||
.buildCoordinatorMetadataImage();
|
||||
|
||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||
.setState(MemberState.UNRELEASED_PARTITIONS)
|
||||
.setMemberEpoch(11)
|
||||
.setPreviousMemberEpoch(11)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 5, 6)))
|
||||
.build();
|
||||
|
||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||
.withMetadataImage(metadataImage)
|
||||
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 2, 3, 4),
|
||||
mkTopicAssignment(topicId2, 5, 6, 7))))
|
||||
|
@ -493,19 +621,28 @@ public class CurrentAssignmentBuilderTest {
|
|||
|
||||
@Test
|
||||
public void testUnreleasedPartitionsToUnrevokedPartitions() {
|
||||
String topic1 = "topic1";
|
||||
String topic2 = "topic2";
|
||||
Uuid topicId1 = Uuid.randomUuid();
|
||||
Uuid topicId2 = Uuid.randomUuid();
|
||||
|
||||
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
|
||||
.addTopic(topicId1, topic1, 10)
|
||||
.addTopic(topicId2, topic2, 10)
|
||||
.buildCoordinatorMetadataImage();
|
||||
|
||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||
.setState(MemberState.UNRELEASED_PARTITIONS)
|
||||
.setMemberEpoch(11)
|
||||
.setPreviousMemberEpoch(11)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 5, 6)))
|
||||
.build();
|
||||
|
||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||
.withMetadataImage(metadataImage)
|
||||
.withTargetAssignment(12, new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 3),
|
||||
mkTopicAssignment(topicId2, 6))))
|
||||
|
@ -517,6 +654,7 @@ public class CurrentAssignmentBuilderTest {
|
|||
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||
.setMemberEpoch(11)
|
||||
.setPreviousMemberEpoch(11)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 3),
|
||||
mkTopicAssignment(topicId2, 6)))
|
||||
|
@ -530,13 +668,21 @@ public class CurrentAssignmentBuilderTest {
|
|||
|
||||
@Test
|
||||
public void testUnknownState() {
|
||||
String topic1 = "topic1";
|
||||
String topic2 = "topic2";
|
||||
Uuid topicId1 = Uuid.randomUuid();
|
||||
Uuid topicId2 = Uuid.randomUuid();
|
||||
|
||||
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
|
||||
.addTopic(topicId1, topic1, 10)
|
||||
.addTopic(topicId2, topic2, 10)
|
||||
.buildCoordinatorMetadataImage();
|
||||
|
||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||
.setState(MemberState.UNKNOWN)
|
||||
.setMemberEpoch(11)
|
||||
.setPreviousMemberEpoch(11)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 3),
|
||||
mkTopicAssignment(topicId2, 6)))
|
||||
|
@ -548,6 +694,7 @@ public class CurrentAssignmentBuilderTest {
|
|||
// When the member is in an unknown state, the member is first to force
|
||||
// a reset of the client side member state.
|
||||
assertThrows(FencedMemberEpochException.class, () -> new CurrentAssignmentBuilder(member)
|
||||
.withMetadataImage(metadataImage)
|
||||
.withTargetAssignment(12, new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 3),
|
||||
mkTopicAssignment(topicId2, 6))))
|
||||
|
@ -556,6 +703,7 @@ public class CurrentAssignmentBuilderTest {
|
|||
|
||||
// Then the member rejoins with no owned partitions.
|
||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||
.withMetadataImage(metadataImage)
|
||||
.withTargetAssignment(12, new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 3),
|
||||
mkTopicAssignment(topicId2, 6))))
|
||||
|
@ -568,6 +716,7 @@ public class CurrentAssignmentBuilderTest {
|
|||
.setState(MemberState.STABLE)
|
||||
.setMemberEpoch(12)
|
||||
.setPreviousMemberEpoch(11)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 3),
|
||||
mkTopicAssignment(topicId2, 6)))
|
||||
|
@ -575,4 +724,355 @@ public class CurrentAssignmentBuilderTest {
|
|||
updatedMember
|
||||
);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@CsvSource({
|
||||
"10, 11, 11, false", // When advancing to a new target assignment, the assignment should
|
||||
"10, 11, 11, true", // always take the subscription into account.
|
||||
"10, 10, 10, true",
|
||||
})
|
||||
public void testStableToStableWithAssignmentTopicsNoLongerInSubscription(
|
||||
int memberEpoch,
|
||||
int targetAssignmentEpoch,
|
||||
int expectedMemberEpoch,
|
||||
boolean hasSubscriptionChanged
|
||||
) {
|
||||
String topic1 = "topic1";
|
||||
String topic2 = "topic2";
|
||||
Uuid topicId1 = Uuid.randomUuid();
|
||||
Uuid topicId2 = Uuid.randomUuid();
|
||||
|
||||
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
|
||||
.addTopic(topicId1, topic1, 10)
|
||||
.addTopic(topicId2, topic2, 10)
|
||||
.buildCoordinatorMetadataImage();
|
||||
|
||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||
.setState(MemberState.STABLE)
|
||||
.setMemberEpoch(memberEpoch)
|
||||
.setPreviousMemberEpoch(memberEpoch)
|
||||
.setSubscribedTopicNames(List.of(topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||
.build();
|
||||
|
||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||
.withMetadataImage(metadataImage)
|
||||
.withTargetAssignment(targetAssignmentEpoch, new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 4, 5, 6))))
|
||||
.withHasSubscriptionChanged(hasSubscriptionChanged)
|
||||
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
|
||||
.withOwnedTopicPartitions(Arrays.asList(
|
||||
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
|
||||
.setTopicId(topicId2)
|
||||
.setPartitions(Arrays.asList(4, 5, 6))))
|
||||
.build();
|
||||
|
||||
assertEquals(
|
||||
new ConsumerGroupMember.Builder("member")
|
||||
.setState(MemberState.STABLE)
|
||||
.setMemberEpoch(expectedMemberEpoch)
|
||||
.setPreviousMemberEpoch(memberEpoch)
|
||||
.setSubscribedTopicNames(List.of(topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||
.build(),
|
||||
updatedMember
|
||||
);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@CsvSource({
|
||||
"10, 11, 10, false", // When advancing to a new target assignment, the assignment should always
|
||||
"10, 11, 10, true", // take the subscription into account.
|
||||
"10, 10, 10, true"
|
||||
})
|
||||
public void testStableToUnrevokedPartitionsWithAssignmentTopicsNoLongerInSubscription(
|
||||
int memberEpoch,
|
||||
int targetAssignmentEpoch,
|
||||
int expectedMemberEpoch,
|
||||
boolean hasSubscriptionChanged
|
||||
) {
|
||||
String topic1 = "topic1";
|
||||
String topic2 = "topic2";
|
||||
Uuid topicId1 = Uuid.randomUuid();
|
||||
Uuid topicId2 = Uuid.randomUuid();
|
||||
|
||||
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
|
||||
.addTopic(topicId1, topic1, 10)
|
||||
.addTopic(topicId2, topic2, 10)
|
||||
.buildCoordinatorMetadataImage();
|
||||
|
||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||
.setState(MemberState.STABLE)
|
||||
.setMemberEpoch(memberEpoch)
|
||||
.setPreviousMemberEpoch(memberEpoch)
|
||||
.setSubscribedTopicNames(List.of(topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||
.build();
|
||||
|
||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||
.withMetadataImage(metadataImage)
|
||||
.withTargetAssignment(targetAssignmentEpoch, new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 4, 5, 6))))
|
||||
.withHasSubscriptionChanged(hasSubscriptionChanged)
|
||||
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
|
||||
.withOwnedTopicPartitions(Arrays.asList(
|
||||
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
|
||||
.setTopicId(topicId1)
|
||||
.setPartitions(Arrays.asList(1, 2, 3)),
|
||||
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
|
||||
.setTopicId(topicId2)
|
||||
.setPartitions(Arrays.asList(4, 5, 6))))
|
||||
.build();
|
||||
|
||||
assertEquals(
|
||||
new ConsumerGroupMember.Builder("member")
|
||||
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||
.setMemberEpoch(expectedMemberEpoch)
|
||||
.setPreviousMemberEpoch(memberEpoch)
|
||||
.setSubscribedTopicNames(List.of(topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||
.setPartitionsPendingRevocation(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 2, 3)))
|
||||
.build(),
|
||||
updatedMember
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRemainsInUnrevokedPartitionsWithAssignmentTopicsNoLongerInSubscription() {
|
||||
String topic1 = "topic1";
|
||||
String topic2 = "topic2";
|
||||
Uuid topicId1 = Uuid.randomUuid();
|
||||
Uuid topicId2 = Uuid.randomUuid();
|
||||
|
||||
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
|
||||
.addTopic(topicId1, topic1, 10)
|
||||
.addTopic(topicId2, topic2, 10)
|
||||
.buildCoordinatorMetadataImage();
|
||||
|
||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||
.setMemberEpoch(10)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setSubscribedTopicNames(List.of(topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 5, 6)))
|
||||
.setPartitionsPendingRevocation(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1),
|
||||
mkTopicAssignment(topicId2, 4)))
|
||||
.build();
|
||||
|
||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||
.withMetadataImage(metadataImage)
|
||||
.withTargetAssignment(12, new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 3, 4),
|
||||
mkTopicAssignment(topicId2, 6, 7))))
|
||||
.withHasSubscriptionChanged(true)
|
||||
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
|
||||
.withOwnedTopicPartitions(Arrays.asList(
|
||||
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
|
||||
.setTopicId(topicId1)
|
||||
.setPartitions(Arrays.asList(1, 2, 3)),
|
||||
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
|
||||
.setTopicId(topicId2)
|
||||
.setPartitions(Arrays.asList(4, 5, 6))))
|
||||
.build();
|
||||
|
||||
assertEquals(
|
||||
new ConsumerGroupMember.Builder("member")
|
||||
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||
.setMemberEpoch(10)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setSubscribedTopicNames(List.of(topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId2, 5, 6)))
|
||||
.setPartitionsPendingRevocation(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 4)))
|
||||
.build(),
|
||||
updatedMember
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSubscribedTopicNameAndUnresolvedRegularExpression() {
|
||||
String fooTopic = "foo";
|
||||
String barTopic = "bar";
|
||||
Uuid fooTopicId = Uuid.randomUuid();
|
||||
Uuid barTopicId = Uuid.randomUuid();
|
||||
|
||||
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
|
||||
.addTopic(fooTopicId, fooTopic, 10)
|
||||
.addTopic(barTopicId, barTopic, 10)
|
||||
.buildCoordinatorMetadataImage();
|
||||
|
||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||
.setState(MemberState.STABLE)
|
||||
.setMemberEpoch(10)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setSubscribedTopicNames(List.of(fooTopic))
|
||||
.setSubscribedTopicRegex("bar*")
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(fooTopicId, 1, 2, 3),
|
||||
mkTopicAssignment(barTopicId, 4, 5, 6)))
|
||||
.build();
|
||||
|
||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||
.withMetadataImage(metadataImage)
|
||||
.withTargetAssignment(10, new Assignment(mkAssignment(
|
||||
mkTopicAssignment(fooTopicId, 1, 2, 3),
|
||||
mkTopicAssignment(barTopicId, 4, 5, 6))))
|
||||
.withHasSubscriptionChanged(true)
|
||||
.withResolvedRegularExpressions(Map.of())
|
||||
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
|
||||
.withOwnedTopicPartitions(Arrays.asList(
|
||||
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
|
||||
.setTopicId(fooTopicId)
|
||||
.setPartitions(Arrays.asList(1, 2, 3)),
|
||||
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
|
||||
.setTopicId(barTopicId)
|
||||
.setPartitions(Arrays.asList(4, 5, 6))))
|
||||
.build();
|
||||
|
||||
assertEquals(
|
||||
new ConsumerGroupMember.Builder("member")
|
||||
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||
.setMemberEpoch(10)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setSubscribedTopicNames(List.of(fooTopic))
|
||||
.setSubscribedTopicRegex("bar*")
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(fooTopicId, 1, 2, 3)))
|
||||
.setPartitionsPendingRevocation(mkAssignment(
|
||||
mkTopicAssignment(barTopicId, 4, 5, 6)))
|
||||
.build(),
|
||||
updatedMember
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnresolvedRegularExpression() {
|
||||
String fooTopic = "foo";
|
||||
String barTopic = "bar";
|
||||
Uuid fooTopicId = Uuid.randomUuid();
|
||||
Uuid barTopicId = Uuid.randomUuid();
|
||||
|
||||
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
|
||||
.addTopic(fooTopicId, fooTopic, 10)
|
||||
.addTopic(barTopicId, barTopic, 10)
|
||||
.buildCoordinatorMetadataImage();
|
||||
|
||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||
.setState(MemberState.STABLE)
|
||||
.setMemberEpoch(10)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setSubscribedTopicNames(List.of())
|
||||
.setSubscribedTopicRegex("bar*")
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(fooTopicId, 1, 2, 3),
|
||||
mkTopicAssignment(barTopicId, 4, 5, 6)))
|
||||
.build();
|
||||
|
||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||
.withMetadataImage(metadataImage)
|
||||
.withTargetAssignment(10, new Assignment(mkAssignment(
|
||||
mkTopicAssignment(fooTopicId, 1, 2, 3),
|
||||
mkTopicAssignment(barTopicId, 4, 5, 6))))
|
||||
.withHasSubscriptionChanged(true)
|
||||
.withResolvedRegularExpressions(Map.of())
|
||||
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
|
||||
.withOwnedTopicPartitions(Arrays.asList(
|
||||
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
|
||||
.setTopicId(fooTopicId)
|
||||
.setPartitions(Arrays.asList(1, 2, 3)),
|
||||
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
|
||||
.setTopicId(barTopicId)
|
||||
.setPartitions(Arrays.asList(4, 5, 6))))
|
||||
.build();
|
||||
|
||||
assertEquals(
|
||||
new ConsumerGroupMember.Builder("member")
|
||||
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||
.setMemberEpoch(10)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setSubscribedTopicNames(List.of())
|
||||
.setSubscribedTopicRegex("bar*")
|
||||
.setAssignedPartitions(mkAssignment())
|
||||
.setPartitionsPendingRevocation(mkAssignment(
|
||||
mkTopicAssignment(fooTopicId, 1, 2, 3),
|
||||
mkTopicAssignment(barTopicId, 4, 5, 6)))
|
||||
.build(),
|
||||
updatedMember
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSubscribedTopicNameAndResolvedRegularExpression() {
|
||||
String fooTopic = "foo";
|
||||
String barTopic = "bar";
|
||||
Uuid fooTopicId = Uuid.randomUuid();
|
||||
Uuid barTopicId = Uuid.randomUuid();
|
||||
|
||||
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
|
||||
.addTopic(fooTopicId, fooTopic, 10)
|
||||
.addTopic(barTopicId, barTopic, 10)
|
||||
.buildCoordinatorMetadataImage();
|
||||
|
||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||
.setState(MemberState.STABLE)
|
||||
.setMemberEpoch(10)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setSubscribedTopicNames(List.of(fooTopic))
|
||||
.setSubscribedTopicRegex("bar*")
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(fooTopicId, 1, 2, 3),
|
||||
mkTopicAssignment(barTopicId, 4, 5, 6)))
|
||||
.build();
|
||||
|
||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||
.withMetadataImage(metadataImage)
|
||||
.withTargetAssignment(10, new Assignment(mkAssignment(
|
||||
mkTopicAssignment(fooTopicId, 1, 2, 3),
|
||||
mkTopicAssignment(barTopicId, 4, 5, 6))))
|
||||
.withHasSubscriptionChanged(true)
|
||||
.withResolvedRegularExpressions(Map.of(
|
||||
"bar*", new ResolvedRegularExpression(
|
||||
Set.of("bar"),
|
||||
12345L,
|
||||
0L
|
||||
)
|
||||
))
|
||||
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
|
||||
.withOwnedTopicPartitions(Arrays.asList(
|
||||
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
|
||||
.setTopicId(fooTopicId)
|
||||
.setPartitions(Arrays.asList(1, 2, 3)),
|
||||
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
|
||||
.setTopicId(barTopicId)
|
||||
.setPartitions(Arrays.asList(4, 5, 6))))
|
||||
.build();
|
||||
|
||||
assertEquals(
|
||||
new ConsumerGroupMember.Builder("member")
|
||||
.setState(MemberState.STABLE)
|
||||
.setMemberEpoch(10)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setSubscribedTopicNames(List.of(fooTopic))
|
||||
.setSubscribedTopicRegex("bar*")
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(fooTopicId, 1, 2, 3),
|
||||
mkTopicAssignment(barTopicId, 4, 5, 6)))
|
||||
.build(),
|
||||
updatedMember
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,10 +17,16 @@
|
|||
package org.apache.kafka.coordinator.group.modern.share;
|
||||
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
|
||||
import org.apache.kafka.coordinator.common.runtime.MetadataImageBuilder;
|
||||
import org.apache.kafka.coordinator.group.modern.Assignment;
|
||||
import org.apache.kafka.coordinator.group.modern.MemberState;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.CsvSource;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
|
||||
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
|
||||
|
@ -30,19 +36,28 @@ public class ShareGroupAssignmentBuilderTest {
|
|||
|
||||
@Test
|
||||
public void testStableToStable() {
|
||||
String topic1 = "topic1";
|
||||
String topic2 = "topic2";
|
||||
Uuid topicId1 = Uuid.randomUuid();
|
||||
Uuid topicId2 = Uuid.randomUuid();
|
||||
|
||||
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
|
||||
.addTopic(topicId1, topic1, 10)
|
||||
.addTopic(topicId2, topic2, 10)
|
||||
.buildCoordinatorMetadataImage();
|
||||
|
||||
ShareGroupMember member = new ShareGroupMember.Builder("member")
|
||||
.setState(MemberState.STABLE)
|
||||
.setMemberEpoch(10)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||
.build();
|
||||
|
||||
ShareGroupMember updatedMember = new ShareGroupAssignmentBuilder(member)
|
||||
.withMetadataImage(metadataImage)
|
||||
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 4, 5, 6))))
|
||||
|
@ -53,6 +68,7 @@ public class ShareGroupAssignmentBuilderTest {
|
|||
.setState(MemberState.STABLE)
|
||||
.setMemberEpoch(11)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||
|
@ -63,19 +79,28 @@ public class ShareGroupAssignmentBuilderTest {
|
|||
|
||||
@Test
|
||||
public void testStableToStableWithNewPartitions() {
|
||||
String topic1 = "topic1";
|
||||
String topic2 = "topic2";
|
||||
Uuid topicId1 = Uuid.randomUuid();
|
||||
Uuid topicId2 = Uuid.randomUuid();
|
||||
|
||||
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
|
||||
.addTopic(topicId1, topic1, 10)
|
||||
.addTopic(topicId2, topic2, 10)
|
||||
.buildCoordinatorMetadataImage();
|
||||
|
||||
ShareGroupMember member = new ShareGroupMember.Builder("member")
|
||||
.setState(MemberState.STABLE)
|
||||
.setMemberEpoch(10)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||
.build();
|
||||
|
||||
ShareGroupMember updatedMember = new ShareGroupAssignmentBuilder(member)
|
||||
.withMetadataImage(metadataImage)
|
||||
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 2, 3, 4),
|
||||
mkTopicAssignment(topicId2, 4, 5, 6, 7))))
|
||||
|
@ -86,6 +111,7 @@ public class ShareGroupAssignmentBuilderTest {
|
|||
.setState(MemberState.STABLE)
|
||||
.setMemberEpoch(11)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setSubscribedTopicNames(List.of(topic1, topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 2, 3, 4),
|
||||
mkTopicAssignment(topicId2, 4, 5, 6, 7)))
|
||||
|
@ -93,4 +119,56 @@ public class ShareGroupAssignmentBuilderTest {
|
|||
updatedMember
|
||||
);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@CsvSource({
|
||||
"10, 11, false", // When advancing to a new target assignment, the assignment should always
|
||||
"10, 11, true", // take the subscription into account.
|
||||
"10, 10, true"
|
||||
})
|
||||
public void testStableToStableWithAssignmentTopicsNoLongerInSubscription(
|
||||
int memberEpoch,
|
||||
int targetAssignmentEpoch,
|
||||
boolean hasSubscriptionChanged
|
||||
) {
|
||||
String topic1 = "topic1";
|
||||
String topic2 = "topic2";
|
||||
Uuid topicId1 = Uuid.randomUuid();
|
||||
Uuid topicId2 = Uuid.randomUuid();
|
||||
|
||||
CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
|
||||
.addTopic(topicId1, topic1, 10)
|
||||
.addTopic(topicId2, topic2, 10)
|
||||
.buildCoordinatorMetadataImage();
|
||||
|
||||
ShareGroupMember member = new ShareGroupMember.Builder("member")
|
||||
.setState(MemberState.STABLE)
|
||||
.setMemberEpoch(memberEpoch)
|
||||
.setPreviousMemberEpoch(memberEpoch)
|
||||
.setSubscribedTopicNames(List.of(topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||
.build();
|
||||
|
||||
ShareGroupMember updatedMember = new ShareGroupAssignmentBuilder(member)
|
||||
.withMetadataImage(metadataImage)
|
||||
.withTargetAssignment(targetAssignmentEpoch, new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||
mkTopicAssignment(topicId2, 4, 5, 6))))
|
||||
.withHasSubscriptionChanged(hasSubscriptionChanged)
|
||||
.build();
|
||||
|
||||
assertEquals(
|
||||
new ShareGroupMember.Builder("member")
|
||||
.setState(MemberState.STABLE)
|
||||
.setMemberEpoch(targetAssignmentEpoch)
|
||||
.setPreviousMemberEpoch(memberEpoch)
|
||||
.setSubscribedTopicNames(List.of(topic2))
|
||||
.setAssignedPartitions(mkAssignment(
|
||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||
.build(),
|
||||
updatedMember
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
if (!useLastKnownLeaderInBalancedRecovery || !eligibleLeaderReplicasEnabled) return;
|
||||
if (record.isr() != null && record.isr().isEmpty() && (partition.lastKnownElr.length != 1 ||
|
||||
partition.lastKnownElr[0] != partition.leader)) {
|
||||
if (record.leader() == NO_LEADER && partition.lastKnownElr.length == 0) {
|
||||
// Only update the last known leader when the first time the partition becomes leaderless.
|
||||
record.setLastKnownElr(List.of(partition.leader));
|
||||
} else if ((record.leader() >= 0 || (partition.leader != NO_LEADER && record.leader() != NO_LEADER))
|
||||
&& partition.lastKnownElr.length > 0) {
|
||||
} else if (record.leader() >= 0 && partition.lastKnownElr.length > 0) {
|
||||
// Clear the LastKnownElr field if the partition will have or continues to have a valid leader.
|
||||
record.setLastKnownElr(List.of());
|
||||
}
|
||||
|
|
|
@ -406,7 +406,14 @@ public final class QuorumController implements Controller {
|
|||
|
||||
KafkaEventQueue queue = null;
|
||||
try {
|
||||
queue = new KafkaEventQueue(time, logContext, threadNamePrefix);
|
||||
queue = new KafkaEventQueue(
|
||||
time,
|
||||
logContext,
|
||||
threadNamePrefix,
|
||||
EventQueue.VoidEvent.INSTANCE,
|
||||
controllerMetrics::updateIdleTime
|
||||
);
|
||||
|
||||
return new QuorumController(
|
||||
nonFatalFaultHandler,
|
||||
fatalFaultHandler,
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.kafka.controller.metrics;
|
|||
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.server.metrics.KafkaYammerMetrics;
|
||||
import org.apache.kafka.server.metrics.TimeRatio;
|
||||
|
||||
import com.yammer.metrics.core.Gauge;
|
||||
import com.yammer.metrics.core.Histogram;
|
||||
|
@ -48,6 +49,8 @@ public class QuorumControllerMetrics implements AutoCloseable {
|
|||
"ControllerEventManager", "EventQueueTimeMs");
|
||||
private static final MetricName EVENT_QUEUE_PROCESSING_TIME_MS = getMetricName(
|
||||
"ControllerEventManager", "EventQueueProcessingTimeMs");
|
||||
private static final MetricName AVERAGE_IDLE_RATIO = getMetricName(
|
||||
"ControllerEventManager", "AvgIdleRatio");
|
||||
private static final MetricName LAST_APPLIED_RECORD_OFFSET = getMetricName(
|
||||
"KafkaController", "LastAppliedRecordOffset");
|
||||
private static final MetricName LAST_COMMITTED_RECORD_OFFSET = getMetricName(
|
||||
|
@ -64,6 +67,7 @@ public class QuorumControllerMetrics implements AutoCloseable {
|
|||
"KafkaController", "EventQueueOperationsTimedOutCount");
|
||||
private static final MetricName NEW_ACTIVE_CONTROLLERS_COUNT = getMetricName(
|
||||
"KafkaController", "NewActiveControllersCount");
|
||||
|
||||
private static final String TIME_SINCE_LAST_HEARTBEAT_RECEIVED_METRIC_NAME = "TimeSinceLastHeartbeatReceivedMs";
|
||||
private static final String BROKER_ID_TAG = "broker";
|
||||
|
||||
|
@ -75,6 +79,7 @@ public class QuorumControllerMetrics implements AutoCloseable {
|
|||
private final AtomicLong lastAppliedRecordTimestamp = new AtomicLong(0);
|
||||
private final Consumer<Long> eventQueueTimeUpdater;
|
||||
private final Consumer<Long> eventQueueProcessingTimeUpdater;
|
||||
private final TimeRatio avgIdleTimeRatio;
|
||||
|
||||
private final AtomicLong timedOutHeartbeats = new AtomicLong(0);
|
||||
private final AtomicLong operationsStarted = new AtomicLong(0);
|
||||
|
@ -109,6 +114,7 @@ public class QuorumControllerMetrics implements AutoCloseable {
|
|||
this.eventQueueTimeUpdater = newHistogram(EVENT_QUEUE_TIME_MS, true);
|
||||
this.eventQueueProcessingTimeUpdater = newHistogram(EVENT_QUEUE_PROCESSING_TIME_MS, true);
|
||||
this.sessionTimeoutMs = sessionTimeoutMs;
|
||||
this.avgIdleTimeRatio = new TimeRatio(1);
|
||||
registry.ifPresent(r -> r.newGauge(LAST_APPLIED_RECORD_OFFSET, new Gauge<Long>() {
|
||||
@Override
|
||||
public Long value() {
|
||||
|
@ -157,6 +163,20 @@ public class QuorumControllerMetrics implements AutoCloseable {
|
|||
return newActiveControllers();
|
||||
}
|
||||
}));
|
||||
registry.ifPresent(r -> r.newGauge(AVERAGE_IDLE_RATIO, new Gauge<Double>() {
|
||||
@Override
|
||||
public Double value() {
|
||||
synchronized (avgIdleTimeRatio) {
|
||||
return avgIdleTimeRatio.measure();
|
||||
}
|
||||
}
|
||||
}));
|
||||
}
|
||||
|
||||
public void updateIdleTime(long idleDurationMs) {
|
||||
synchronized (avgIdleTimeRatio) {
|
||||
avgIdleTimeRatio.record((double) idleDurationMs, time.milliseconds());
|
||||
}
|
||||
}
|
||||
|
||||
public void addTimeSinceLastHeartbeatMetric(int brokerId) {
|
||||
|
@ -291,7 +311,8 @@ public class QuorumControllerMetrics implements AutoCloseable {
|
|||
TIMED_OUT_BROKER_HEARTBEAT_COUNT,
|
||||
EVENT_QUEUE_OPERATIONS_STARTED_COUNT,
|
||||
EVENT_QUEUE_OPERATIONS_TIMED_OUT_COUNT,
|
||||
NEW_ACTIVE_CONTROLLERS_COUNT
|
||||
NEW_ACTIVE_CONTROLLERS_COUNT,
|
||||
AVERAGE_IDLE_RATIO
|
||||
).forEach(r::removeMetric));
|
||||
removeTimeSinceLastHeartbeatMetrics();
|
||||
}
|
||||
|
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package kafka.server;
|
||||
package org.apache.kafka.metadata;
|
||||
|
||||
import org.apache.kafka.image.MetadataDelta;
|
||||
import org.apache.kafka.image.MetadataImage;
|
||||
|
@ -24,18 +24,20 @@ import org.apache.kafka.image.publisher.MetadataPublisher;
|
|||
import org.apache.kafka.server.common.MetadataVersion;
|
||||
import org.apache.kafka.server.fault.FaultHandler;
|
||||
|
||||
import java.util.function.Supplier;
|
||||
|
||||
public class MetadataVersionConfigValidator implements MetadataPublisher {
|
||||
private final String name;
|
||||
private final KafkaConfig config;
|
||||
private final Supplier<Boolean> hasMultiLogDirs;
|
||||
private final FaultHandler faultHandler;
|
||||
|
||||
public MetadataVersionConfigValidator(
|
||||
KafkaConfig config,
|
||||
FaultHandler faultHandler
|
||||
int id,
|
||||
Supplier<Boolean> hasMultiLogDirs,
|
||||
FaultHandler faultHandler
|
||||
) {
|
||||
int id = config.brokerId();
|
||||
this.name = "MetadataVersionPublisher(id=" + id + ")";
|
||||
this.config = config;
|
||||
this.hasMultiLogDirs = hasMultiLogDirs;
|
||||
this.faultHandler = faultHandler;
|
||||
}
|
||||
|
||||
|
@ -46,9 +48,9 @@ public class MetadataVersionConfigValidator implements MetadataPublisher {
|
|||
|
||||
@Override
|
||||
public void onMetadataUpdate(
|
||||
MetadataDelta delta,
|
||||
MetadataImage newImage,
|
||||
LoaderManifest manifest
|
||||
MetadataDelta delta,
|
||||
MetadataImage newImage,
|
||||
LoaderManifest manifest
|
||||
) {
|
||||
if (delta.featuresDelta() != null) {
|
||||
if (delta.metadataVersionChanged().isPresent()) {
|
||||
|
@ -57,13 +59,22 @@ public class MetadataVersionConfigValidator implements MetadataPublisher {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Validate some configurations for the new MetadataVersion. A new MetadataVersion can take place when
|
||||
* a FeatureLevelRecord for "metadata.version" is read from the cluster metadata.
|
||||
*/
|
||||
@SuppressWarnings("ThrowableNotThrown")
|
||||
private void onMetadataVersionChanged(MetadataVersion metadataVersion) {
|
||||
try {
|
||||
this.config.validateWithMetadataVersion(metadataVersion);
|
||||
} catch (Throwable t) {
|
||||
if (this.hasMultiLogDirs.get() && !metadataVersion.isDirectoryAssignmentSupported()) {
|
||||
String errorMsg = String.format(
|
||||
"Multiple log directories (aka JBOD) are not supported in the current MetadataVersion %s. Need %s or higher",
|
||||
metadataVersion, MetadataVersion.IBP_3_7_IV2
|
||||
);
|
||||
|
||||
this.faultHandler.handleFault(
|
||||
"Broker configuration does not support the cluster MetadataVersion", t);
|
||||
"Broker configuration does not support the cluster MetadataVersion",
|
||||
new IllegalArgumentException(errorMsg)
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -822,6 +822,48 @@ public class PartitionChangeBuilderTest {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEligibleLeaderReplicas_lastKnownElrShouldBePopulatedWhenNoLeader() {
|
||||
PartitionRegistration partition = new PartitionRegistration.Builder()
|
||||
.setReplicas(new int[] {1, 2, 3})
|
||||
.setDirectories(new Uuid[] {
|
||||
DirectoryId.UNASSIGNED,
|
||||
DirectoryId.UNASSIGNED,
|
||||
DirectoryId.UNASSIGNED
|
||||
})
|
||||
.setIsr(new int[] {1})
|
||||
.setElr(new int[] {2})
|
||||
.setLeader(1)
|
||||
.setLeaderRecoveryState(LeaderRecoveryState.RECOVERED)
|
||||
.setLeaderEpoch(100)
|
||||
.setPartitionEpoch(200)
|
||||
.build();
|
||||
|
||||
short version = 2; // ELR supported
|
||||
Uuid topicId = Uuid.fromString("FbrrdcfiR-KC2CPSTHaJrg");
|
||||
|
||||
// No replica is acceptable as leader, so election yields NO_LEADER.
|
||||
// We intentionally do not change target ISR so record.isr remains null.
|
||||
PartitionChangeBuilder builder = new PartitionChangeBuilder(partition, topicId, 0, r -> false,
|
||||
metadataVersionForPartitionChangeRecordVersion(version), 3)
|
||||
.setElection(Election.PREFERRED)
|
||||
.setEligibleLeaderReplicasEnabled(isElrEnabled(version))
|
||||
.setDefaultDirProvider(DEFAULT_DIR_PROVIDER)
|
||||
.setUseLastKnownLeaderInBalancedRecovery(true);
|
||||
|
||||
ApiMessageAndVersion change = builder.build().get();
|
||||
PartitionChangeRecord record = (PartitionChangeRecord) change.message();
|
||||
|
||||
assertEquals(NO_LEADER, record.leader());
|
||||
// There is no ISR update if we do not perform the leader verification on the ISR members.
|
||||
assertNull(record.isr(), record.toString());
|
||||
assertEquals(1, record.lastKnownElr().size(), record.toString());
|
||||
assertEquals(1, record.lastKnownElr().get(0), record.toString());
|
||||
partition = partition.merge((PartitionChangeRecord) builder.build().get().message());
|
||||
assertArrayEquals(new int[] {1}, partition.lastKnownElr);
|
||||
}
|
||||
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("partitionChangeRecordVersions")
|
||||
public void testEligibleLeaderReplicas_IsrExpandAboveMinISR(short version) {
|
||||
|
|
|
@ -45,6 +45,7 @@ public class QuorumControllerMetricsTest {
|
|||
Set<String> expected = Set.of(
|
||||
"kafka.controller:type=ControllerEventManager,name=EventQueueProcessingTimeMs",
|
||||
"kafka.controller:type=ControllerEventManager,name=EventQueueTimeMs",
|
||||
"kafka.controller:type=ControllerEventManager,name=AvgIdleRatio",
|
||||
"kafka.controller:type=KafkaController,name=ActiveControllerCount",
|
||||
"kafka.controller:type=KafkaController,name=EventQueueOperationsStartedCount",
|
||||
"kafka.controller:type=KafkaController,name=EventQueueOperationsTimedOutCount",
|
||||
|
@ -189,6 +190,35 @@ public class QuorumControllerMetricsTest {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAvgIdleRatio() {
|
||||
final double delta = 0.001;
|
||||
MetricsRegistry registry = new MetricsRegistry();
|
||||
MockTime time = new MockTime();
|
||||
try (QuorumControllerMetrics metrics = new QuorumControllerMetrics(Optional.of(registry), time, 9000)) {
|
||||
Gauge<Double> avgIdleRatio = (Gauge<Double>) registry.allMetrics().get(metricName("ControllerEventManager", "AvgIdleRatio"));
|
||||
|
||||
// No idle time recorded yet; returns default ratio of 1.0
|
||||
assertEquals(1.0, avgIdleRatio.value(), delta);
|
||||
|
||||
// First recording is dropped to establish the interval start time
|
||||
// This is because TimeRatio needs an initial timestamp to measure intervals from
|
||||
metrics.updateIdleTime(10);
|
||||
time.sleep(40);
|
||||
metrics.updateIdleTime(20);
|
||||
// avgIdleRatio = (20ms idle) / (40ms interval) = 0.5
|
||||
assertEquals(0.5, avgIdleRatio.value(), delta);
|
||||
|
||||
time.sleep(20);
|
||||
metrics.updateIdleTime(1);
|
||||
// avgIdleRatio = (1ms idle) / (20ms interval) = 0.05
|
||||
assertEquals(0.05, avgIdleRatio.value(), delta);
|
||||
|
||||
} finally {
|
||||
registry.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
private static void assertMetricHistogram(MetricsRegistry registry, MetricName metricName, long count, double sum) {
|
||||
Histogram histogram = (Histogram) registry.allMetrics().get(metricName);
|
||||
|
||||
|
|
|
@ -0,0 +1,140 @@
|
|||
/*
|
||||
* 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.metadata;
|
||||
|
||||
import org.apache.kafka.common.metadata.FeatureLevelRecord;
|
||||
import org.apache.kafka.image.MetadataDelta;
|
||||
import org.apache.kafka.image.MetadataImage;
|
||||
import org.apache.kafka.image.MetadataProvenance;
|
||||
import org.apache.kafka.image.loader.LogDeltaManifest;
|
||||
import org.apache.kafka.raft.LeaderAndEpoch;
|
||||
import org.apache.kafka.server.common.MetadataVersion;
|
||||
import org.apache.kafka.server.fault.FaultHandler;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.function.Supplier;
|
||||
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.ArgumentMatchers.eq;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.times;
|
||||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.verifyNoMoreInteractions;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
@SuppressWarnings({"unchecked", "ThrowableNotThrown"})
|
||||
public class MetadataVersionConfigValidatorTest {
|
||||
|
||||
private static final LogDeltaManifest TEST_MANIFEST = LogDeltaManifest.newBuilder()
|
||||
.provenance(MetadataProvenance.EMPTY)
|
||||
.leaderAndEpoch(LeaderAndEpoch.UNKNOWN)
|
||||
.numBatches(1)
|
||||
.elapsedNs(90)
|
||||
.numBytes(88)
|
||||
.build();
|
||||
public static final MetadataProvenance TEST_PROVENANCE =
|
||||
new MetadataProvenance(50, 3, 8000, true);
|
||||
|
||||
void executeMetadataUpdate(
|
||||
MetadataVersion metadataVersion,
|
||||
Supplier<Boolean> multiLogDirSupplier,
|
||||
FaultHandler faultHandler
|
||||
) throws Exception {
|
||||
try (MetadataVersionConfigValidator validator = new MetadataVersionConfigValidator(0, multiLogDirSupplier, faultHandler)) {
|
||||
MetadataDelta delta = new MetadataDelta.Builder()
|
||||
.setImage(MetadataImage.EMPTY)
|
||||
.build();
|
||||
if (metadataVersion != null) {
|
||||
delta.replay(new FeatureLevelRecord().
|
||||
setName(MetadataVersion.FEATURE_NAME).
|
||||
setFeatureLevel(metadataVersion.featureLevel()));
|
||||
}
|
||||
MetadataImage image = delta.apply(TEST_PROVENANCE);
|
||||
|
||||
validator.onMetadataUpdate(delta, image, TEST_MANIFEST);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testValidatesConfigOnMetadataChange() throws Exception {
|
||||
MetadataVersion metadataVersion = MetadataVersion.IBP_3_7_IV2;
|
||||
FaultHandler faultHandler = mock(FaultHandler.class);
|
||||
Supplier<Boolean> multiLogDirSupplier = mock(Supplier.class);
|
||||
when(multiLogDirSupplier.get()).thenReturn(false);
|
||||
|
||||
executeMetadataUpdate(metadataVersion, multiLogDirSupplier, faultHandler);
|
||||
|
||||
verify(multiLogDirSupplier, times(1)).get();
|
||||
verifyNoMoreInteractions(faultHandler);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testInvokesFaultHandlerOnException() throws Exception {
|
||||
MetadataVersion metadataVersion = MetadataVersion.IBP_3_7_IV1;
|
||||
Supplier<Boolean> multiLogDirSupplier = mock(Supplier.class);
|
||||
FaultHandler faultHandler = mock(FaultHandler.class);
|
||||
|
||||
when(multiLogDirSupplier.get()).thenReturn(true);
|
||||
|
||||
executeMetadataUpdate(metadataVersion, multiLogDirSupplier, faultHandler);
|
||||
|
||||
verify(multiLogDirSupplier, times(1)).get();
|
||||
verify(faultHandler, times(1)).handleFault(
|
||||
eq("Broker configuration does not support the cluster MetadataVersion"),
|
||||
any(IllegalArgumentException.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testValidateWithMetadataVersionJbodSupport() throws Exception {
|
||||
FaultHandler faultHandler = mock(FaultHandler.class);
|
||||
validate(MetadataVersion.IBP_3_6_IV2, false, faultHandler);
|
||||
verifyNoMoreInteractions(faultHandler);
|
||||
|
||||
faultHandler = mock(FaultHandler.class);
|
||||
validate(MetadataVersion.IBP_3_7_IV0, false, faultHandler);
|
||||
verifyNoMoreInteractions(faultHandler);
|
||||
|
||||
faultHandler = mock(FaultHandler.class);
|
||||
validate(MetadataVersion.IBP_3_7_IV2, false, faultHandler);
|
||||
verifyNoMoreInteractions(faultHandler);
|
||||
|
||||
faultHandler = mock(FaultHandler.class);
|
||||
validate(MetadataVersion.IBP_3_6_IV2, true, faultHandler);
|
||||
verify(faultHandler, times(1)).handleFault(
|
||||
eq("Broker configuration does not support the cluster MetadataVersion"),
|
||||
any(IllegalArgumentException.class));
|
||||
|
||||
faultHandler = mock(FaultHandler.class);
|
||||
validate(MetadataVersion.IBP_3_7_IV0, true, faultHandler);
|
||||
verify(faultHandler, times(1)).handleFault(
|
||||
eq("Broker configuration does not support the cluster MetadataVersion"),
|
||||
any(IllegalArgumentException.class));
|
||||
|
||||
faultHandler = mock(FaultHandler.class);
|
||||
validate(MetadataVersion.IBP_3_7_IV2, true, faultHandler);
|
||||
verifyNoMoreInteractions(faultHandler);
|
||||
}
|
||||
|
||||
private void validate(MetadataVersion metadataVersion, boolean jbodConfig, FaultHandler faultHandler)
|
||||
throws Exception {
|
||||
Supplier<Boolean> multiLogDirSupplier = () -> jbodConfig;
|
||||
|
||||
executeMetadataUpdate(metadataVersion, multiLogDirSupplier, faultHandler);
|
||||
}
|
||||
}
|
|
@ -29,6 +29,7 @@ import org.apache.kafka.raft.LogOffsetMetadata;
|
|||
import org.apache.kafka.raft.QuorumState;
|
||||
import org.apache.kafka.raft.ReplicaKey;
|
||||
import org.apache.kafka.server.common.OffsetAndEpoch;
|
||||
import org.apache.kafka.server.metrics.TimeRatio;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.OptionalLong;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -33,6 +33,7 @@ import java.util.TreeMap;
|
|||
import java.util.concurrent.RejectedExecutionException;
|
||||
import java.util.concurrent.locks.Condition;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
import java.util.function.Consumer;
|
||||
import java.util.function.Function;
|
||||
import java.util.function.UnaryOperator;
|
||||
|
||||
|
@ -278,22 +279,22 @@ public final class KafkaEventQueue implements EventQueue {
|
|||
remove(toRun);
|
||||
continue;
|
||||
}
|
||||
if (awaitNs == Long.MAX_VALUE) {
|
||||
try {
|
||||
|
||||
long startIdleMs = time.milliseconds();
|
||||
try {
|
||||
if (awaitNs == Long.MAX_VALUE) {
|
||||
cond.await();
|
||||
} catch (InterruptedException e) {
|
||||
log.warn("Interrupted while waiting for a new event. " +
|
||||
"Shutting down event queue");
|
||||
interrupted = true;
|
||||
}
|
||||
} else {
|
||||
try {
|
||||
} else {
|
||||
cond.awaitNanos(awaitNs);
|
||||
} catch (InterruptedException e) {
|
||||
log.warn("Interrupted while waiting for a deferred event. " +
|
||||
"Shutting down event queue");
|
||||
interrupted = true;
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
log.warn(
|
||||
"Interrupted while waiting for a {} event. Shutting down event queue",
|
||||
(awaitNs == Long.MAX_VALUE) ? "new" : "deferred"
|
||||
);
|
||||
interrupted = true;
|
||||
} finally {
|
||||
idleTimeCallback.accept(Math.max(time.milliseconds() - startIdleMs, 0));
|
||||
}
|
||||
} finally {
|
||||
lock.unlock();
|
||||
|
@ -440,12 +441,18 @@ public final class KafkaEventQueue implements EventQueue {
|
|||
*/
|
||||
private boolean interrupted;
|
||||
|
||||
/**
|
||||
* Optional callback for queue idle time tracking.
|
||||
*/
|
||||
private final Consumer<Long> idleTimeCallback;
|
||||
|
||||
|
||||
public KafkaEventQueue(
|
||||
Time time,
|
||||
LogContext logContext,
|
||||
String threadNamePrefix
|
||||
) {
|
||||
this(time, logContext, threadNamePrefix, VoidEvent.INSTANCE);
|
||||
this(time, logContext, threadNamePrefix, VoidEvent.INSTANCE, __ -> { });
|
||||
}
|
||||
|
||||
public KafkaEventQueue(
|
||||
|
@ -453,6 +460,16 @@ public final class KafkaEventQueue implements EventQueue {
|
|||
LogContext logContext,
|
||||
String threadNamePrefix,
|
||||
Event cleanupEvent
|
||||
) {
|
||||
this(time, logContext, threadNamePrefix, cleanupEvent, __ -> { });
|
||||
}
|
||||
|
||||
public KafkaEventQueue(
|
||||
Time time,
|
||||
LogContext logContext,
|
||||
String threadNamePrefix,
|
||||
Event cleanupEvent,
|
||||
Consumer<Long> idleTimeCallback
|
||||
) {
|
||||
this.time = time;
|
||||
this.cleanupEvent = Objects.requireNonNull(cleanupEvent);
|
||||
|
@ -463,6 +480,7 @@ public final class KafkaEventQueue implements EventQueue {
|
|||
this.eventHandler, false);
|
||||
this.shuttingDown = false;
|
||||
this.interrupted = false;
|
||||
this.idleTimeCallback = Objects.requireNonNull(idleTimeCallback);
|
||||
this.eventHandlerThread.start();
|
||||
}
|
||||
|
||||
|
|
|
@ -14,7 +14,7 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.kafka.raft.internals;
|
||||
package org.apache.kafka.server.metrics;
|
||||
|
||||
import org.apache.kafka.common.metrics.MeasurableStat;
|
||||
import org.apache.kafka.common.metrics.MetricConfig;
|
||||
|
@ -46,11 +46,26 @@ public class TimeRatio implements MeasurableStat {
|
|||
|
||||
@Override
|
||||
public double measure(MetricConfig config, long currentTimestampMs) {
|
||||
return measure();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void record(MetricConfig config, double value, long currentTimestampMs) {
|
||||
record(value, currentTimestampMs);
|
||||
}
|
||||
|
||||
/**
|
||||
* Measures the ratio of recorded duration to the interval duration
|
||||
* since the last measurement.
|
||||
*
|
||||
* @return The measured ratio value between 0.0 and 1.0
|
||||
*/
|
||||
public double measure() {
|
||||
if (lastRecordedTimestampMs < 0) {
|
||||
// Return the default value if no recordings have been captured.
|
||||
return defaultRatio;
|
||||
} else {
|
||||
// We measure the ratio over the
|
||||
// We measure the ratio over the interval
|
||||
double intervalDurationMs = Math.max(lastRecordedTimestampMs - intervalStartTimestampMs, 0);
|
||||
final double ratio;
|
||||
if (intervalDurationMs == 0) {
|
||||
|
@ -61,15 +76,20 @@ public class TimeRatio implements MeasurableStat {
|
|||
ratio = totalRecordedDurationMs / intervalDurationMs;
|
||||
}
|
||||
|
||||
// The next interval begins at the
|
||||
// The next interval begins at the last recorded timestamp
|
||||
intervalStartTimestampMs = lastRecordedTimestampMs;
|
||||
totalRecordedDurationMs = 0;
|
||||
return ratio;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void record(MetricConfig config, double value, long currentTimestampMs) {
|
||||
/**
|
||||
* Records a duration value at the specified timestamp.
|
||||
*
|
||||
* @param value The duration value to record
|
||||
* @param currentTimestampMs The current timestamp in milliseconds
|
||||
*/
|
||||
public void record(double value, long currentTimestampMs) {
|
||||
if (intervalStartTimestampMs < 0) {
|
||||
// Discard the initial value since the value occurred prior to the interval start
|
||||
intervalStartTimestampMs = currentTimestampMs;
|
||||
|
@ -78,5 +98,4 @@ public class TimeRatio implements MeasurableStat {
|
|||
lastRecordedTimestampMs = currentTimestampMs;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -424,4 +424,48 @@ public class KafkaEventQueueTest {
|
|||
assertEquals(InterruptedException.class, ieTrapper2.exception.get().getClass());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIdleTimeCallback() throws Exception {
|
||||
MockTime time = new MockTime();
|
||||
AtomicLong lastIdleTimeMs = new AtomicLong(0);
|
||||
|
||||
try (KafkaEventQueue queue = new KafkaEventQueue(
|
||||
time,
|
||||
logContext,
|
||||
"testIdleTimeCallback",
|
||||
EventQueue.VoidEvent.INSTANCE,
|
||||
lastIdleTimeMs::set)) {
|
||||
time.sleep(2);
|
||||
assertEquals(0, lastIdleTimeMs.get(), "Last idle time should be 0ms");
|
||||
|
||||
// Test 1: Two events with a wait in between using FutureEvent
|
||||
CompletableFuture<String> event1 = new CompletableFuture<>();
|
||||
queue.append(new FutureEvent<>(event1, () -> {
|
||||
time.sleep(1);
|
||||
return "event1-processed";
|
||||
}));
|
||||
assertEquals("event1-processed", event1.get());
|
||||
|
||||
long waitTime5Ms = 5;
|
||||
time.sleep(waitTime5Ms);
|
||||
CompletableFuture<String> event2 = new CompletableFuture<>();
|
||||
queue.append(new FutureEvent<>(event2, () -> {
|
||||
time.sleep(1);
|
||||
return "event2-processed";
|
||||
}));
|
||||
assertEquals("event2-processed", event2.get());
|
||||
assertEquals(waitTime5Ms, lastIdleTimeMs.get(), "Idle time should be " + waitTime5Ms + "ms, was: " + lastIdleTimeMs.get());
|
||||
|
||||
// Test 2: Deferred event
|
||||
long waitTime2Ms = 2;
|
||||
CompletableFuture<Void> deferredEvent2 = new CompletableFuture<>();
|
||||
queue.scheduleDeferred("deferred2",
|
||||
__ -> OptionalLong.of(time.nanoseconds() + TimeUnit.MILLISECONDS.toNanos(waitTime2Ms)),
|
||||
() -> deferredEvent2.complete(null));
|
||||
time.sleep(waitTime2Ms);
|
||||
deferredEvent2.get();
|
||||
assertEquals(waitTime2Ms, lastIdleTimeMs.get(), "Idle time should be " + waitTime2Ms + "ms, was: " + lastIdleTimeMs.get());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -14,7 +14,7 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.kafka.raft.internals;
|
||||
package org.apache.kafka.server.metrics;
|
||||
|
||||
import org.apache.kafka.common.metrics.MetricConfig;
|
||||
import org.apache.kafka.common.utils.MockTime;
|
|
@ -71,11 +71,15 @@ public final class AssignmentsManager {
|
|||
*/
|
||||
static final long MIN_NOISY_FAILURE_INTERVAL_NS = TimeUnit.MINUTES.toNanos(2);
|
||||
|
||||
@Deprecated(since = "4.2")
|
||||
static final MetricName DEPRECATED_QUEUED_REPLICA_TO_DIR_ASSIGNMENTS_METRIC =
|
||||
KafkaYammerMetrics.getMetricName("org.apache.kafka.server", "AssignmentsManager", "QueuedReplicaToDirAssignments");
|
||||
|
||||
/**
|
||||
* The metric reflecting the number of pending assignments.
|
||||
*/
|
||||
static final MetricName QUEUED_REPLICA_TO_DIR_ASSIGNMENTS_METRIC =
|
||||
metricName("QueuedReplicaToDirAssignments");
|
||||
KafkaYammerMetrics.getMetricName("kafka.server", "AssignmentsManager", "QueuedReplicaToDirAssignments");
|
||||
|
||||
/**
|
||||
* The event at which we send assignments, if appropriate.
|
||||
|
@ -142,10 +146,6 @@ public final class AssignmentsManager {
|
|||
*/
|
||||
private final KafkaEventQueue eventQueue;
|
||||
|
||||
static MetricName metricName(String name) {
|
||||
return KafkaYammerMetrics.getMetricName("org.apache.kafka.server", "AssignmentsManager", name);
|
||||
}
|
||||
|
||||
public AssignmentsManager(
|
||||
Time time,
|
||||
NodeToControllerChannelManager channelManager,
|
||||
|
@ -182,12 +182,18 @@ public final class AssignmentsManager {
|
|||
this.ready = new ConcurrentHashMap<>();
|
||||
this.inflight = Map.of();
|
||||
this.metricsRegistry = metricsRegistry;
|
||||
this.metricsRegistry.newGauge(DEPRECATED_QUEUED_REPLICA_TO_DIR_ASSIGNMENTS_METRIC, new Gauge<Integer>() {
|
||||
@Override
|
||||
public Integer value() {
|
||||
return numPending();
|
||||
}
|
||||
});
|
||||
this.metricsRegistry.newGauge(QUEUED_REPLICA_TO_DIR_ASSIGNMENTS_METRIC, new Gauge<Integer>() {
|
||||
@Override
|
||||
public Integer value() {
|
||||
return numPending();
|
||||
}
|
||||
});
|
||||
@Override
|
||||
public Integer value() {
|
||||
return numPending();
|
||||
}
|
||||
});
|
||||
this.previousGlobalFailures = 0;
|
||||
this.eventQueue = new KafkaEventQueue(time,
|
||||
new LogContext("[AssignmentsManager id=" + nodeId + "]"),
|
||||
|
@ -248,6 +254,7 @@ public final class AssignmentsManager {
|
|||
log.error("Unexpected exception shutting down NodeToControllerChannelManager", e);
|
||||
}
|
||||
try {
|
||||
metricsRegistry.removeMetric(DEPRECATED_QUEUED_REPLICA_TO_DIR_ASSIGNMENTS_METRIC);
|
||||
metricsRegistry.removeMetric(QUEUED_REPLICA_TO_DIR_ASSIGNMENTS_METRIC);
|
||||
} catch (Exception e) {
|
||||
log.error("Unexpected exception removing metrics.", e);
|
||||
|
|
|
@ -250,6 +250,13 @@ public class AssignmentsManagerTest {
|
|||
return queuedReplicaToDirAssignments.value();
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked") // do not warn about Gauge typecast.
|
||||
int deprecatedQueuedReplicaToDirAssignments() {
|
||||
Gauge<Integer> queuedReplicaToDirAssignments =
|
||||
(Gauge<Integer>) findMetric(AssignmentsManager.DEPRECATED_QUEUED_REPLICA_TO_DIR_ASSIGNMENTS_METRIC);
|
||||
return queuedReplicaToDirAssignments.value();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws Exception {
|
||||
try {
|
||||
|
@ -279,10 +286,12 @@ public class AssignmentsManagerTest {
|
|||
public void testSuccessfulAssignment() throws Exception {
|
||||
try (TestEnv testEnv = new TestEnv()) {
|
||||
assertEquals(0, testEnv.queuedReplicaToDirAssignments());
|
||||
assertEquals(0, testEnv.deprecatedQueuedReplicaToDirAssignments());
|
||||
testEnv.onAssignment(new TopicIdPartition(TOPIC_1, 0), DIR_1);
|
||||
TestUtils.retryOnExceptionWithTimeout(60_000, () -> {
|
||||
assertEquals(1, testEnv.assignmentsManager.numPending());
|
||||
assertEquals(1, testEnv.queuedReplicaToDirAssignments());
|
||||
assertEquals(1, testEnv.deprecatedQueuedReplicaToDirAssignments());
|
||||
});
|
||||
assertEquals(0, testEnv.assignmentsManager.previousGlobalFailures());
|
||||
assertEquals(1, testEnv.assignmentsManager.numInFlight());
|
||||
|
@ -290,6 +299,7 @@ public class AssignmentsManagerTest {
|
|||
TestUtils.retryOnExceptionWithTimeout(60_000, () -> {
|
||||
assertEquals(0, testEnv.assignmentsManager.numPending());
|
||||
assertEquals(0, testEnv.queuedReplicaToDirAssignments());
|
||||
assertEquals(0, testEnv.deprecatedQueuedReplicaToDirAssignments());
|
||||
assertEquals(1, testEnv.success(new TopicIdPartition(TOPIC_1, 0)));
|
||||
});
|
||||
assertEquals(0, testEnv.assignmentsManager.previousGlobalFailures());
|
||||
|
|
|
@ -90,10 +90,16 @@ public class RemoteStorageMetrics {
|
|||
"kafka.server", "BrokerTopicMetrics", REMOTE_DELETE_LAG_SEGMENTS);
|
||||
public static final MetricName REMOTE_LOG_MANAGER_TASKS_AVG_IDLE_PERCENT_METRIC = getMetricName(
|
||||
"kafka.log.remote", "RemoteLogManager", REMOTE_LOG_MANAGER_TASKS_AVG_IDLE_PERCENT);
|
||||
public static final MetricName REMOTE_LOG_READER_TASK_QUEUE_SIZE_METRIC = getMetricName(
|
||||
@Deprecated(since = "4.2")
|
||||
public static final MetricName DEPRECATE_REMOTE_LOG_READER_TASK_QUEUE_SIZE_METRIC = getMetricName(
|
||||
"org.apache.kafka.storage.internals.log", "RemoteStorageThreadPool", REMOTE_LOG_READER_TASK_QUEUE_SIZE);
|
||||
public static final MetricName REMOTE_LOG_READER_AVG_IDLE_PERCENT_METRIC = getMetricName(
|
||||
@Deprecated(since = "4.2")
|
||||
public static final MetricName DEPRECATE_REMOTE_LOG_READER_AVG_IDLE_PERCENT_METRIC = getMetricName(
|
||||
"org.apache.kafka.storage.internals.log", "RemoteStorageThreadPool", REMOTE_LOG_READER_AVG_IDLE_PERCENT);
|
||||
public static final MetricName REMOTE_LOG_READER_TASK_QUEUE_SIZE_METRIC = getMetricName(
|
||||
"kafka.log.remote", "RemoteStorageThreadPool", REMOTE_LOG_READER_TASK_QUEUE_SIZE);
|
||||
public static final MetricName REMOTE_LOG_READER_AVG_IDLE_PERCENT_METRIC = getMetricName(
|
||||
"kafka.log.remote", "RemoteStorageThreadPool", REMOTE_LOG_READER_AVG_IDLE_PERCENT);
|
||||
public static final MetricName REMOTE_LOG_READER_FETCH_RATE_AND_TIME_METRIC = getMetricName(
|
||||
"kafka.log.remote", "RemoteLogManager", REMOTE_LOG_READER_FETCH_RATE_AND_TIME_MS);
|
||||
|
||||
|
@ -115,6 +121,8 @@ public class RemoteStorageMetrics {
|
|||
metrics.add(REMOTE_DELETE_LAG_BYTES_METRIC);
|
||||
metrics.add(REMOTE_DELETE_LAG_SEGMENTS_METRIC);
|
||||
metrics.add(REMOTE_LOG_MANAGER_TASKS_AVG_IDLE_PERCENT_METRIC);
|
||||
metrics.add(DEPRECATE_REMOTE_LOG_READER_AVG_IDLE_PERCENT_METRIC);
|
||||
metrics.add(DEPRECATE_REMOTE_LOG_READER_TASK_QUEUE_SIZE_METRIC);
|
||||
metrics.add(REMOTE_LOG_READER_AVG_IDLE_PERCENT_METRIC);
|
||||
metrics.add(REMOTE_LOG_READER_TASK_QUEUE_SIZE_METRIC);
|
||||
metrics.add(REMOTE_LOG_METADATA_COUNT_METRIC);
|
||||
|
|
|
@ -18,12 +18,4 @@ package org.apache.kafka.storage.internals.log;
|
|||
|
||||
// Mapping of epoch to the first offset of the subsequent epoch
|
||||
public record EpochEntry(int epoch, long startOffset) {
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "EpochEntry(" +
|
||||
"epoch=" + epoch +
|
||||
", startOffset=" + startOffset +
|
||||
')';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -27,12 +27,4 @@ public record LastRecord(OptionalLong lastDataOffset, short producerEpoch) {
|
|||
public LastRecord {
|
||||
Objects.requireNonNull(lastDataOffset, "lastDataOffset must be non null");
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "LastRecord(" +
|
||||
"lastDataOffset=" + lastDataOffset +
|
||||
", producerEpoch=" + producerEpoch +
|
||||
')';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,13 +26,4 @@ public record LoadedLogOffsets(long logStartOffset, long recoveryPoint, LogOffse
|
|||
this.recoveryPoint = recoveryPoint;
|
||||
this.nextOffsetMetadata = Objects.requireNonNull(nextOffsetMetadata, "nextOffsetMetadata should not be null");
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "LoadedLogOffsets(" +
|
||||
"logStartOffset=" + logStartOffset +
|
||||
", recoveryPoint=" + recoveryPoint +
|
||||
", nextOffsetMetadata=" + nextOffsetMetadata +
|
||||
')';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,14 +23,4 @@ package org.apache.kafka.storage.internals.log;
|
|||
*/
|
||||
public record LogOffsetSnapshot(long logStartOffset, LogOffsetMetadata logEndOffset, LogOffsetMetadata highWatermark,
|
||||
LogOffsetMetadata lastStableOffset) {
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "LogOffsetSnapshot(" +
|
||||
"logStartOffset=" + logStartOffset +
|
||||
", logEndOffset=" + logEndOffset +
|
||||
", highWatermark=" + highWatermark +
|
||||
", lastStableOffset=" + lastStableOffset +
|
||||
')';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,16 +25,4 @@ import java.util.Optional;
|
|||
*/
|
||||
public record LogReadInfo(FetchDataInfo fetchedData, Optional<FetchResponseData.EpochEndOffset> divergingEpoch,
|
||||
long highWatermark, long logStartOffset, long logEndOffset, long lastStableOffset) {
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "LogReadInfo(" +
|
||||
"fetchedData=" + fetchedData +
|
||||
", divergingEpoch=" + divergingEpoch +
|
||||
", highWatermark=" + highWatermark +
|
||||
", logStartOffset=" + logStartOffset +
|
||||
", logEndOffset=" + logEndOffset +
|
||||
", lastStableOffset=" + lastStableOffset +
|
||||
')';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,12 +32,4 @@ public record OffsetPosition(long offset, int position) implements IndexEntry {
|
|||
public long indexValue() {
|
||||
return position;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "OffsetPosition(" +
|
||||
"offset=" + offset +
|
||||
", position=" + position +
|
||||
')';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,15 +22,4 @@ import org.apache.kafka.server.storage.log.FetchIsolation;
|
|||
|
||||
public record RemoteStorageFetchInfo(int fetchMaxBytes, boolean minOneMessage, TopicIdPartition topicIdPartition,
|
||||
FetchRequest.PartitionData fetchInfo, FetchIsolation fetchIsolation) {
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "RemoteStorageFetchInfo{" +
|
||||
"fetchMaxBytes=" + fetchMaxBytes +
|
||||
", minOneMessage=" + minOneMessage +
|
||||
", topicIdPartition=" + topicIdPartition +
|
||||
", fetchInfo=" + fetchInfo +
|
||||
", fetchIsolation=" + fetchIsolation +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
package org.apache.kafka.storage.internals.log;
|
||||
|
||||
import org.apache.kafka.common.utils.ThreadUtils;
|
||||
import org.apache.kafka.server.log.remote.storage.RemoteStorageMetrics;
|
||||
import org.apache.kafka.server.metrics.KafkaMetricsGroup;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
|
@ -32,8 +33,12 @@ import static org.apache.kafka.server.log.remote.storage.RemoteStorageMetrics.RE
|
|||
|
||||
public final class RemoteStorageThreadPool extends ThreadPoolExecutor {
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(RemoteStorageThreadPool.class);
|
||||
private final KafkaMetricsGroup metricsGroup = new KafkaMetricsGroup(this.getClass());
|
||||
@Deprecated(since = "4.2")
|
||||
// This metrics group is used to register deprecated metrics. It will be removed in Kafka 5.0
|
||||
private final KafkaMetricsGroup deprecatedLogMetricsGroup = new KafkaMetricsGroup("org.apache.kafka.storage.internals.log", "RemoteStorageThreadPool");
|
||||
private final KafkaMetricsGroup logRemoteMetricsGroup = new KafkaMetricsGroup("kafka.log.remote", "RemoteStorageThreadPool");
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
public RemoteStorageThreadPool(String threadNamePattern,
|
||||
int numThreads,
|
||||
int maxPendingTasks) {
|
||||
|
@ -45,9 +50,13 @@ public final class RemoteStorageThreadPool extends ThreadPoolExecutor {
|
|||
ThreadUtils.createThreadFactory(threadNamePattern, false,
|
||||
(t, e) -> LOGGER.error("Uncaught exception in thread '{}':", t.getName(), e))
|
||||
);
|
||||
metricsGroup.newGauge(REMOTE_LOG_READER_TASK_QUEUE_SIZE_METRIC.getName(),
|
||||
deprecatedLogMetricsGroup.newGauge(RemoteStorageMetrics.DEPRECATE_REMOTE_LOG_READER_TASK_QUEUE_SIZE_METRIC.getName(),
|
||||
() -> getQueue().size());
|
||||
metricsGroup.newGauge(REMOTE_LOG_READER_AVG_IDLE_PERCENT_METRIC.getName(),
|
||||
deprecatedLogMetricsGroup.newGauge(RemoteStorageMetrics.DEPRECATE_REMOTE_LOG_READER_AVG_IDLE_PERCENT_METRIC.getName(),
|
||||
() -> 1 - (double) getActiveCount() / (double) getCorePoolSize());
|
||||
logRemoteMetricsGroup.newGauge(REMOTE_LOG_READER_TASK_QUEUE_SIZE_METRIC.getName(),
|
||||
() -> getQueue().size());
|
||||
logRemoteMetricsGroup.newGauge(REMOTE_LOG_READER_AVG_IDLE_PERCENT_METRIC.getName(),
|
||||
() -> 1 - (double) getActiveCount() / (double) getCorePoolSize());
|
||||
}
|
||||
|
||||
|
@ -59,6 +68,7 @@ public final class RemoteStorageThreadPool extends ThreadPoolExecutor {
|
|||
}
|
||||
|
||||
public void removeMetrics() {
|
||||
REMOTE_STORAGE_THREAD_POOL_METRICS.forEach(metricsGroup::removeMetric);
|
||||
REMOTE_STORAGE_THREAD_POOL_METRICS.forEach(deprecatedLogMetricsGroup::removeMetric);
|
||||
REMOTE_STORAGE_THREAD_POOL_METRICS.forEach(logRemoteMetricsGroup::removeMetric);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,17 +21,4 @@ package org.apache.kafka.storage.internals.log;
|
|||
*/
|
||||
public record RollParams(long maxSegmentMs, int maxSegmentBytes, long maxTimestampInMessages, long maxOffsetInMessages,
|
||||
int messagesSize, long now) {
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "RollParams(" +
|
||||
"maxSegmentMs=" + maxSegmentMs +
|
||||
", maxSegmentBytes=" + maxSegmentBytes +
|
||||
", maxTimestampInMessages=" + maxTimestampInMessages +
|
||||
", maxOffsetInMessages=" + maxOffsetInMessages +
|
||||
", messagesSize=" + messagesSize +
|
||||
", now=" + now +
|
||||
')';
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -36,11 +36,4 @@ public record TimestampOffset(long timestamp, long offset) implements IndexEntry
|
|||
public long indexValue() {
|
||||
return offset;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.format("TimestampOffset(offset = %d, timestamp = %d)",
|
||||
offset,
|
||||
timestamp);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1934,8 +1934,8 @@ public class UnifiedLog implements AutoCloseable {
|
|||
deleteRetentionSizeBreachedSegments() +
|
||||
deleteRetentionMsBreachedSegments();
|
||||
} else {
|
||||
// If cleanup.policy is empty and remote storage is disabled, we should not delete any local
|
||||
// log segments
|
||||
// If cleanup.policy is empty and remote storage is disabled, we should not delete any local log segments
|
||||
// unless the log start offset advances through deleteRecords
|
||||
return deleteLogStartOffsetBreachedSegments();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,6 +17,9 @@
|
|||
package org.apache.kafka.streams.integration;
|
||||
|
||||
import org.apache.kafka.clients.consumer.ConsumerConfig;
|
||||
import org.apache.kafka.clients.producer.KafkaProducer;
|
||||
import org.apache.kafka.clients.producer.ProducerConfig;
|
||||
import org.apache.kafka.clients.producer.ProducerRecord;
|
||||
import org.apache.kafka.common.errors.TimeoutException;
|
||||
import org.apache.kafka.common.serialization.Serdes;
|
||||
import org.apache.kafka.common.utils.LogCaptureAppender;
|
||||
|
@ -28,6 +31,9 @@ import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThr
|
|||
import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
|
||||
import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
|
||||
import org.apache.kafka.streams.kstream.KStream;
|
||||
import org.apache.kafka.streams.kstream.KTable;
|
||||
import org.apache.kafka.streams.kstream.Materialized;
|
||||
import org.apache.kafka.streams.kstream.Named;
|
||||
import org.apache.kafka.streams.processor.PunctuationType;
|
||||
import org.apache.kafka.streams.processor.api.Processor;
|
||||
import org.apache.kafka.streams.processor.api.ProcessorContext;
|
||||
|
@ -93,6 +99,7 @@ public class AdjustStreamThreadCountTest {
|
|||
|
||||
private final List<KafkaStreams.State> stateTransitionHistory = new ArrayList<>();
|
||||
private static String inputTopic;
|
||||
private static String outputTopic;
|
||||
private static StreamsBuilder builder;
|
||||
private static Properties properties;
|
||||
private static String appId = "";
|
||||
|
@ -103,10 +110,21 @@ public class AdjustStreamThreadCountTest {
|
|||
final String testId = safeUniqueTestName(testInfo);
|
||||
appId = "appId_" + testId;
|
||||
inputTopic = "input" + testId;
|
||||
outputTopic = "output" + testId;
|
||||
IntegrationTestUtils.cleanStateBeforeTest(CLUSTER, inputTopic);
|
||||
|
||||
builder = new StreamsBuilder();
|
||||
builder.stream(inputTopic);
|
||||
// Build a simple stateful topology to exercise concurrency with state stores
|
||||
final KStream<String, String> source = builder.stream(inputTopic);
|
||||
final KTable<String, Long> counts = source
|
||||
.groupByKey()
|
||||
.count(Named.as("counts"), Materialized.as("counts-store"));
|
||||
counts
|
||||
.toStream()
|
||||
.mapValues(Object::toString)
|
||||
.to(outputTopic);
|
||||
|
||||
produceTestRecords(inputTopic, CLUSTER);
|
||||
|
||||
properties = mkObjectProperties(
|
||||
mkMap(
|
||||
|
@ -121,6 +139,21 @@ public class AdjustStreamThreadCountTest {
|
|||
);
|
||||
}
|
||||
|
||||
private void produceTestRecords(final String inputTopic, final EmbeddedKafkaCluster cluster) {
|
||||
final Properties props = new Properties();
|
||||
props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers());
|
||||
props.put(ProducerConfig.CLIENT_ID_CONFIG, "test-client");
|
||||
props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
|
||||
props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
|
||||
try (KafkaProducer<String, String> producer = new KafkaProducer<>(props)) {
|
||||
for (int i = 0; i < 1000; i++) {
|
||||
final String key = "key-" + (i % 50);
|
||||
final String value = "value-" + i;
|
||||
producer.send(new ProducerRecord<>(inputTopic, key, value));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void startStreamsAndWaitForRunning(final KafkaStreams kafkaStreams) throws InterruptedException {
|
||||
kafkaStreams.start();
|
||||
waitForRunning();
|
||||
|
@ -251,7 +284,13 @@ public class AdjustStreamThreadCountTest {
|
|||
assertTrue(latch.await(30, TimeUnit.SECONDS));
|
||||
one.join();
|
||||
two.join();
|
||||
|
||||
waitForCondition(
|
||||
() -> kafkaStreams.metadataForLocalThreads().size() == oldThreadCount &&
|
||||
kafkaStreams.state() == KafkaStreams.State.RUNNING,
|
||||
DEFAULT_DURATION.toMillis(),
|
||||
"Kafka Streams did not stabilize at the expected thread count and RUNNING state."
|
||||
);
|
||||
|
||||
threadMetadata = kafkaStreams.metadataForLocalThreads();
|
||||
assertThat(threadMetadata.size(), equalTo(oldThreadCount));
|
||||
} catch (final AssertionError e) {
|
||||
|
|
|
@ -72,9 +72,10 @@ import org.junit.jupiter.api.AfterAll;
|
|||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Tag;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.Timeout;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.Arguments;
|
||||
import org.junit.jupiter.params.provider.MethodSource;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
@ -169,6 +170,15 @@ public class EosIntegrationTest {
|
|||
|
||||
private String stateTmpDir;
|
||||
|
||||
private static java.util.stream.Stream<Arguments> groupProtocolAndProcessingThreadsParameters() {
|
||||
return java.util.stream.Stream.of(
|
||||
Arguments.of("classic", true),
|
||||
Arguments.of("classic", false),
|
||||
Arguments.of("streams", true),
|
||||
Arguments.of("streams", false)
|
||||
);
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
public void createTopics() throws Exception {
|
||||
applicationId = "appId-" + TEST_NUMBER.getAndIncrement();
|
||||
|
@ -181,16 +191,19 @@ public class EosIntegrationTest {
|
|||
CLUSTER.createTopic(MULTI_PARTITION_INPUT_TOPIC, NUM_TOPIC_PARTITIONS, 1);
|
||||
CLUSTER.createTopic(MULTI_PARTITION_THROUGH_TOPIC, NUM_TOPIC_PARTITIONS, 1);
|
||||
CLUSTER.createTopic(MULTI_PARTITION_OUTPUT_TOPIC, NUM_TOPIC_PARTITIONS, 1);
|
||||
CLUSTER.setGroupStandbyReplicas(applicationId, 1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldBeAbleToRunWithEosEnabled() throws Exception {
|
||||
runSimpleCopyTest(1, SINGLE_PARTITION_INPUT_TOPIC, null, SINGLE_PARTITION_OUTPUT_TOPIC, false);
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {"classic", "streams"})
|
||||
public void shouldBeAbleToRunWithEosEnabled(final String groupProtocol) throws Exception {
|
||||
runSimpleCopyTest(1, SINGLE_PARTITION_INPUT_TOPIC, null, SINGLE_PARTITION_OUTPUT_TOPIC, false, groupProtocol);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldCommitCorrectOffsetIfInputTopicIsTransactional() throws Exception {
|
||||
runSimpleCopyTest(1, SINGLE_PARTITION_INPUT_TOPIC, null, SINGLE_PARTITION_OUTPUT_TOPIC, true);
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {"classic", "streams"})
|
||||
public void shouldCommitCorrectOffsetIfInputTopicIsTransactional(final String groupProtocol) throws Exception {
|
||||
runSimpleCopyTest(1, SINGLE_PARTITION_INPUT_TOPIC, null, SINGLE_PARTITION_OUTPUT_TOPIC, true, groupProtocol);
|
||||
|
||||
try (final Admin adminClient = Admin.create(mkMap(mkEntry(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers())));
|
||||
final Consumer<byte[], byte[]> consumer = new KafkaConsumer<>(mkMap(
|
||||
|
@ -215,36 +228,42 @@ public class EosIntegrationTest {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldBeAbleToRestartAfterClose() throws Exception {
|
||||
runSimpleCopyTest(2, SINGLE_PARTITION_INPUT_TOPIC, null, SINGLE_PARTITION_OUTPUT_TOPIC, false);
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {"classic", "streams"})
|
||||
public void shouldBeAbleToRestartAfterClose(final String groupProtocol) throws Exception {
|
||||
runSimpleCopyTest(2, SINGLE_PARTITION_INPUT_TOPIC, null, SINGLE_PARTITION_OUTPUT_TOPIC, false, groupProtocol);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldBeAbleToCommitToMultiplePartitions() throws Exception {
|
||||
runSimpleCopyTest(1, SINGLE_PARTITION_INPUT_TOPIC, null, MULTI_PARTITION_OUTPUT_TOPIC, false);
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {"classic", "streams"})
|
||||
public void shouldBeAbleToCommitToMultiplePartitions(final String groupProtocol) throws Exception {
|
||||
runSimpleCopyTest(1, SINGLE_PARTITION_INPUT_TOPIC, null, MULTI_PARTITION_OUTPUT_TOPIC, false, groupProtocol);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldBeAbleToCommitMultiplePartitionOffsets() throws Exception {
|
||||
runSimpleCopyTest(1, MULTI_PARTITION_INPUT_TOPIC, null, SINGLE_PARTITION_OUTPUT_TOPIC, false);
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {"classic", "streams"})
|
||||
public void shouldBeAbleToCommitMultiplePartitionOffsets(final String groupProtocol) throws Exception {
|
||||
runSimpleCopyTest(1, MULTI_PARTITION_INPUT_TOPIC, null, SINGLE_PARTITION_OUTPUT_TOPIC, false, groupProtocol);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldBeAbleToRunWithTwoSubtopologies() throws Exception {
|
||||
runSimpleCopyTest(1, SINGLE_PARTITION_INPUT_TOPIC, SINGLE_PARTITION_THROUGH_TOPIC, SINGLE_PARTITION_OUTPUT_TOPIC, false);
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {"classic", "streams"})
|
||||
public void shouldBeAbleToRunWithTwoSubtopologies(final String groupProtocol) throws Exception {
|
||||
runSimpleCopyTest(1, SINGLE_PARTITION_INPUT_TOPIC, SINGLE_PARTITION_THROUGH_TOPIC, SINGLE_PARTITION_OUTPUT_TOPIC, false, groupProtocol);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldBeAbleToRunWithTwoSubtopologiesAndMultiplePartitions() throws Exception {
|
||||
runSimpleCopyTest(1, MULTI_PARTITION_INPUT_TOPIC, MULTI_PARTITION_THROUGH_TOPIC, MULTI_PARTITION_OUTPUT_TOPIC, false);
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {"classic", "streams"})
|
||||
public void shouldBeAbleToRunWithTwoSubtopologiesAndMultiplePartitions(final String groupProtocol) throws Exception {
|
||||
runSimpleCopyTest(1, MULTI_PARTITION_INPUT_TOPIC, MULTI_PARTITION_THROUGH_TOPIC, MULTI_PARTITION_OUTPUT_TOPIC, false, groupProtocol);
|
||||
}
|
||||
|
||||
private void runSimpleCopyTest(final int numberOfRestarts,
|
||||
final String inputTopic,
|
||||
final String throughTopic,
|
||||
final String outputTopic,
|
||||
final boolean inputTopicTransactional) throws Exception {
|
||||
final boolean inputTopicTransactional,
|
||||
final String groupProtocol) throws Exception {
|
||||
final StreamsBuilder builder = new StreamsBuilder();
|
||||
final KStream<Long, Long> input = builder.stream(inputTopic);
|
||||
KStream<Long, Long> output = input;
|
||||
|
@ -263,6 +282,7 @@ public class EosIntegrationTest {
|
|||
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG), "earliest");
|
||||
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), MAX_POLL_INTERVAL_MS - 1);
|
||||
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG), MAX_POLL_INTERVAL_MS);
|
||||
properties.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol);
|
||||
|
||||
for (int i = 0; i < numberOfRestarts; ++i) {
|
||||
final Properties config = StreamsTestUtils.getStreamsConfig(
|
||||
|
@ -326,8 +346,9 @@ public class EosIntegrationTest {
|
|||
return recordsPerKey;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldBeAbleToPerformMultipleTransactions() throws Exception {
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {"classic", "streams"})
|
||||
public void shouldBeAbleToPerformMultipleTransactions(final String groupProtocol) throws Exception {
|
||||
final StreamsBuilder builder = new StreamsBuilder();
|
||||
builder.stream(SINGLE_PARTITION_INPUT_TOPIC).to(SINGLE_PARTITION_OUTPUT_TOPIC);
|
||||
|
||||
|
@ -337,6 +358,7 @@ public class EosIntegrationTest {
|
|||
properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
||||
properties.put(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1000");
|
||||
properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
||||
properties.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol);
|
||||
|
||||
final Properties config = StreamsTestUtils.getStreamsConfig(
|
||||
applicationId,
|
||||
|
@ -374,8 +396,8 @@ public class EosIntegrationTest {
|
|||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void shouldNotViolateEosIfOneTaskFails(final boolean processingThreadsEnabled) throws Exception {
|
||||
@MethodSource("groupProtocolAndProcessingThreadsParameters")
|
||||
public void shouldNotViolateEosIfOneTaskFails(final String groupProtocol, final boolean processingThreadsEnabled) throws Exception {
|
||||
|
||||
// this test writes 10 + 5 + 5 records per partition (running with 2 partitions)
|
||||
// the app is supposed to copy all 40 records into the output topic
|
||||
|
@ -386,7 +408,7 @@ public class EosIntegrationTest {
|
|||
// -> the failure only kills one thread
|
||||
// after fail over, we should read 40 committed records (even if 50 record got written)
|
||||
|
||||
try (final KafkaStreams streams = getKafkaStreams("dummy", false, "appDir", 2, processingThreadsEnabled)) {
|
||||
try (final KafkaStreams streams = getKafkaStreams("dummy", false, "appDir", 2, groupProtocol, processingThreadsEnabled)) {
|
||||
startApplicationAndWaitUntilRunning(streams);
|
||||
|
||||
final List<KeyValue<Long, Long>> committedDataBeforeFailure = prepareData(0L, 10L, 0L, 1L);
|
||||
|
@ -476,8 +498,8 @@ public class EosIntegrationTest {
|
|||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void shouldNotViolateEosIfOneTaskFailsWithState(final boolean processingThreadsEnabled) throws Exception {
|
||||
@MethodSource("groupProtocolAndProcessingThreadsParameters")
|
||||
public void shouldNotViolateEosIfOneTaskFailsWithState(final String groupProtocol, final boolean processingThreadsEnabled) throws Exception {
|
||||
|
||||
// this test updates a store with 10 + 5 + 5 records per partition (running with 2 partitions)
|
||||
// the app is supposed to emit all 40 update records into the output topic
|
||||
|
@ -493,7 +515,7 @@ public class EosIntegrationTest {
|
|||
|
||||
// We need more processing time under "with state" situation, so increasing the max.poll.interval.ms
|
||||
// to avoid unexpected rebalance during test, which will cause unexpected fail over triggered
|
||||
try (final KafkaStreams streams = getKafkaStreams("dummy", true, "appDir", 2, processingThreadsEnabled)) {
|
||||
try (final KafkaStreams streams = getKafkaStreams("dummy", true, "appDir", 2, groupProtocol, processingThreadsEnabled)) {
|
||||
startApplicationAndWaitUntilRunning(streams);
|
||||
|
||||
final List<KeyValue<Long, Long>> committedDataBeforeFailure = prepareData(0L, 10L, 0L, 1L);
|
||||
|
@ -594,8 +616,8 @@ public class EosIntegrationTest {
|
|||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void shouldNotViolateEosIfOneTaskGetsFencedUsingIsolatedAppInstances(final boolean processingThreadsEnabled) throws Exception {
|
||||
@MethodSource("groupProtocolAndProcessingThreadsParameters")
|
||||
public void shouldNotViolateEosIfOneTaskGetsFencedUsingIsolatedAppInstances(final String groupProtocol, final boolean processingThreadsEnabled) throws Exception {
|
||||
// this test writes 10 + 5 + 5 + 10 records per partition (running with 2 partitions)
|
||||
// the app is supposed to copy all 60 records into the output topic
|
||||
//
|
||||
|
@ -607,10 +629,9 @@ public class EosIntegrationTest {
|
|||
//
|
||||
// afterward, the "stalling" thread resumes, and another rebalance should get triggered
|
||||
// we write the remaining 20 records and verify to read 60 result records
|
||||
|
||||
try (
|
||||
final KafkaStreams streams1 = getKafkaStreams("streams1", false, "appDir1", 1, processingThreadsEnabled);
|
||||
final KafkaStreams streams2 = getKafkaStreams("streams2", false, "appDir2", 1, processingThreadsEnabled)
|
||||
final KafkaStreams streams1 = getKafkaStreams("streams1", false, "appDir1", 1, groupProtocol, processingThreadsEnabled);
|
||||
final KafkaStreams streams2 = getKafkaStreams("streams2", false, "appDir2", 1, groupProtocol, processingThreadsEnabled)
|
||||
) {
|
||||
startApplicationAndWaitUntilRunning(streams1);
|
||||
startApplicationAndWaitUntilRunning(streams2);
|
||||
|
@ -667,13 +688,10 @@ public class EosIntegrationTest {
|
|||
"Expected a host to start stalling"
|
||||
);
|
||||
final String observedStallingHost = stallingHost.get();
|
||||
final KafkaStreams stallingInstance;
|
||||
final KafkaStreams remainingInstance;
|
||||
if ("streams1".equals(observedStallingHost)) {
|
||||
stallingInstance = streams1;
|
||||
remainingInstance = streams2;
|
||||
} else if ("streams2".equals(observedStallingHost)) {
|
||||
stallingInstance = streams2;
|
||||
remainingInstance = streams1;
|
||||
} else {
|
||||
throw new IllegalArgumentException("unexpected host name: " + observedStallingHost);
|
||||
|
@ -683,8 +701,7 @@ public class EosIntegrationTest {
|
|||
// the assignment is. We only really care that the remaining instance only sees one host
|
||||
// that owns both partitions.
|
||||
waitForCondition(
|
||||
() -> stallingInstance.metadataForAllStreamsClients().size() == 2
|
||||
&& remainingInstance.metadataForAllStreamsClients().size() == 1
|
||||
() -> remainingInstance.metadataForAllStreamsClients().size() == 1
|
||||
&& remainingInstance.metadataForAllStreamsClients().iterator().next().topicPartitions().size() == 2,
|
||||
MAX_WAIT_TIME_MS,
|
||||
() -> "Should have rebalanced.\n" +
|
||||
|
@ -755,12 +772,12 @@ public class EosIntegrationTest {
|
|||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void shouldWriteLatestOffsetsToCheckpointOnShutdown(final boolean processingThreadsEnabled) throws Exception {
|
||||
@MethodSource("groupProtocolAndProcessingThreadsParameters")
|
||||
public void shouldWriteLatestOffsetsToCheckpointOnShutdown(final String groupProtocol, final boolean processingThreadsEnabled) throws Exception {
|
||||
final List<KeyValue<Long, Long>> writtenData = prepareData(0L, 10, 0L, 1L);
|
||||
final List<KeyValue<Long, Long>> expectedResult = computeExpectedResult(writtenData);
|
||||
|
||||
try (final KafkaStreams streams = getKafkaStreams("streams", true, "appDir", 1, processingThreadsEnabled)) {
|
||||
try (final KafkaStreams streams = getKafkaStreams("streams", true, "appDir", 1, groupProtocol, processingThreadsEnabled)) {
|
||||
writeInputData(writtenData);
|
||||
|
||||
startApplicationAndWaitUntilRunning(streams);
|
||||
|
@ -787,9 +804,9 @@ public class EosIntegrationTest {
|
|||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
@MethodSource("groupProtocolAndProcessingThreadsParameters")
|
||||
public void shouldCheckpointRestoredOffsetsWhenClosingCleanDuringRestoring(
|
||||
final boolean processingThreadsEnabled) throws Exception {
|
||||
final String groupProtocol, final boolean processingThreadsEnabled) throws Exception {
|
||||
|
||||
final Properties streamsConfiguration = new Properties();
|
||||
streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId);
|
||||
|
@ -801,6 +818,7 @@ public class EosIntegrationTest {
|
|||
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
||||
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(applicationId).getPath());
|
||||
streamsConfiguration.put(InternalConfig.PROCESSING_THREADS_ENABLED, processingThreadsEnabled);
|
||||
streamsConfiguration.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol);
|
||||
streamsConfiguration.put(StreamsConfig.restoreConsumerPrefix(ConsumerConfig.MAX_POLL_RECORDS_CONFIG), 100);
|
||||
final String stateStoreName = "stateStore";
|
||||
|
||||
|
@ -934,8 +952,13 @@ public class EosIntegrationTest {
|
|||
static final AtomicReference<TaskId> TASK_WITH_DATA = new AtomicReference<>();
|
||||
static final AtomicBoolean DID_REVOKE_IDLE_TASK = new AtomicBoolean(false);
|
||||
|
||||
@Test
|
||||
public void shouldNotCommitActiveTasksWithPendingInputIfRevokedTaskDidNotMakeProgress() throws Exception {
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {"classic", "streams"})
|
||||
public void shouldNotCommitActiveTasksWithPendingInputIfRevokedTaskDidNotMakeProgress(final String groupProtocol) throws Exception {
|
||||
// Reset static variables to ensure test isolation
|
||||
TASK_WITH_DATA.set(null);
|
||||
DID_REVOKE_IDLE_TASK.set(false);
|
||||
|
||||
final AtomicBoolean requestCommit = new AtomicBoolean(false);
|
||||
|
||||
final StreamsBuilder builder = new StreamsBuilder();
|
||||
|
@ -970,6 +993,7 @@ public class EosIntegrationTest {
|
|||
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG), MAX_POLL_INTERVAL_MS);
|
||||
properties.put(StreamsConfig.producerPrefix(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG), Integer.MAX_VALUE);
|
||||
properties.put(StreamsConfig.TASK_ASSIGNOR_CLASS_CONFIG, TestTaskAssignor.class.getName());
|
||||
properties.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol);
|
||||
|
||||
final Properties config = StreamsTestUtils.getStreamsConfig(
|
||||
applicationId,
|
||||
|
@ -1003,9 +1027,9 @@ public class EosIntegrationTest {
|
|||
// add second thread, to trigger rebalance
|
||||
// expect idle task to get revoked -- this should not trigger a TX commit
|
||||
streams.addStreamThread();
|
||||
|
||||
waitForCondition(DID_REVOKE_IDLE_TASK::get, "Idle Task was not revoked as expected.");
|
||||
|
||||
if (groupProtocol.equals("classic")) {
|
||||
waitForCondition(DID_REVOKE_IDLE_TASK::get, "Idle Task was not revoked as expected.");
|
||||
}
|
||||
// best-effort sanity check (might pass and not detect issue in slow environments)
|
||||
try {
|
||||
readResult(SINGLE_PARTITION_OUTPUT_TOPIC, 1, "consumer", 10_000L);
|
||||
|
@ -1104,6 +1128,7 @@ public class EosIntegrationTest {
|
|||
final boolean withState,
|
||||
final String appDir,
|
||||
final int numberOfStreamsThreads,
|
||||
final String groupProtocol,
|
||||
final boolean processingThreadsEnabled) {
|
||||
commitRequested = new AtomicInteger(0);
|
||||
errorInjected = new AtomicBoolean(false);
|
||||
|
@ -1212,6 +1237,7 @@ public class EosIntegrationTest {
|
|||
properties.put(StreamsConfig.STATE_DIR_CONFIG, stateTmpDir + appDir);
|
||||
properties.put(StreamsConfig.APPLICATION_SERVER_CONFIG, dummyHostName + ":2142");
|
||||
properties.put(InternalConfig.PROCESSING_THREADS_ENABLED, processingThreadsEnabled);
|
||||
properties.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol);
|
||||
|
||||
final Properties config = StreamsTestUtils.getStreamsConfig(
|
||||
applicationId,
|
||||
|
|
|
@ -28,8 +28,8 @@ import org.apache.kafka.common.serialization.StringSerializer;
|
|||
import org.apache.kafka.common.utils.MockTime;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig;
|
||||
import org.apache.kafka.streams.CloseOptions;
|
||||
import org.apache.kafka.streams.KafkaStreams;
|
||||
import org.apache.kafka.streams.KafkaStreams.CloseOptions;
|
||||
import org.apache.kafka.streams.KeyValue;
|
||||
import org.apache.kafka.streams.StreamsBuilder;
|
||||
import org.apache.kafka.streams.StreamsConfig;
|
||||
|
@ -159,7 +159,7 @@ public class KafkaStreamsCloseOptionsIntegrationTest {
|
|||
IntegrationTestUtils.startApplicationAndWaitUntilRunning(streams);
|
||||
IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived(resultConsumerConfig, OUTPUT_TOPIC, 10);
|
||||
|
||||
streams.close(new CloseOptions().leaveGroup(true).timeout(Duration.ofSeconds(30)));
|
||||
streams.close(CloseOptions.groupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP).withTimeout(Duration.ofSeconds(30)));
|
||||
waitForEmptyConsumerGroup(adminClient, streamsConfig.getProperty(StreamsConfig.APPLICATION_ID_CONFIG), 0);
|
||||
}
|
||||
|
||||
|
|
|
@ -112,6 +112,9 @@ import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.wa
|
|||
import static org.apache.kafka.streams.utils.TestUtils.safeUniqueTestName;
|
||||
import static org.apache.kafka.test.TestUtils.waitForCondition;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.hamcrest.Matchers.allOf;
|
||||
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
||||
import static org.hamcrest.Matchers.lessThanOrEqualTo;
|
||||
import static org.hamcrest.core.IsEqual.equalTo;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.junit.jupiter.api.Assertions.fail;
|
||||
|
@ -685,6 +688,52 @@ public class RestoreIntegrationTest {
|
|||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void shouldRecordRestoreMetrics(final boolean useNewProtocol) throws Exception {
|
||||
final AtomicInteger numReceived = new AtomicInteger(0);
|
||||
final StreamsBuilder builder = new StreamsBuilder();
|
||||
|
||||
final Properties props = props();
|
||||
|
||||
if (useNewProtocol) {
|
||||
props.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.STREAMS.name());
|
||||
}
|
||||
|
||||
props.put(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG, "DEBUG");
|
||||
|
||||
createStateForRestoration(inputStream, 10000);
|
||||
|
||||
final CountDownLatch shutdownLatch = new CountDownLatch(1);
|
||||
builder.table(inputStream, Consumed.with(Serdes.Integer(), Serdes.Integer()), Materialized.as("store"))
|
||||
.toStream()
|
||||
.foreach((key, value) -> {
|
||||
if (numReceived.incrementAndGet() == numberOfKeys) {
|
||||
shutdownLatch.countDown();
|
||||
}
|
||||
});
|
||||
|
||||
kafkaStreams = new KafkaStreams(builder.build(), props);
|
||||
|
||||
final AtomicLong restored = new AtomicLong(0);
|
||||
final TrackingStateRestoreListener restoreListener = new TrackingStateRestoreListener(restored);
|
||||
kafkaStreams.setGlobalStateRestoreListener(restoreListener);
|
||||
kafkaStreams.start();
|
||||
|
||||
assertTrue(shutdownLatch.await(30, TimeUnit.SECONDS));
|
||||
assertThat(numReceived.get(), equalTo(numberOfKeys));
|
||||
|
||||
final Map<String, Long> taskIdToMetricValue = kafkaStreams.metrics().entrySet().stream()
|
||||
.filter(e -> e.getKey().name().equals("restore-latency-max"))
|
||||
.collect(Collectors.toMap(e -> e.getKey().tags().get("task-id"), e -> ((Double) e.getValue().metricValue()).longValue()));
|
||||
|
||||
for (final Map.Entry<TopicPartition, Long> entry : restoreListener.changelogToRestoreTime().entrySet()) {
|
||||
final long lowerBound = entry.getValue() - TimeUnit.NANOSECONDS.convert(1, TimeUnit.SECONDS);
|
||||
final long upperBound = entry.getValue() + TimeUnit.NANOSECONDS.convert(1, TimeUnit.SECONDS);
|
||||
assertThat(taskIdToMetricValue.get("0_" + entry.getKey().partition()), allOf(greaterThanOrEqualTo(lowerBound), lessThanOrEqualTo(upperBound)));
|
||||
}
|
||||
}
|
||||
|
||||
private void validateReceivedMessages(final List<KeyValue<Integer, Integer>> expectedRecords,
|
||||
final String outputTopic) throws Exception {
|
||||
final Properties consumerProperties = new Properties();
|
||||
|
@ -971,4 +1020,4 @@ public class RestoreIntegrationTest {
|
|||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1337,6 +1337,8 @@ public class IntegrationTestUtils {
|
|||
public final Map<TopicPartition, AtomicLong> changelogToStartOffset = new ConcurrentHashMap<>();
|
||||
public final Map<TopicPartition, AtomicLong> changelogToEndOffset = new ConcurrentHashMap<>();
|
||||
public final Map<TopicPartition, AtomicLong> changelogToTotalNumRestored = new ConcurrentHashMap<>();
|
||||
private final Map<TopicPartition, AtomicLong> changelogToRestoreStartTime = new ConcurrentHashMap<>();
|
||||
private final Map<TopicPartition, AtomicLong> changelogToRestoreEndTime = new ConcurrentHashMap<>();
|
||||
private final AtomicLong restored;
|
||||
|
||||
public TrackingStateRestoreListener() {
|
||||
|
@ -1355,6 +1357,7 @@ public class IntegrationTestUtils {
|
|||
changelogToStartOffset.put(topicPartition, new AtomicLong(startingOffset));
|
||||
changelogToEndOffset.put(topicPartition, new AtomicLong(endingOffset));
|
||||
changelogToTotalNumRestored.put(topicPartition, new AtomicLong(0L));
|
||||
changelogToRestoreStartTime.put(topicPartition, new AtomicLong(System.nanoTime()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1372,6 +1375,7 @@ public class IntegrationTestUtils {
|
|||
if (restored != null) {
|
||||
restored.addAndGet(totalRestored);
|
||||
}
|
||||
changelogToRestoreEndTime.put(topicPartition, new AtomicLong(System.nanoTime()));
|
||||
}
|
||||
|
||||
public long totalNumRestored() {
|
||||
|
@ -1381,6 +1385,11 @@ public class IntegrationTestUtils {
|
|||
}
|
||||
return totalNumRestored;
|
||||
}
|
||||
|
||||
public Map<TopicPartition, Long> changelogToRestoreTime() {
|
||||
return changelogToRestoreStartTime.entrySet().stream()
|
||||
.collect(Collectors.toMap(Map.Entry::getKey, e -> changelogToRestoreEndTime.get(e.getKey()).get() - e.getValue().get()));
|
||||
}
|
||||
}
|
||||
|
||||
public static class TrackingStandbyUpdateListener implements StandbyUpdateListener {
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue