diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml
index c02c47c7584be..bd260e8f1fc76 100644
--- a/.github/workflows/ci.yml
+++ b/.github/workflows/ci.yml
@@ -19,7 +19,7 @@ on:
push:
branches:
- 'trunk'
- - '4.0'
+ - '4.1'
schedule:
- cron: '0 0 * * 6,0' # Run on Saturday and Sunday at midnight UTC
@@ -28,7 +28,7 @@ on:
types: [ opened, synchronize, ready_for_review, reopened ]
branches:
- 'trunk'
- - '4.0'
+ - '4.1'
concurrency:
group: ${{ github.workflow }}-${{ github.event.pull_request.number || github.ref }}
diff --git a/LICENSE-binary b/LICENSE-binary
index fc87fb913f879..b601ccebfb7ce 100644
--- a/LICENSE-binary
+++ b/LICENSE-binary
@@ -206,11 +206,11 @@ This project bundles some components that are also licensed under the Apache
License Version 2.0:
- caffeine-3.2.0
-- commons-beanutils-1.9.4
+- commons-beanutils-1.11.0
- commons-collections-3.2.2
- commons-digester-2.1
-- commons-lang3-3.12.0
-- commons-logging-1.3.2
+- commons-lang3-3.18.0
+- commons-logging-1.3.5
- commons-validator-1.9.0
- hash4j-0.22.0
- jackson-annotations-2.19.0
diff --git a/build.gradle b/build.gradle
index b4e9cbb801133..97250637b7010 100644
--- a/build.gradle
+++ b/build.gradle
@@ -193,7 +193,10 @@ allprojects {
// ensure we have a single version in the classpath despite transitive dependencies
libs.scalaLibrary,
libs.scalaReflect,
- libs.jacksonAnnotations
+ // Workaround before `commons-validator` has new release. See KAFKA-19359.
+ libs.commonsBeanutils,
+ libs.jacksonAnnotations,
+ libs.commonsLang
)
}
}
@@ -1058,6 +1061,7 @@ project(':core') {
testImplementation libs.junitJupiter
testImplementation libs.caffeine
testImplementation testLog4j2Libs
+ testImplementation libs.mockOAuth2Server
testRuntimeOnly runtimeTestLibs
}
diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml
index 2a93275083d2f..ca1c3d6576332 100644
--- a/checkstyle/suppressions.xml
+++ b/checkstyle/suppressions.xml
@@ -194,7 +194,7 @@
+ files="(KafkaStreams|KStreamImpl|KTableImpl|InternalTopologyBuilder|StreamsPartitionAssignor|StreamThread|IQv2StoreIntegrationTest|KStreamImplTest|RocksDBStore|StreamTask).java"/>
diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerSubscriptionTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerSubscriptionTest.java
index 7e014537cd943..e8c32d3790878 100644
--- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerSubscriptionTest.java
+++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerSubscriptionTest.java
@@ -374,6 +374,49 @@ public void testAsyncConsumerRe2JPatternExpandSubscription() throws InterruptedE
}
}
+ @ClusterTest
+ public void testTopicIdSubscriptionWithRe2JRegexAndOffsetsFetch() throws InterruptedException {
+ var topic1 = "topic1"; // matches subscribed pattern
+ cluster.createTopic(topic1, 2, (short) BROKER_COUNT);
+
+ Map config = Map.of(GROUP_PROTOCOL_CONFIG, GroupProtocol.CONSUMER.name().toLowerCase(Locale.ROOT));
+ try (
+ Producer producer = cluster.producer();
+ Consumer consumer = cluster.consumer(config)
+ ) {
+ assertEquals(0, consumer.assignment().size());
+
+ // Subscribe to broker-side regex and fetch. This will require metadata for topic IDs.
+ var pattern = new SubscriptionPattern("topic.*");
+ consumer.subscribe(pattern);
+ var assignment = Set.of(
+ new TopicPartition(topic, 0),
+ new TopicPartition(topic, 1),
+ new TopicPartition(topic1, 0),
+ new TopicPartition(topic1, 1));
+ awaitAssignment(consumer, assignment);
+ var totalRecords = 10;
+ var startingTimestamp = System.currentTimeMillis();
+ var tp = new TopicPartition(topic1, 0);
+ sendRecords(producer, tp, totalRecords, startingTimestamp);
+ consumeAndVerifyRecords(consumer, tp, totalRecords, 0, 0, startingTimestamp);
+
+ // Fetch offsets for known and unknown topics. This will require metadata for topic names temporarily (transient topics)
+ var topic2 = "newTopic2";
+ cluster.createTopic(topic2, 2, (short) BROKER_COUNT);
+ var unassignedPartition = new TopicPartition(topic2, 0);
+ var offsets = consumer.endOffsets(List.of(unassignedPartition, tp));
+ var expectedOffsets = Map.of(
+ unassignedPartition, 0L,
+ tp, (long) totalRecords);
+ assertEquals(expectedOffsets, offsets);
+
+ // Fetch records again with the regex subscription. This will require metadata for topic IDs again.
+ sendRecords(producer, tp, totalRecords, startingTimestamp);
+ consumeAndVerifyRecords(consumer, tp, totalRecords, totalRecords, 0, startingTimestamp);
+ }
+ }
+
@ClusterTest
public void testRe2JPatternSubscriptionAndTopicSubscription() throws InterruptedException {
Map config = Map.of(GROUP_PROTOCOL_CONFIG, GroupProtocol.CONSUMER.name().toLowerCase(Locale.ROOT));
diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/producer/ProducerSendWhileDeletionTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/producer/ProducerSendWhileDeletionTest.java
new file mode 100644
index 0000000000000..e775247d69409
--- /dev/null
+++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/producer/ProducerSendWhileDeletionTest.java
@@ -0,0 +1,342 @@
+/*
+ * 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.producer;
+
+import kafka.server.KafkaBroker;
+
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.NewPartitionReassignment;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.test.ClusterInstance;
+import org.apache.kafka.common.test.TestUtils;
+import org.apache.kafka.common.test.api.ClusterConfigProperty;
+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.server.config.ServerLogConfigs;
+import org.apache.kafka.storage.internals.checkpoint.OffsetCheckpointFile;
+import org.apache.kafka.storage.internals.log.UnifiedLog;
+
+import java.io.File;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.kafka.clients.producer.ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG;
+import static org.apache.kafka.clients.producer.ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG;
+import static org.apache.kafka.clients.producer.ProducerConfig.MAX_BLOCK_MS_CONFIG;
+import static org.apache.kafka.clients.producer.ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG;
+import static org.apache.kafka.clients.producer.ProducerSendWhileDeletionTest.BROKER_COUNT;
+import static org.apache.kafka.server.config.ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG;
+import static org.apache.kafka.server.config.ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG;
+import static org.apache.kafka.server.config.ServerLogConfigs.NUM_PARTITIONS_CONFIG;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+
+@ClusterTestDefaults(
+ types = {Type.KRAFT},
+ brokers = BROKER_COUNT,
+ serverProperties = {
+ @ClusterConfigProperty(key = NUM_PARTITIONS_CONFIG, value = "2"),
+ @ClusterConfigProperty(key = DEFAULT_REPLICATION_FACTOR_CONFIG, value = "2"),
+ @ClusterConfigProperty(key = AUTO_LEADER_REBALANCE_ENABLE_CONFIG, value = "false"),
+ @ClusterConfigProperty(key = "log.segment.delete.delay.ms", value = "1000"),
+ @ClusterConfigProperty(key = ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, value = "100")
+ }
+)
+public class ProducerSendWhileDeletionTest {
+
+ public static final int BROKER_COUNT = 2;
+ private static final int DEFAULT_LINGER_MS = 5;
+ private final int numRecords = 10;
+ private final String topic = "topic";
+ private final ClusterInstance cluster;
+
+ public ProducerSendWhileDeletionTest(ClusterInstance cluster) {
+ this.cluster = cluster;
+ }
+
+ /**
+ * Tests that Producer gets self-recovered when a topic is deleted mid-way of produce.
+ *
+ * Producer will attempt to send messages to the partition specified in each record, and should
+ * succeed as long as the partition is included in the metadata.
+ */
+ @ClusterTest
+ public void testSendWithTopicDeletionMidWay() throws Exception {
+ try (var admin = cluster.admin();
+ var producer = createProducer()
+ ) {
+ // Create topic with leader as 0 for the 2 partitions.
+ var topicAssignment = Map.of(
+ 0, List.of(0, 1),
+ 1, List.of(0, 1)
+ );
+ admin.createTopics(List.of(new NewTopic(topic, topicAssignment)));
+
+ // Change leader to 1 for both the partitions to increase leader epoch from 0 -> 1
+ var reassignment = Map.of(
+ new TopicPartition(topic, 0), Optional.of(new NewPartitionReassignment(List.of(1, 0))),
+ new TopicPartition(topic, 1), Optional.of(new NewPartitionReassignment(List.of(1, 0)))
+ );
+ admin.alterPartitionReassignments(reassignment).all().get();
+
+ for (var i = 1; i <= numRecords; i++) {
+ var resp = producer.send(
+ new ProducerRecord<>(topic, null, ("value" + i).getBytes())
+ ).get();
+ assertEquals(topic, resp.topic());
+ }
+
+ // Start topic deletion
+ admin.deleteTopics(List.of(topic)).all().get();
+ // Verify that the topic is deleted when no metadata request comes in
+ verifyTopicDeletion();
+
+ // Producer should be able to send messages even after topic gets deleted and auto-created
+ var finalResp = producer.send(new ProducerRecord<>(topic, null, "value".getBytes())).get();
+ assertEquals(topic, finalResp.topic());
+ }
+ }
+
+ /**
+ * Tests that Producer produce to new topic id after recreation.
+ *
+ * Producer will attempt to send messages to the partition specified in each record, and should
+ * succeed as long as the metadata has been updated with new topic id.
+ */
+ @ClusterTest
+ public void testSendWithRecreatedTopic() throws Exception {
+ try (var admin = cluster.admin();
+ var producer = createProducer()
+ ) {
+ cluster.createTopic(topic, 1, (short) 1);
+ var topicId = topicMetadata().topicId();
+
+ for (int i = 1; i <= numRecords; i++) {
+ var resp = producer.send(new ProducerRecord<>(topic, null, ("value" + i).getBytes())).get();
+ assertEquals(topic, resp.topic());
+ }
+
+ // Start topic deletion
+ admin.deleteTopics(List.of(topic)).all().get();
+
+ // Verify that the topic is deleted when no metadata request comes in
+ verifyTopicDeletion();
+ cluster.createTopic(topic, 1, (short) 1);
+ assertNotEquals(topicId, topicMetadata().topicId());
+
+ // Producer should be able to send messages even after topic gets recreated
+ var recordMetadata = producer.send(new ProducerRecord<>(topic, null, "value".getBytes(StandardCharsets.UTF_8))).get();
+ assertEquals(topic, recordMetadata.topic());
+ assertEquals(0, recordMetadata.offset());
+ }
+ }
+
+ @ClusterTest
+ public void testSendWhileTopicGetRecreated() {
+ int maxNumTopicRecreationAttempts = 5;
+ var recreateTopicFuture = CompletableFuture.supplyAsync(() -> {
+ var topicIds = new HashSet();
+ while (topicIds.size() < maxNumTopicRecreationAttempts) {
+ try (var admin = cluster.admin()) {
+ if (admin.listTopics().names().get().contains(topic)) {
+ admin.deleteTopics(List.of(topic)).all().get();
+ }
+ topicIds.add(admin.createTopics(List.of(new NewTopic(topic, 2, (short) 1))).topicId(topic).get());
+ } catch (Exception e) {
+ // ignore
+ }
+ }
+ return topicIds;
+ });
+
+ AtomicInteger numAcks = new AtomicInteger(0);
+ var producerFuture = CompletableFuture.runAsync(() -> {
+ try (var producer = createProducer()) {
+ for (int i = 1; i <= numRecords; i++) {
+ producer.send(new ProducerRecord<>(topic, null, ("value" + i).getBytes()),
+ (metadata, exception) -> {
+ numAcks.incrementAndGet();
+ });
+ }
+ producer.flush();
+ }
+ });
+ var topicIds = recreateTopicFuture.join();
+ producerFuture.join();
+ assertEquals(maxNumTopicRecreationAttempts, topicIds.size());
+ assertEquals(numRecords, numAcks.intValue());
+ }
+
+ @ClusterTest
+ public void testSendWithTopicReassignmentIsMidWay() throws Exception {
+ var partition0 = new TopicPartition(topic, 0);
+
+ try (var admin = cluster.admin();
+ var producer = createProducer()
+ ) {
+ // Create topic with leader as 0 for the 1 partition.
+ admin.createTopics(List.of(new NewTopic(topic, Map.of(0, List.of(0)))));
+ assertLeader(partition0, 0);
+
+ var topicDetails = topicMetadata();
+ for (var i = 1; i <= numRecords; i++) {
+ var resp = producer.send(new ProducerRecord<>(topic, null, ("value" + i).getBytes())).get();
+ assertEquals(topic, resp.topic());
+ }
+
+ var reassignment = Map.of(
+ partition0, Optional.of(new NewPartitionReassignment(List.of(1)))
+ );
+ // Change replica assignment from 0 to 1. Leadership moves to 1.
+ admin.alterPartitionReassignments(reassignment).all().get();
+
+ assertLeader(partition0, 1);
+ assertEquals(topicDetails.topicId(), topicMetadata().topicId());
+
+ // Producer should be able to send messages even after topic gets reassigned
+ var recordMetadata = producer.send(new ProducerRecord<>(topic, null, "value".getBytes())).get();
+ assertEquals(topic, recordMetadata.topic());
+ }
+ }
+
+ private Producer createProducer() {
+ return cluster.producer(Map.of(
+ MAX_BLOCK_MS_CONFIG, 5000L,
+ REQUEST_TIMEOUT_MS_CONFIG, 10000,
+ DELIVERY_TIMEOUT_MS_CONFIG, 10000 + DEFAULT_LINGER_MS,
+ KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()
+ ));
+ }
+
+ private void verifyTopicDeletion() throws InterruptedException {
+ var topicPartitions = List.of(
+ new TopicPartition(topic, 0),
+ new TopicPartition(topic, 1)
+ );
+
+ // ensure that the topic-partition has been deleted from all brokers' replica managers
+ TestUtils.waitForCondition(() ->
+ cluster.brokers().values().stream()
+ .allMatch(broker -> topicPartitions.stream()
+ .allMatch(tp -> broker.replicaManager().onlinePartition(tp).isEmpty())
+ ), "Replica manager's should have deleted all of this topic's partitions");
+
+ // ensure that logs from all replicas are deleted
+ TestUtils.waitForCondition(() -> cluster.brokers().values().stream()
+ .allMatch(broker -> topicPartitions.stream()
+ .allMatch(tp -> broker.logManager().getLog(tp, false).isEmpty())
+ ), "Replica logs not deleted after delete topic is complete");
+
+ // ensure that topic is removed from all cleaner offsets
+ TestUtils.waitForCondition(() -> cluster.brokers().values().stream()
+ .allMatch(broker -> topicPartitions.stream()
+ .allMatch(tp -> partitionNotInCheckpoint(broker, tp))
+ ), "Cleaner offset for deleted partition should have been removed");
+
+ TestUtils.waitForCondition(() -> cluster.brokers().values().stream()
+ .allMatch(broker -> broker.config().logDirs().stream()
+ .allMatch(logDir -> topicPartitions.stream().noneMatch(tp ->
+ new File(logDir, tp.topic() + "-" + tp.partition()).exists())
+ )
+ ), "Failed to soft-delete the data to a delete directory");
+
+ TestUtils.waitForCondition(() -> cluster.brokers().values().stream()
+ .allMatch(broker -> broker.config().logDirs().stream()
+ .allMatch(logDir -> deletionDirectoriesAbsent(logDir, topicPartitions))
+ ), "Failed to hard-delete the delete directory");
+ }
+
+ private boolean partitionNotInCheckpoint(KafkaBroker broker, TopicPartition tp) {
+ List liveLogDirs = new ArrayList<>();
+ broker.logManager().liveLogDirs().foreach(liveLogDirs::add);
+ var checkpoints = liveLogDirs.stream().map(logDir -> {
+ try {
+ return new OffsetCheckpointFile(new File(logDir, "cleaner-offset-checkpoint"), null).read();
+ } catch (Exception e) {
+ return new HashMap();
+ }
+ }).toList();
+ return checkpoints.stream().noneMatch(checkpointsPerLogDir ->
+ checkpointsPerLogDir.containsKey(tp));
+ }
+
+ private boolean deletionDirectoriesAbsent(String logDir, List topicPartitions) {
+ var directoryNames = new File(logDir).list();
+ if (directoryNames == null) {
+ return true;
+ }
+ return topicPartitions.stream().allMatch(tp ->
+ Arrays.stream(directoryNames).noneMatch(directoryName ->
+ directoryName.startsWith(tp.topic() + "-" + tp.partition()) &&
+ directoryName.endsWith(UnifiedLog.DELETE_DIR_SUFFIX)));
+ }
+
+ private TopicDescription topicMetadata() throws Exception {
+ try (var admin = cluster.admin()) {
+ return admin.describeTopics(List.of(topic))
+ .allTopicNames()
+ .get()
+ .get(topic);
+
+ }
+ }
+
+ private void assertLeader(TopicPartition topicPartition, Integer expectedLeaderOpt) throws InterruptedException {
+ try (var admin = cluster.admin()) {
+ TestUtils.waitForCondition(() -> {
+ try {
+ Optional currentLeader = getCurrentLeader(admin, topicPartition);
+ return currentLeader.equals(Optional.of(expectedLeaderOpt));
+ } catch (Exception e) {
+ if (e.getCause() instanceof UnknownTopicOrPartitionException) {
+ return false;
+ }
+ throw new RuntimeException(e);
+ }
+ }, "Waiting for leader to become " + expectedLeaderOpt);
+ }
+ }
+
+ private Optional getCurrentLeader(Admin admin, TopicPartition topicPartition) throws Exception {
+ return admin.describeTopics(List.of(topicPartition.topic()))
+ .allTopicNames()
+ .get()
+ .get(topicPartition.topic())
+ .partitions()
+ .stream()
+ .filter(p -> p.partition() == topicPartition.partition())
+ .findFirst()
+ .map(TopicPartitionInfo::leader)
+ .map(Node::id);
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/clients/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/Metadata.java
index 4c567b7d466b5..0986d8a67bc36 100644
--- a/clients/src/main/java/org/apache/kafka/clients/Metadata.java
+++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java
@@ -508,7 +508,7 @@ private MetadataSnapshot handleMetadataResponse(MetadataResponse metadataRespons
topicId = null;
}
- if (!retainTopic(topicName, metadata.isInternal(), nowMs))
+ if (!retainTopic(topicName, topicId, metadata.isInternal(), nowMs))
continue;
if (metadata.isInternal())
@@ -758,10 +758,20 @@ public Map topicNames() {
return metadataSnapshot.topicNames();
}
+ /**
+ * Based on the topic name, check if the topic metadata should be kept when received in a metadata response.
+ */
protected boolean retainTopic(String topic, boolean isInternal, long nowMs) {
return true;
}
+ /**
+ * Based on the topic name and topic ID, check if the topic metadata should be kept when received in a metadata response.
+ */
+ protected boolean retainTopic(String topicName, Uuid topicId, boolean isInternal, long nowMs) {
+ return retainTopic(topicName, isInternal, nowMs);
+ }
+
public static class MetadataRequestAndVersion {
public final MetadataRequest.Builder requestBuilder;
public final int requestVersion;
diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
index b283d65cbee06..843509eb0a942 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
@@ -579,10 +579,12 @@ static KafkaAdminClient createInternal(AdminClientConfig config,
Time time) {
Metrics metrics = null;
String clientId = generateClientId(config);
+ List reporters = CommonClientConfigs.metricsReporters(clientId, config);
Optional clientTelemetryReporter = CommonClientConfigs.telemetryReporter(clientId, config);
+ clientTelemetryReporter.ifPresent(reporters::add);
try {
- metrics = new Metrics(new MetricConfig(), new LinkedList<>(), time);
+ metrics = new Metrics(new MetricConfig(), reporters, time);
LogContext logContext = createLogContext(clientId);
return new KafkaAdminClient(config, clientId, time, metadataManager, metrics,
client, null, logContext, clientTelemetryReporter);
@@ -627,9 +629,7 @@ private KafkaAdminClient(AdminClientConfig config,
CommonClientConfigs.RETRY_BACKOFF_EXP_BASE,
retryBackoffMaxMs,
CommonClientConfigs.RETRY_BACKOFF_JITTER);
- List reporters = CommonClientConfigs.metricsReporters(this.clientId, config);
this.clientTelemetryReporter = clientTelemetryReporter;
- this.clientTelemetryReporter.ifPresent(reporters::add);
this.metadataRecoveryStrategy = MetadataRecoveryStrategy.forName(config.getString(AdminClientConfig.METADATA_RECOVERY_STRATEGY_CONFIG));
this.partitionLeaderCache = new HashMap<>();
this.adminFetchMetricsManager = new AdminFetchMetricsManager(metrics);
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractFetch.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractFetch.java
index 533cbbbaa9851..9d96712a4734b 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractFetch.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractFetch.java
@@ -45,6 +45,7 @@
import java.io.Closeable;
import java.time.Duration;
+import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
@@ -54,7 +55,6 @@
import java.util.Optional;
import java.util.Set;
import java.util.function.Predicate;
-import java.util.stream.Collectors;
import static org.apache.kafka.clients.consumer.internals.FetchUtils.requestMetadataUpdate;
@@ -223,10 +223,15 @@ protected void handleFetchSuccess(final Node fetchTarget,
}
if (!partitionsWithUpdatedLeaderInfo.isEmpty()) {
- List leaderNodes = response.data().nodeEndpoints().stream()
- .map(e -> new Node(e.nodeId(), e.host(), e.port(), e.rack()))
- .filter(e -> !e.equals(Node.noNode()))
- .collect(Collectors.toList());
+ List leaderNodes = new ArrayList<>();
+
+ for (FetchResponseData.NodeEndpoint e : response.data().nodeEndpoints()) {
+ Node node = new Node(e.nodeId(), e.host(), e.port(), e.rack());
+
+ if (!node.equals(Node.noNode()))
+ leaderNodes.add(node);
+ }
+
Set updatedPartitions = metadata.updatePartitionLeadership(partitionsWithUpdatedLeaderInfo, leaderNodes);
updatedPartitions.forEach(
tp -> {
@@ -397,7 +402,7 @@ protected Map prepareCloseFetchSessi
fetchable.put(fetchTarget, sessionHandler.newBuilder());
});
- return fetchable.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().build()));
+ return convert(fetchable);
}
/**
@@ -470,7 +475,21 @@ protected Map prepareFetchRequests()
}
}
- return fetchable.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().build()));
+ return convert(fetchable);
+ }
+
+ /**
+ * This method converts {@link FetchSessionHandler.Builder} instances to
+ * {@link FetchSessionHandler.FetchRequestData} instances. It intentionally forgoes use of the Java Collections
+ * Streams API to reduce overhead in the critical network path.
+ */
+ private Map convert(Map fetchable) {
+ Map map = new HashMap<>(fetchable.size());
+
+ for (Map.Entry entry : fetchable.entrySet())
+ map.put(entry.getKey(), entry.getValue().build());
+
+ return map;
}
/**
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java
index 30d28fb722f3d..74ecf7f9bb8d4 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java
@@ -21,7 +21,6 @@
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
import org.apache.kafka.clients.consumer.internals.metrics.RebalanceMetricsManager;
import org.apache.kafka.common.KafkaException;
-import org.apache.kafka.common.TopicIdPartition;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.requests.AbstractResponse;
@@ -44,7 +43,6 @@
import java.util.TreeSet;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.stream.Collectors;
import static java.util.Collections.unmodifiableList;
@@ -373,6 +371,9 @@ private void replaceTargetAssignmentWithNewAssignment(Map nameFromMetadata = findTopicNameInGlobalOrLocalCache(topicId);
nameFromMetadata.ifPresent(resolvedTopicName -> {
// Name resolved, so assignment is ready for reconciliation.
- topicPartitions.forEach(tp ->
- assignmentReadyToReconcile.add(new TopicIdPartition(topicId, tp, resolvedTopicName))
- );
+ assignmentReadyToReconcile.addAll(topicId, resolvedTopicName, topicPartitions);
it.remove();
});
}
@@ -1135,7 +1134,7 @@ CompletableFuture revokePartitions(Set partitionsToRevoke)
// Ensure the set of partitions to revoke are still assigned
Set revokedPartitions = new HashSet<>(partitionsToRevoke);
revokedPartitions.retainAll(subscriptions.assignedPartitions());
- log.info("Revoking previously assigned partitions {}", revokedPartitions.stream().map(TopicPartition::toString).collect(Collectors.joining(", ")));
+ log.info("Revoking previously assigned partitions {}", revokedPartitions);
signalPartitionsBeingRevoked(revokedPartitions);
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManager.java
index b615977075441..e07424a63938e 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManager.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManager.java
@@ -44,7 +44,6 @@
import java.util.SortedSet;
import java.util.TreeSet;
import java.util.concurrent.CompletableFuture;
-import java.util.stream.Collectors;
import static org.apache.kafka.clients.consumer.CloseOptions.GroupMembershipOperation.DEFAULT;
import static org.apache.kafka.clients.consumer.CloseOptions.GroupMembershipOperation.LEAVE_GROUP;
@@ -415,7 +414,7 @@ private void logPausedPartitionsBeingRevoked(Set partitionsToRev
Set revokePausedPartitions = subscriptions.pausedPartitions();
revokePausedPartitions.retainAll(partitionsToRevoke);
if (!revokePausedPartitions.isEmpty()) {
- log.info("The pause flag in partitions [{}] will be removed due to revocation.", revokePausedPartitions.stream().map(TopicPartition::toString).collect(Collectors.joining(", ")));
+ log.info("The pause flag in partitions {} will be removed due to revocation.", revokePausedPartitions);
}
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMetadata.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMetadata.java
index 434e989f068e5..677beaa5fa1c8 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMetadata.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMetadata.java
@@ -18,6 +18,7 @@
import org.apache.kafka.clients.Metadata;
import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.internals.ClusterResourceListeners;
import org.apache.kafka.common.requests.MetadataRequest;
import org.apache.kafka.common.utils.LogContext;
@@ -66,14 +67,34 @@ public boolean allowAutoTopicCreation() {
return allowAutoTopicCreation;
}
+ /**
+ * Constructs a metadata request builder for fetching cluster metadata for the topics the consumer needs.
+ * This will include:
+ *
+ * - topics the consumer is subscribed to using topic names (calls to subscribe with topic name list or client-side regex)
+ * - topics the consumer is subscribed to using topic IDs (calls to subscribe with broker-side regex RE2J)
+ * - topics involved in calls for fetching offsets (transient topics)
+ *
+ * Note that this will generate a request for all topics in the cluster only when the consumer is subscribed to a client-side regex.
+ */
@Override
public synchronized MetadataRequest.Builder newMetadataRequestBuilder() {
- if (subscription.hasPatternSubscription() || subscription.hasRe2JPatternSubscription())
+ if (subscription.hasPatternSubscription()) {
+ // Consumer subscribed to client-side regex => request all topics to compute regex
return MetadataRequest.Builder.allTopics();
+ }
+ if (subscription.hasRe2JPatternSubscription() && transientTopics.isEmpty()) {
+ // Consumer subscribed to broker-side regex and no need for transient topic names metadata => request topic IDs
+ return MetadataRequest.Builder.forTopicIds(subscription.assignedTopicIds());
+ }
+ // Subscription to explicit topic names or transient topics present.
+ // Note that in the case of RE2J broker-side regex subscription, we may end up in this path
+ // if there are transient topics. They are just needed temporarily (lifetime of offsets-related API calls),
+ // so we'll request them to unblock their APIs, then go back to requesting assigned topic IDs as needed
List topics = new ArrayList<>();
topics.addAll(subscription.metadataTopics());
topics.addAll(transientTopics);
- return new MetadataRequest.Builder(topics, allowAutoTopicCreation);
+ return MetadataRequest.Builder.forTopicNames(topics, allowAutoTopicCreation);
}
synchronized void addTransientTopics(Set topics) {
@@ -86,6 +107,15 @@ synchronized void clearTransientTopics() {
this.transientTopics.clear();
}
+ /**
+ * Check if the metadata for the topic should be retained, based on the topic name.
+ * It will return true for:
+ *
+ * - topic names the consumer subscribed to
+ * - topic names that match a client-side regex the consumer subscribed to
+ * - topics involved in fetching offsets
+ *
+ */
@Override
protected synchronized boolean retainTopic(String topic, boolean isInternal, long nowMs) {
if (transientTopics.contains(topic) || subscription.needsMetadata(topic))
@@ -94,6 +124,21 @@ protected synchronized boolean retainTopic(String topic, boolean isInternal, lon
if (isInternal && !includeInternalTopics)
return false;
- return subscription.matchesSubscribedPattern(topic) || subscription.isAssignedFromRe2j(topic);
+ return subscription.matchesSubscribedPattern(topic);
+ }
+
+ /**
+ * Check if the metadata for the topic should be retained, based on topic name and topic ID.
+ * This will return true for:
+ *
+ * - topic names the consumer subscribed to
+ * - topic names that match a client-side regex the consumer subscribed to
+ * - topic IDs that have been received in an assignment from the broker after the consumer subscribed to a broker-side regex
+ * - topics involved in fetching offsets
+ *
+ */
+ @Override
+ protected synchronized boolean retainTopic(String topicName, Uuid topicId, boolean isInternal, long nowMs) {
+ return retainTopic(topicName, isInternal, nowMs) || subscription.isAssignedFromRe2j(topicId);
}
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java
index 550f7c8258bc7..d2d178a88c38b 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java
@@ -35,13 +35,13 @@
import java.io.Closeable;
import java.time.Duration;
+import java.util.ArrayList;
import java.util.Collection;
import java.util.LinkedList;
import java.util.List;
import java.util.Objects;
import java.util.concurrent.BlockingQueue;
import java.util.function.Supplier;
-import java.util.stream.Collectors;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.DEFAULT_CLOSE_TIMEOUT_MS;
import static org.apache.kafka.common.utils.Utils.closeQuietly;
@@ -144,6 +144,7 @@ void initializeResources() {
*
*/
void runOnce() {
+ // The following code avoids use of the Java Collections Streams API to reduce overhead in this loop.
processApplicationEvents();
final long currentTimeMs = time.milliseconds();
@@ -152,19 +153,24 @@ void runOnce() {
}
lastPollTimeMs = currentTimeMs;
- final long pollWaitTimeMs = requestManagers.entries().stream()
- .map(rm -> rm.poll(currentTimeMs))
- .mapToLong(networkClientDelegate::addAll)
- .filter(ms -> ms <= MAX_POLL_TIMEOUT_MS)
- .min()
- .orElse(MAX_POLL_TIMEOUT_MS);
+ long pollWaitTimeMs = MAX_POLL_TIMEOUT_MS;
+
+ for (RequestManager rm : requestManagers.entries()) {
+ NetworkClientDelegate.PollResult pollResult = rm.poll(currentTimeMs);
+ long timeoutMs = networkClientDelegate.addAll(pollResult);
+ pollWaitTimeMs = Math.min(pollWaitTimeMs, timeoutMs);
+ }
networkClientDelegate.poll(pollWaitTimeMs, currentTimeMs);
- cachedMaximumTimeToWait = requestManagers.entries().stream()
- .mapToLong(rm -> rm.maximumTimeToWait(currentTimeMs))
- .min()
- .orElse(Long.MAX_VALUE);
+ long maxTimeToWaitMs = Long.MAX_VALUE;
+
+ for (RequestManager rm : requestManagers.entries()) {
+ long waitMs = rm.maximumTimeToWait(currentTimeMs);
+ maxTimeToWaitMs = Math.min(maxTimeToWaitMs, waitMs);
+ }
+
+ cachedMaximumTimeToWait = maxTimeToWaitMs;
reapExpiredApplicationEvents(currentTimeMs);
List> uncompletedEvents = applicationEventReaper.uncompletedEvents();
@@ -235,10 +241,11 @@ private void reapExpiredApplicationEvents(long currentTimeMs) {
static void runAtClose(final Collection requestManagers,
final NetworkClientDelegate networkClientDelegate,
final long currentTimeMs) {
- // These are the optional outgoing requests at the
- requestManagers.stream()
- .map(rm -> rm.pollOnClose(currentTimeMs))
- .forEach(networkClientDelegate::addAll);
+ // These are the optional outgoing requests at the time of closing the consumer
+ for (RequestManager rm : requestManagers) {
+ NetworkClientDelegate.PollResult pollResult = rm.pollOnClose(currentTimeMs);
+ networkClientDelegate.addAll(pollResult);
+ }
}
public boolean isRunning() {
@@ -362,12 +369,13 @@ void cleanup() {
* If there is a metadata error, complete all uncompleted events that require subscription metadata.
*/
private void maybeFailOnMetadataError(List> events) {
- List extends CompletableApplicationEvent>> subscriptionMetadataEvent = events.stream()
- .filter(e -> e instanceof CompletableApplicationEvent>)
- .map(e -> (CompletableApplicationEvent>) e)
- .filter(CompletableApplicationEvent::requireSubscriptionMetadata)
- .collect(Collectors.toList());
-
+ List> subscriptionMetadataEvent = new ArrayList<>();
+
+ for (CompletableEvent> ce : events) {
+ if (ce instanceof CompletableApplicationEvent && ((CompletableApplicationEvent>) ce).requireSubscriptionMetadata())
+ subscriptionMetadataEvent.add((CompletableApplicationEvent>) ce);
+ }
+
if (subscriptionMetadataEvent.isEmpty())
return;
networkClientDelegate.getAndClearMetadataError().ifPresent(metadataError ->
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java
index b42cf85a8602a..3f66b6ce3c383 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java
@@ -30,7 +30,6 @@
import java.util.Optional;
import java.util.Set;
import java.util.SortedSet;
-import java.util.stream.Collectors;
/**
* This class encapsulates the invocation of the callback methods defined in the {@link ConsumerRebalanceListener}
@@ -55,7 +54,7 @@ public class ConsumerRebalanceListenerInvoker {
}
public Exception invokePartitionsAssigned(final SortedSet assignedPartitions) {
- log.info("Adding newly assigned partitions: {}", assignedPartitions.stream().map(TopicPartition::toString).collect(Collectors.joining(", ")));
+ log.info("Adding newly assigned partitions: {}", assignedPartitions);
Optional listener = subscriptions.rebalanceListener();
@@ -67,8 +66,12 @@ public Exception invokePartitionsAssigned(final SortedSet assign
} catch (WakeupException | InterruptException e) {
throw e;
} catch (Exception e) {
- log.error("User provided listener {} failed on invocation of onPartitionsAssigned for partitions {}",
- listener.get().getClass().getName(), assignedPartitions, e);
+ log.error(
+ "User provided listener {} failed on invocation of onPartitionsAssigned for partitions {}",
+ listener.get().getClass().getName(),
+ assignedPartitions,
+ e
+ );
return e;
}
}
@@ -77,11 +80,11 @@ public Exception invokePartitionsAssigned(final SortedSet assign
}
public Exception invokePartitionsRevoked(final SortedSet revokedPartitions) {
- log.info("Revoke previously assigned partitions {}", revokedPartitions.stream().map(TopicPartition::toString).collect(Collectors.joining(", ")));
+ log.info("Revoke previously assigned partitions {}", revokedPartitions);
Set revokePausedPartitions = subscriptions.pausedPartitions();
revokePausedPartitions.retainAll(revokedPartitions);
if (!revokePausedPartitions.isEmpty())
- log.info("The pause flag in partitions [{}] will be removed due to revocation.", revokePausedPartitions.stream().map(TopicPartition::toString).collect(Collectors.joining(", ")));
+ log.info("The pause flag in partitions {} will be removed due to revocation.", revokePausedPartitions);
Optional listener = subscriptions.rebalanceListener();
@@ -93,8 +96,12 @@ public Exception invokePartitionsRevoked(final SortedSet revoked
} catch (WakeupException | InterruptException e) {
throw e;
} catch (Exception e) {
- log.error("User provided listener {} failed on invocation of onPartitionsRevoked for partitions {}",
- listener.get().getClass().getName(), revokedPartitions, e);
+ log.error(
+ "User provided listener {} failed on invocation of onPartitionsRevoked for partitions {}",
+ listener.get().getClass().getName(),
+ revokedPartitions,
+ e
+ );
return e;
}
}
@@ -103,11 +110,11 @@ public Exception invokePartitionsRevoked(final SortedSet revoked
}
public Exception invokePartitionsLost(final SortedSet lostPartitions) {
- log.info("Lost previously assigned partitions {}", lostPartitions.stream().map(TopicPartition::toString).collect(Collectors.joining(", ")));
+ log.info("Lost previously assigned partitions {}", lostPartitions);
Set lostPausedPartitions = subscriptions.pausedPartitions();
lostPausedPartitions.retainAll(lostPartitions);
if (!lostPausedPartitions.isEmpty())
- log.info("The pause flag in partitions [{}] will be removed due to partition lost.", lostPartitions.stream().map(TopicPartition::toString).collect(Collectors.joining(", ")));
+ log.info("The pause flag in partitions {} will be removed due to partition lost.", lostPartitions);
Optional listener = subscriptions.rebalanceListener();
@@ -119,8 +126,12 @@ public Exception invokePartitionsLost(final SortedSet lostPartit
} catch (WakeupException | InterruptException e) {
throw e;
} catch (Exception e) {
- log.error("User provided listener {} failed on invocation of onPartitionsLost for partitions {}",
- listener.get().getClass().getName(), lostPartitions, e);
+ log.error(
+ "User provided listener {} failed on invocation of onPartitionsLost for partitions {}",
+ listener.get().getClass().getName(),
+ lostPartitions,
+ e
+ );
return e;
}
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchCollector.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchCollector.java
index 2aa8aeaaffbad..bbe216c2fc837 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchCollector.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchCollector.java
@@ -155,7 +155,10 @@ private Fetch fetchRecords(final CompletedFetch nextInLineFetch, int maxRe
log.debug("Not returning fetched records for partition {} since it is no longer assigned", tp);
} else if (!subscriptions.isFetchable(tp)) {
// this can happen when a partition is paused before fetched records are returned to the consumer's
- // poll call or if the offset is being reset
+ // poll call or if the offset is being reset.
+ // It can also happen under the Consumer rebalance protocol, when the consumer changes its subscription.
+ // Until the consumer receives an updated assignment from the coordinator, it can hold assigned partitions
+ // that are not in the subscription anymore, so we make them not fetchable.
log.debug("Not returning fetched records for assigned partition {} since it is no longer fetchable", tp);
} else {
SubscriptionState.FetchPosition position = subscriptions.position(tp);
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
index e237165f5b771..4659f4cf0cd5a 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
@@ -25,6 +25,7 @@
import org.apache.kafka.clients.consumer.SubscriptionPattern;
import org.apache.kafka.common.IsolationLevel;
import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.internals.PartitionStates;
import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset;
import org.apache.kafka.common.utils.LogContext;
@@ -91,6 +92,13 @@ private enum SubscriptionType {
/* the list of topics the user has requested */
private Set subscription;
+ /**
+ * Topic IDs received in an assignment from the coordinator when using the Consumer rebalance protocol.
+ * This will be used to include assigned topic IDs in metadata requests when the consumer
+ * does not know the topic names (ex. when the user subscribes to a RE2J regex computed on the broker)
+ */
+ private Set assignedTopicIds;
+
/* The list of topics the group has subscribed to. This may include some topics which are not part
* of `subscription` for the leader of a group since it is responsible for detecting metadata changes
* which require a group rebalance. */
@@ -149,6 +157,7 @@ public SubscriptionState(LogContext logContext, AutoOffsetResetStrategy defaultR
this.log = logContext.logger(this.getClass());
this.defaultResetStrategy = defaultResetStrategy;
this.subscription = new TreeSet<>(); // use a sorted set for better logging
+ this.assignedTopicIds = new TreeSet<>();
this.assignment = new PartitionStates<>();
this.groupSubscription = new HashSet<>();
this.subscribedPattern = null;
@@ -338,6 +347,7 @@ public synchronized void unsubscribe() {
this.subscription = Collections.emptySet();
this.groupSubscription = Collections.emptySet();
this.assignment.clear();
+ this.assignedTopicIds = Collections.emptySet();
this.subscribedPattern = null;
this.subscriptionType = SubscriptionType.NONE;
this.assignmentId++;
@@ -477,7 +487,7 @@ public synchronized List fetchablePartitions(Predicate result = new ArrayList<>();
assignment.forEach((topicPartition, topicPartitionState) -> {
// Cheap check is first to avoid evaluating the predicate if possible
- if ((subscriptionType.equals(SubscriptionType.AUTO_TOPICS_SHARE) || topicPartitionState.isFetchable())
+ if ((subscriptionType.equals(SubscriptionType.AUTO_TOPICS_SHARE) || isFetchableAndSubscribed(topicPartition, topicPartitionState))
&& isAvailable.test(topicPartition)) {
result.add(topicPartition);
}
@@ -485,23 +495,34 @@ public synchronized List fetchablePartitions(Predicate assignedTopicIds() {
+ return assignedTopicIds;
+ }
+
+ /**
+ * Set the set of topic IDs that have been assigned to the consumer by the coordinator.
+ * This is used for topic IDs received in an assignment when using the new consumer rebalance protocol (KIP-848).
+ */
+ public synchronized void setAssignedTopicIds(Set assignedTopicIds) {
+ this.assignedTopicIds = assignedTopicIds;
+ }
+
/**
* Enable fetching and updating positions for the given partitions that were assigned to the
* consumer, but waiting for the onPartitionsAssigned callback to complete. This is
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicIdPartitionSet.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicIdPartitionSet.java
index 920fb63515df3..6bf708d9354e7 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicIdPartitionSet.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicIdPartitionSet.java
@@ -61,15 +61,24 @@ public TopicIdPartitionSet() {
}
/**
- * Add a new topic (id+name) and partition. This will keep it, and also save references to the topic ID, topic name and partition.
+ * Add a single partition to the assignment, along with its topic ID and name.
+ * This will keep it, and also save references to the topic ID, topic name and partition.
+ * Visible for testing.
*/
- public void add(TopicIdPartition topicIdPartition) {
+ void add(TopicIdPartition topicIdPartition) {
topicIdPartitions.add(topicIdPartition);
topicPartitions.add(topicIdPartition.topicPartition());
topicIds.add(topicIdPartition.topicId());
topicNames.add(topicIdPartition.topicPartition().topic());
}
+ /**
+ * Add a set of partitions to the assignment, along with the topic ID and name.
+ */
+ public void addAll(Uuid topicId, String topicName, Set partitions) {
+ partitions.forEach(tp -> add(new TopicIdPartition(topicId, tp, topicName)));
+ }
+
public boolean isEmpty() {
return this.topicIdPartitions.isEmpty();
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java
index 2d9cab0dd9686..fcef3ce2647af 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java
@@ -33,15 +33,16 @@
import org.slf4j.Logger;
+import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
+import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
-import java.util.stream.Collectors;
import static org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.PollResult.EMPTY;
@@ -84,16 +85,23 @@ public TopicMetadataRequestManager(final LogContext context, final Time time, fi
@Override
public NetworkClientDelegate.PollResult poll(final long currentTimeMs) {
// Prune any requests which have timed out
- List expiredRequests = inflightRequests.stream()
- .filter(TimedRequestState::isExpired)
- .collect(Collectors.toList());
- expiredRequests.forEach(TopicMetadataRequestState::expire);
+ Iterator requestStateIterator = inflightRequests.iterator();
- List requests = inflightRequests.stream()
- .map(req -> req.send(currentTimeMs))
- .filter(Optional::isPresent)
- .map(Optional::get)
- .collect(Collectors.toList());
+ while (requestStateIterator.hasNext()) {
+ TopicMetadataRequestState requestState = requestStateIterator.next();
+
+ if (requestState.isExpired()) {
+ requestState.expire();
+ requestStateIterator.remove();
+ }
+ }
+
+ List requests = new ArrayList<>();
+
+ for (TopicMetadataRequestState request : inflightRequests) {
+ Optional unsentRequest = request.send(currentTimeMs);
+ unsentRequest.ifPresent(requests::add);
+ }
return requests.isEmpty() ? EMPTY : new NetworkClientDelegate.PollResult(0, requests);
}
@@ -181,7 +189,9 @@ private Optional send(final long currentTim
}
private void expire() {
- completeFutureAndRemoveRequest(
+ // The request state is removed from inflightRequests via an iterator by the caller of this method,
+ // so don't remove it from inflightRequests here.
+ future.completeExceptionally(
new TimeoutException("Timeout expired while fetching topic metadata"));
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java
index 8d6564083ff94..853c5484df5be 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java
@@ -310,7 +310,7 @@ private void process(final AssignmentChangeEvent event) {
manager.updateTimerAndMaybeCommit(event.currentTimeMs());
}
- log.info("Assigned to partition(s): {}", event.partitions().stream().map(TopicPartition::toString).collect(Collectors.joining(", ")));
+ log.info("Assigned to partition(s): {}", event.partitions());
try {
if (subscriptions.assignFromUser(new HashSet<>(event.partitions())))
metadata.requestUpdateForNewTopics();
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java
index 5a0358df8964f..b4440de06264b 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java
@@ -25,11 +25,10 @@
import java.time.Duration;
import java.util.ArrayList;
import java.util.Collection;
+import java.util.Iterator;
import java.util.List;
import java.util.Objects;
import java.util.concurrent.CompletableFuture;
-import java.util.function.Consumer;
-import java.util.stream.Collectors;
/**
* {@code CompletableEventReaper} is responsible for tracking {@link CompletableEvent time-bound events} and removing
@@ -85,26 +84,39 @@ public void add(CompletableEvent> event) {
* @return The number of events that were expired
*/
public long reap(long currentTimeMs) {
- Consumer> expireEvent = event -> {
- long pastDueMs = currentTimeMs - event.deadlineMs();
- TimeoutException error = new TimeoutException(String.format("%s was %s ms past its expiration of %s", event.getClass().getSimpleName(), pastDueMs, event.deadlineMs()));
+ int count = 0;
+ Iterator> iterator = tracked.iterator();
+
+ while (iterator.hasNext()) {
+ CompletableEvent> event = iterator.next();
+
+ if (event.future().isDone()) {
+ // Remove any events that are already complete.
+ iterator.remove();
+ continue;
+ }
+
+ long deadlineMs = event.deadlineMs();
+ long pastDueMs = currentTimeMs - deadlineMs;
+
+ if (pastDueMs < 0)
+ continue;
+
+ TimeoutException error = new TimeoutException(String.format("%s was %s ms past its expiration of %s", event.getClass().getSimpleName(), pastDueMs, deadlineMs));
+
+ // Complete (exceptionally) any events that have passed their deadline AND aren't already complete.
if (event.future().completeExceptionally(error)) {
- log.debug("Event {} completed exceptionally since its expiration of {} passed {} ms ago", event, event.deadlineMs(), pastDueMs);
+ log.debug("Event {} completed exceptionally since its expiration of {} passed {} ms ago", event, deadlineMs, pastDueMs);
} else {
log.trace("Event {} not completed exceptionally since it was previously completed", event);
}
- };
-
- // First, complete (exceptionally) any events that have passed their deadline AND aren't already complete.
- long count = tracked.stream()
- .filter(e -> !e.future().isDone())
- .filter(e -> currentTimeMs >= e.deadlineMs())
- .peek(expireEvent)
- .count();
- // Second, remove any events that are already complete, just to make sure we don't hold references. This will
- // include any events that finished successfully as well as any events we just completed exceptionally above.
- tracked.removeIf(e -> e.future().isDone());
+
+ count++;
+
+ // Remove the events so that we don't hold a reference to it.
+ iterator.remove();
+ }
return count;
}
@@ -131,29 +143,12 @@ public long reap(long currentTimeMs) {
public long reap(Collection> events) {
Objects.requireNonNull(events, "Event queue to reap must be non-null");
- Consumer> expireEvent = event -> {
- TimeoutException error = new TimeoutException(String.format("%s could not be completed before the consumer closed", event.getClass().getSimpleName()));
-
- if (event.future().completeExceptionally(error)) {
- log.debug("Event {} completed exceptionally since the consumer is closing", event);
- } else {
- log.trace("Event {} not completed exceptionally since it was completed prior to the consumer closing", event);
- }
- };
-
- long trackedExpiredCount = tracked.stream()
- .filter(e -> !e.future().isDone())
- .peek(expireEvent)
- .count();
+ long trackedExpiredCount = completeEventsExceptionallyOnClose(tracked);
tracked.clear();
- long eventExpiredCount = events.stream()
- .filter(e -> e instanceof CompletableEvent>)
- .map(e -> (CompletableEvent>) e)
- .filter(e -> !e.future().isDone())
- .peek(expireEvent)
- .count();
+ long eventExpiredCount = completeEventsExceptionallyOnClose(events);
events.clear();
+
return trackedExpiredCount + eventExpiredCount;
}
@@ -166,9 +161,51 @@ public boolean contains(CompletableEvent> event) {
}
public List> uncompletedEvents() {
- return tracked.stream()
- .filter(e -> !e.future().isDone())
- .collect(Collectors.toList());
+ // The following code does not use the Java Collections Streams API to reduce overhead in the critical
+ // path of the ConsumerNetworkThread loop.
+ List> events = new ArrayList<>();
+
+ for (CompletableEvent> event : tracked) {
+ if (!event.future().isDone())
+ events.add(event);
+ }
+
+ return events;
+ }
+
+ /**
+ * For all the {@link CompletableEvent}s in the collection, if they're not already complete, invoke
+ * {@link CompletableFuture#completeExceptionally(Throwable)}.
+ *
+ * @param events Collection of objects, assumed to be subclasses of {@link ApplicationEvent} or
+ * {@link BackgroundEvent}, but will only perform completion for any
+ * unfinished {@link CompletableEvent}s
+ *
+ * @return Number of events closed
+ */
+ private long completeEventsExceptionallyOnClose(Collection> events) {
+ long count = 0;
+
+ for (Object o : events) {
+ if (!(o instanceof CompletableEvent))
+ continue;
+
+ CompletableEvent> event = (CompletableEvent>) o;
+
+ if (event.future().isDone())
+ continue;
+
+ count++;
+
+ TimeoutException error = new TimeoutException(String.format("%s could not be completed before the consumer closed", event.getClass().getSimpleName()));
+
+ if (event.future().completeExceptionally(error)) {
+ log.debug("Event {} completed exceptionally since the consumer is closing", event);
+ } else {
+ log.trace("Event {} not completed exceptionally since it was completed prior to the consumer closing", event);
+ }
+ }
+
+ return count;
}
-
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
index 71d201b71f949..1b1d7bb7e832c 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
@@ -50,13 +50,11 @@
import org.apache.kafka.common.errors.AuthorizationException;
import org.apache.kafka.common.errors.InterruptException;
import org.apache.kafka.common.errors.InvalidTopicException;
-import org.apache.kafka.common.errors.InvalidTxnStateException;
import org.apache.kafka.common.errors.ProducerFencedException;
import org.apache.kafka.common.errors.RecordTooLargeException;
import org.apache.kafka.common.errors.RetriableException;
import org.apache.kafka.common.errors.SerializationException;
import org.apache.kafka.common.errors.TimeoutException;
-import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.header.Header;
import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.header.internals.RecordHeaders;
@@ -622,13 +620,8 @@ private TransactionManager configureTransactionState(ProducerConfig config,
}
/**
- * Initialize the transactional state for this producer, similar to {@link #initTransactions()} but
- * with additional capabilities to keep a previously prepared transaction.
- *
* Needs to be called before any other methods when the {@code transactional.id} is set in the configuration.
- *
- * When {@code keepPreparedTxn} is {@code false}, this behaves like the standard transactional
- * initialization where the method does the following:
+ * This method does the following:
*
* - Ensures any transactions initiated by previous instances of the producer with the same
* {@code transactional.id} are completed. If the previous instance had failed with a transaction in
@@ -637,39 +630,26 @@ private TransactionManager configureTransactionState(ProducerConfig config,
*
- Gets the internal producer id and epoch, used in all future transactional
* messages issued by the producer.
*
- *
- *
- * When {@code keepPreparedTxn} is set to {@code true}, the producer does not automatically abort existing
- * transactions. Instead, it enters a recovery mode allowing only finalization of those previously
- * prepared transactions.
- * This behavior is especially crucial for 2PC scenarios, where transactions should remain intact
- * until the external transaction manager decides whether to commit or abort.
- *
- *
- * @param keepPreparedTxn true to retain any in-flight prepared transactions (necessary for 2PC
- * recovery), false to abort existing transactions and behave like
- * the standard initTransactions.
- *
* Note that this method will raise {@link TimeoutException} if the transactional state cannot
* be initialized before expiration of {@code max.block.ms}. Additionally, it will raise {@link InterruptException}
* if interrupted. It is safe to retry in either case, but once the transactional state has been successfully
* initialized, this method should no longer be used.
*
- * @throws IllegalStateException if no {@code transactional.id} is configured
- * @throws org.apache.kafka.common.errors.UnsupportedVersionException if the broker does not
- * support transactions (i.e. if its version is lower than 0.11.0.0)
- * @throws org.apache.kafka.common.errors.TransactionalIdAuthorizationException if the configured
- * {@code transactional.id} is unauthorized either for normal transaction writes or 2PC.
- * @throws KafkaException if the producer encounters a fatal error or any other unexpected error
+ * @throws IllegalStateException if no {@code transactional.id} has been configured
+ * @throws org.apache.kafka.common.errors.UnsupportedVersionException fatal error indicating the broker
+ * does not support transactions (i.e. if its version is lower than 0.11.0.0)
+ * @throws org.apache.kafka.common.errors.AuthorizationException error indicating that the configured
+ * transactional.id is not authorized, or the idempotent producer id is unavailable. See the exception for
+ * more details. User may retry this function call after fixing the permission.
+ * @throws KafkaException if the producer has encountered a previous fatal error or for any other unexpected error
* @throws TimeoutException if the time taken for initialize the transaction has surpassed max.block.ms
.
* @throws InterruptException if the thread is interrupted while blocked
*/
- public void initTransactions(boolean keepPreparedTxn) {
+ public void initTransactions() {
throwIfNoTransactionManager();
throwIfProducerClosed();
- throwIfInPreparedState();
long now = time.nanoseconds();
- TransactionalRequestResult result = transactionManager.initializeTransactions(keepPreparedTxn);
+ TransactionalRequestResult result = transactionManager.initializeTransactions(false);
sender.wakeup();
result.await(maxBlockTimeMs, TimeUnit.MILLISECONDS);
producerMetrics.recordInit(time.nanoseconds() - now);
@@ -754,7 +734,6 @@ public void sendOffsetsToTransaction(Map offs
throwIfInvalidGroupMetadata(groupMetadata);
throwIfNoTransactionManager();
throwIfProducerClosed();
- throwIfInPreparedState();
if (!offsets.isEmpty()) {
long start = time.nanoseconds();
@@ -765,48 +744,6 @@ public void sendOffsetsToTransaction(Map offs
}
}
- /**
- * Prepares the current transaction for a two-phase commit. This method will flush all pending messages
- * and transition the producer into a mode where only {@link #commitTransaction()}, {@link #abortTransaction()},
- * or completeTransaction(PreparedTxnState) may be called.
- *
- * This method is used as part of a two-phase commit protocol:
- *
- * - Prepare the transaction by calling this method. This returns a {@link PreparedTxnState} if successful.
- * - Make any external system changes that need to be atomic with this transaction.
- * - Complete the transaction by calling {@link #commitTransaction()}, {@link #abortTransaction()} or
- * completeTransaction(PreparedTxnState).
- *
- *
- * @return the prepared transaction state to use when completing the transaction
- *
- * @throws IllegalStateException if no transactional.id has been configured or no transaction has been started yet.
- * @throws InvalidTxnStateException if the producer is not in a state where preparing
- * a transaction is possible or 2PC is not enabled.
- * @throws ProducerFencedException fatal error indicating another producer with the same transactional.id is active
- * @throws UnsupportedVersionException fatal error indicating the broker
- * does not support transactions (i.e. if its version is lower than 0.11.0.0)
- * @throws AuthorizationException fatal error indicating that the configured
- * transactional.id is not authorized. See the exception for more details
- * @throws KafkaException if the producer has encountered a previous fatal error or for any other unexpected error
- * @throws TimeoutException if the time taken for preparing the transaction has surpassed max.block.ms
- * @throws InterruptException if the thread is interrupted while blocked
- */
- @Override
- public PreparedTxnState prepareTransaction() throws ProducerFencedException {
- throwIfNoTransactionManager();
- throwIfProducerClosed();
- throwIfInPreparedState();
- if (!transactionManager.is2PCEnabled()) {
- throw new InvalidTxnStateException("Cannot prepare a transaction when 2PC is not enabled");
- }
- long now = time.nanoseconds();
- flush();
- transactionManager.prepareTransaction();
- producerMetrics.recordPrepareTxn(time.nanoseconds() - now);
- return transactionManager.preparedTransactionState();
- }
-
/**
* Commits the ongoing transaction. This method will flush any unsent records before actually committing the transaction.
*
@@ -884,40 +821,6 @@ public void abortTransaction() throws ProducerFencedException {
producerMetrics.recordAbortTxn(time.nanoseconds() - abortStart);
}
- /**
- * Completes a prepared transaction by comparing the provided prepared transaction state with the
- * current prepared state on the producer.
- * If they match, the transaction is committed; otherwise, it is aborted.
- *
- * @param preparedTxnState The prepared transaction state to compare against the current state
- * @throws IllegalStateException if no transactional.id has been configured or no transaction has been started
- * @throws InvalidTxnStateException if the producer is not in prepared state
- * @throws ProducerFencedException fatal error indicating another producer with the same transactional.id is active
- * @throws KafkaException if the producer has encountered a previous fatal error or for any other unexpected error
- * @throws TimeoutException if the time taken for completing the transaction has surpassed max.block.ms
- * @throws InterruptException if the thread is interrupted while blocked
- */
- @Override
- public void completeTransaction(PreparedTxnState preparedTxnState) throws ProducerFencedException {
- throwIfNoTransactionManager();
- throwIfProducerClosed();
-
- if (!transactionManager.isPrepared()) {
- throw new InvalidTxnStateException("Cannot complete transaction because no transaction has been prepared. " +
- "Call prepareTransaction() first, or make sure initTransaction(true) was called.");
- }
-
- // Get the current prepared transaction state
- PreparedTxnState currentPreparedState = transactionManager.preparedTransactionState();
-
- // Compare the prepared transaction state token and commit or abort accordingly
- if (currentPreparedState.equals(preparedTxnState)) {
- commitTransaction();
- } else {
- abortTransaction();
- }
- }
-
/**
* Asynchronously send a record to a topic. Equivalent to send(record, null)
.
* See {@link #send(ProducerRecord, Callback)} for details.
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java
index 3e5cb9f5d5ab3..a4aac86df09fc 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java
@@ -142,7 +142,7 @@ public MockProducer() {
}
@Override
- public void initTransactions(boolean keepPreparedTxn) {
+ public void initTransactions() {
verifyNotClosed();
verifyNotFenced();
if (this.transactionInitialized) {
@@ -200,18 +200,6 @@ public void sendOffsetsToTransaction(Map offs
this.sentOffsets = true;
}
- @Override
- public PreparedTxnState prepareTransaction() throws ProducerFencedException {
- verifyNotClosed();
- verifyNotFenced();
- verifyTransactionsInitialized();
- verifyTransactionInFlight();
-
- // Return a new PreparedTxnState with mock values for producerId and epoch
- // Using 1000L and (short)1 as arbitrary values for a valid PreparedTxnState
- return new PreparedTxnState(1000L, (short) 1);
- }
-
@Override
public void commitTransaction() throws ProducerFencedException {
verifyNotClosed();
@@ -257,27 +245,6 @@ public void abortTransaction() throws ProducerFencedException {
this.transactionInFlight = false;
}
- @Override
- public void completeTransaction(PreparedTxnState preparedTxnState) throws ProducerFencedException {
- verifyNotClosed();
- verifyNotFenced();
- verifyTransactionsInitialized();
-
- if (!this.transactionInFlight) {
- throw new IllegalStateException("There is no prepared transaction to complete.");
- }
-
- // For testing purposes, we'll consider a prepared state with producerId=1000L and epoch=1 as valid
- // This should match what's returned in prepareTransaction()
- PreparedTxnState currentState = new PreparedTxnState(1000L, (short) 1);
-
- if (currentState.equals(preparedTxnState)) {
- commitTransaction();
- } else {
- abortTransaction();
- }
- }
-
private synchronized void verifyNotClosed() {
if (this.closed) {
throw new IllegalStateException("MockProducer is already closed.");
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java b/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java
index e6e94691e3454..798034dda6de2 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java
@@ -42,14 +42,7 @@ public interface Producer extends Closeable {
/**
* See {@link KafkaProducer#initTransactions()}
*/
- default void initTransactions() {
- initTransactions(false);
- }
-
- /**
- * See {@link KafkaProducer#initTransactions(boolean)}
- */
- void initTransactions(boolean keepPreparedTxn);
+ void initTransactions();
/**
* See {@link KafkaProducer#beginTransaction()}
@@ -62,11 +55,6 @@ default void initTransactions() {
void sendOffsetsToTransaction(Map offsets,
ConsumerGroupMetadata groupMetadata) throws ProducerFencedException;
- /**
- * See {@link KafkaProducer#prepareTransaction()}
- */
- PreparedTxnState prepareTransaction() throws ProducerFencedException;
-
/**
* See {@link KafkaProducer#commitTransaction()}
*/
@@ -77,11 +65,6 @@ void sendOffsetsToTransaction(Map offsets,
*/
void abortTransaction() throws ProducerFencedException;
- /**
- * See {@link KafkaProducer#completeTransaction(PreparedTxnState)}
- */
- void completeTransaction(PreparedTxnState preparedTxnState) throws ProducerFencedException;
-
/**
* @see KafkaProducer#registerMetricForSubscription(KafkaMetric)
*/
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
index 876555d232a88..64e8646d6f153 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
@@ -565,7 +565,7 @@ private boolean awaitNodeReady(Node node, FindCoordinatorRequest.CoordinatorType
/**
* Handle a produce response
*/
- private void handleProduceResponse(ClientResponse response, Map batches, long now) {
+ private void handleProduceResponse(ClientResponse response, Map batches, Map topicNames, long now) {
RequestHeader requestHeader = response.requestHeader();
int correlationId = requestHeader.correlationId();
if (response.wasTimedOut()) {
@@ -595,9 +595,6 @@ private void handleProduceResponse(ClientResponse response, Map partitionsWithUpdatedLeaderInfo = new HashMap<>();
produceResponse.data().responses().forEach(r -> r.partitionResponses().forEach(p -> {
- // Version 13 drop topic name and add support to topic id. However, metadata can be used to map topic id to topic name.
- String topicName = metadata.topicNames().getOrDefault(r.topicId(), r.name());
- TopicPartition tp = new TopicPartition(topicName, p.index());
ProduceResponse.PartitionResponse partResp = new ProduceResponse.PartitionResponse(
Errors.forCode(p.errorCode()),
p.baseOffset(),
@@ -609,7 +606,20 @@ private void handleProduceResponse(ClientResponse response, Map handleProduceResponse(response, recordsByPartition, time.milliseconds());
+ // Fetch topic names from metadata outside callback as topic ids may change during the callback
+ // for example if topic was recreated.
+ Map topicNames = metadata.topicNames();
+
+ RequestCompletionHandler callback = response -> handleProduceResponse(response, recordsByPartition, topicNames, time.milliseconds());
String nodeId = Integer.toString(destination);
ClientRequest clientRequest = client.newClientRequest(nodeId, requestBuilder, now, acks != 0,
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java
index 5d83cbc0b1b68..20804c505dd7e 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java
@@ -320,9 +320,7 @@ synchronized TransactionalRequestResult initializeTransactions(
.setTransactionalId(transactionalId)
.setTransactionTimeoutMs(transactionTimeoutMs)
.setProducerId(producerIdAndEpoch.producerId)
- .setProducerEpoch(producerIdAndEpoch.epoch)
- .setEnable2Pc(enable2PC)
- .setKeepPreparedTxn(keepPreparedTxn);
+ .setProducerEpoch(producerIdAndEpoch.epoch);
InitProducerIdHandler handler = new InitProducerIdHandler(new InitProducerIdRequest.Builder(requestData),
isEpochBump);
diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java
index 3fcea1968b8e1..77c476b49462d 100755
--- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java
@@ -169,16 +169,20 @@ public class TopicConfig {
"to trigger the unclean leader election immediately if needed.
";
public static final String MIN_IN_SYNC_REPLICAS_CONFIG = "min.insync.replicas";
- public static final String MIN_IN_SYNC_REPLICAS_DOC = "When a producer sets acks to \"all\" (or \"-1\"), " +
- "this configuration specifies the minimum number of replicas that must acknowledge " +
- "a write for the write to be considered successful. If this minimum cannot be met, " +
- "then the producer will raise an exception (either NotEnoughReplicas
or NotEnoughReplicasAfterAppend
).
" +
+ public static final String MIN_IN_SYNC_REPLICAS_DOC = "Specifies the minimum number of in-sync replicas (including the leader) " +
+ "required for a write to succeed when a producer sets acks
to \"all\" (or \"-1\"). In the acks=all
" +
+ "case, every in-sync replica must acknowledge a write for it to be considered successful. E.g., if a topic has " +
+ "replication.factor
of 3 and the ISR set includes all three replicas, then all three replicas must acknowledge an " +
+ "acks=all
write for it to succeed, even if min.insync.replicas
happens to be less than 3. " +
+ "If acks=all
and the current ISR set contains fewer than min.insync.replicas
members, then the producer " +
+ "will raise an exception (either NotEnoughReplicas
or NotEnoughReplicasAfterAppend
).
" +
"Regardless of the acks
setting, the messages will not be visible to the consumers until " +
"they are replicated to all in-sync replicas and the min.insync.replicas
condition is met.
" +
"When used together, min.insync.replicas
and acks
allow you to enforce greater durability guarantees. " +
"A typical scenario would be to create a topic with a replication factor of 3, " +
"set min.insync.replicas
to 2, and produce with acks
of \"all\". " +
- "This will ensure that a majority of replicas must persist a write before it's considered successful by the producer and it's visible to consumers.";
+ "This ensures that a majority of replicas must persist a write before it's considered successful by the producer and it's visible to consumers." +
+ "Note that when the Eligible Leader Replicas feature is enabled, the semantics of this config changes. Please refer to the ELR section for more info.
";
public static final String COMPRESSION_TYPE_CONFIG = "compression.type";
public static final String COMPRESSION_TYPE_DOC = "Specify the final compression type for a given topic. " +
diff --git a/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java b/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java
index 4ec7db604bcbe..2f5e2e50dde75 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java
@@ -211,6 +211,10 @@ public void flush() throws IOException {
* Close this record set
*/
public void close() throws IOException {
+ if (!channel.isOpen()) {
+ return;
+ }
+
flush();
trim();
channel.close();
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java
index e0a213fa62ef8..d3dcabfb4f9c5 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java
@@ -27,7 +27,9 @@
import java.util.Collection;
import java.util.Collections;
+import java.util.HashSet;
import java.util.List;
+import java.util.Set;
import java.util.stream.Collectors;
public class MetadataRequest extends AbstractRequest {
@@ -49,33 +51,33 @@ public Builder(List topics, boolean allowAutoTopicCreation, short allowe
public Builder(List topics, boolean allowAutoTopicCreation, short minVersion, short maxVersion) {
super(ApiKeys.METADATA, minVersion, maxVersion);
+ this.data = requestTopicNamesOrAllTopics(topics, allowAutoTopicCreation);
+ }
+
+ private MetadataRequestData requestTopicNamesOrAllTopics(List topics, boolean allowAutoTopicCreation) {
MetadataRequestData data = new MetadataRequestData();
if (topics == null)
data.setTopics(null);
else {
topics.forEach(topic -> data.topics().add(new MetadataRequestTopic().setName(topic)));
}
-
data.setAllowAutoTopicCreation(allowAutoTopicCreation);
- this.data = data;
+ return data;
}
- public Builder(List topics, boolean allowAutoTopicCreation) {
- this(topics, allowAutoTopicCreation, ApiKeys.METADATA.oldestVersion(), ApiKeys.METADATA.latestVersion());
- }
-
- public Builder(List topicIds) {
- super(ApiKeys.METADATA, ApiKeys.METADATA.oldestVersion(), ApiKeys.METADATA.latestVersion());
+ private static MetadataRequestData requestTopicIds(Set topicIds) {
MetadataRequestData data = new MetadataRequestData();
if (topicIds == null)
data.setTopics(null);
else {
topicIds.forEach(topicId -> data.topics().add(new MetadataRequestTopic().setTopicId(topicId)));
}
+ data.setAllowAutoTopicCreation(false); // can't auto-create without topic name
+ return data;
+ }
- // It's impossible to create topic with topicId
- data.setAllowAutoTopicCreation(false);
- this.data = data;
+ public Builder(List topics, boolean allowAutoTopicCreation) {
+ this(topics, allowAutoTopicCreation, ApiKeys.METADATA.oldestVersion(), ApiKeys.METADATA.latestVersion());
}
public static Builder allTopics() {
@@ -84,6 +86,20 @@ public static Builder allTopics() {
return new Builder(ALL_TOPICS_REQUEST_DATA);
}
+ /**
+ * @return Builder for metadata request using topic names.
+ */
+ public static Builder forTopicNames(List topicNames, boolean allowAutoTopicCreation) {
+ return new MetadataRequest.Builder(topicNames, allowAutoTopicCreation);
+ }
+
+ /**
+ * @return Builder for metadata request using topic IDs.
+ */
+ public static Builder forTopicIds(Set topicIds) {
+ return new MetadataRequest.Builder(requestTopicIds(new HashSet<>(topicIds)));
+ }
+
public boolean emptyTopicList() {
return data.topics().isEmpty();
}
@@ -92,6 +108,13 @@ public boolean isAllTopics() {
return data.topics() == null;
}
+ public List topicIds() {
+ return data.topics()
+ .stream()
+ .map(MetadataRequestTopic::topicId)
+ .collect(Collectors.toList());
+ }
+
public List topics() {
return data.topics()
.stream()
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
index e76ea8f7f3b58..77297e96e6e6b 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
@@ -17,6 +17,7 @@
package org.apache.kafka.common.requests;
import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.message.OffsetFetchRequestData;
import org.apache.kafka.common.message.OffsetFetchResponseData;
import org.apache.kafka.common.message.OffsetFetchResponseData.OffsetFetchResponseGroup;
import org.apache.kafka.common.message.OffsetFetchResponseData.OffsetFetchResponsePartition;
@@ -99,27 +100,12 @@ public OffsetFetchResponse build(short version) {
data.topics().add(newTopic);
topic.partitions().forEach(partition -> {
- OffsetFetchResponsePartition newPartition;
-
- if (version < TOP_LEVEL_ERROR_AND_NULL_TOPICS_MIN_VERSION && group.errorCode() != Errors.NONE.code()) {
- // Versions prior to version 2 do not support a top level error. Therefore,
- // we put it at the partition level.
- newPartition = new OffsetFetchResponsePartition()
- .setPartitionIndex(partition.partitionIndex())
- .setErrorCode(group.errorCode())
- .setCommittedOffset(INVALID_OFFSET)
- .setMetadata(NO_METADATA)
- .setCommittedLeaderEpoch(NO_PARTITION_LEADER_EPOCH);
- } else {
- newPartition = new OffsetFetchResponsePartition()
- .setPartitionIndex(partition.partitionIndex())
- .setErrorCode(partition.errorCode())
- .setCommittedOffset(partition.committedOffset())
- .setMetadata(partition.metadata())
- .setCommittedLeaderEpoch(partition.committedLeaderEpoch());
- }
-
- newTopic.partitions().add(newPartition);
+ newTopic.partitions().add(new OffsetFetchResponsePartition()
+ .setPartitionIndex(partition.partitionIndex())
+ .setErrorCode(partition.errorCode())
+ .setCommittedOffset(partition.committedOffset())
+ .setMetadata(partition.metadata())
+ .setCommittedLeaderEpoch(partition.committedLeaderEpoch()));
});
});
}
@@ -239,4 +225,31 @@ public OffsetFetchResponseData data() {
public boolean shouldClientThrottle(short version) {
return version >= 4;
}
+
+ public static OffsetFetchResponseData.OffsetFetchResponseGroup groupError(
+ OffsetFetchRequestData.OffsetFetchRequestGroup group,
+ Errors error,
+ int version
+ ) {
+ if (version >= TOP_LEVEL_ERROR_AND_NULL_TOPICS_MIN_VERSION) {
+ return new OffsetFetchResponseData.OffsetFetchResponseGroup()
+ .setGroupId(group.groupId())
+ .setErrorCode(error.code());
+ } else {
+ return new OffsetFetchResponseData.OffsetFetchResponseGroup()
+ .setGroupId(group.groupId())
+ .setTopics(group.topics().stream().map(topic ->
+ new OffsetFetchResponseData.OffsetFetchResponseTopics()
+ .setName(topic.name())
+ .setPartitions(topic.partitionIndexes().stream().map(partition ->
+ new OffsetFetchResponseData.OffsetFetchResponsePartitions()
+ .setPartitionIndex(partition)
+ .setErrorCode(error.code())
+ .setCommittedOffset(INVALID_OFFSET)
+ .setMetadata(NO_METADATA)
+ .setCommittedLeaderEpoch(NO_PARTITION_LEADER_EPOCH)
+ ).collect(Collectors.toList()))
+ ).collect(Collectors.toList()));
+ }
+ }
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/ClientCredentialsJwtRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/ClientCredentialsJwtRetriever.java
index 627434f6d3c76..4744fd9128999 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/ClientCredentialsJwtRetriever.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/ClientCredentialsJwtRetriever.java
@@ -39,7 +39,10 @@
import static org.apache.kafka.common.config.SaslConfigs.DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE;
import static org.apache.kafka.common.config.SaslConfigs.SASL_JAAS_CONFIG;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_ID;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_SECRET;
import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_HEADER_URLENCODE;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_SCOPE;
import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_ID_CONFIG;
import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_SECRET_CONFIG;
import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.SCOPE_CONFIG;
@@ -173,8 +176,8 @@ private ConfigOrJaas(ConfigurationUtils cu, JaasOptionsUtils jou) {
private String clientId() {
return getValue(
+ SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_ID,
CLIENT_ID_CONFIG,
- "clientId",
true,
cu::validateString,
jou::validateString
@@ -183,8 +186,8 @@ private String clientId() {
private String clientSecret() {
return getValue(
+ SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_SECRET,
CLIENT_SECRET_CONFIG,
- "clientSecret",
true,
cu::validatePassword,
jou::validateString
@@ -193,8 +196,8 @@ private String clientSecret() {
private String scope() {
return getValue(
+ SASL_OAUTHBEARER_SCOPE,
SCOPE_CONFIG,
- "scope",
false,
cu::validateString,
jou::validateString
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandler.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandler.java
index 6563d36b8b663..60fa8cdb6788a 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandler.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandler.java
@@ -170,7 +170,7 @@ private void handleExtensionsValidatorCallback(OAuthBearerExtensionsValidatorCal
}
private void checkConfigured() {
- if (verificationKeyResolver == null || jwtValidator == null)
+ if (jwtValidator == null)
throw new IllegalStateException(String.format("To use %s, first call the configure method", getClass().getSimpleName()));
}
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtils.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtils.java
index a0819766a38ca..3eebecf8fde10 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtils.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtils.java
@@ -22,6 +22,9 @@
import org.apache.kafka.common.network.ListenerName;
import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
import java.io.File;
import java.net.MalformedURLException;
import java.net.URISyntaxException;
@@ -47,6 +50,8 @@
public class ConfigurationUtils {
+ private static final Logger LOG = LoggerFactory.getLogger(ConfigurationUtils.class);
+
private final Map configs;
private final String prefix;
@@ -344,6 +349,13 @@ public static T getConfiguredInstance(Map configs,
((OAuthBearerConfigurable) o).configure(configs, saslMechanism, jaasConfigEntries);
} catch (Exception e) {
Utils.maybeCloseQuietly(o, "Instance of class " + o.getClass().getName() + " failed call to configure()");
+ LOG.warn(
+ "The class {} defined in the {} configuration encountered an error on configure(): {}",
+ o.getClass().getName(),
+ configName,
+ e.getMessage(),
+ e
+ );
throw new ConfigException(
String.format(
"The class %s defined in the %s configuration encountered an error on configure(): %s",
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/DefaultAssertionCreator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/DefaultAssertionCreator.java
index db562fade8759..52b9eb2fb536c 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/DefaultAssertionCreator.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/DefaultAssertionCreator.java
@@ -16,7 +16,7 @@
*/
package org.apache.kafka.common.security.oauthbearer.internals.secured.assertion;
-import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.oauthbearer.JwtRetrieverException;
import org.apache.kafka.common.security.oauthbearer.internals.secured.CachedFile;
import org.apache.kafka.common.utils.Utils;
@@ -89,7 +89,7 @@ public PrivateKey transform(File file, String contents) {
return privateKey(contents.getBytes(StandardCharsets.UTF_8), passphrase);
} catch (GeneralSecurityException | IOException e) {
- throw new KafkaException("An error occurred generating the OAuth assertion private key from " + file.getPath(), e);
+ throw new JwtRetrieverException("An error occurred generating the OAuth assertion private key from " + file.getPath(), e);
}
}
}
diff --git a/clients/src/main/resources/common/message/FetchRequest.json b/clients/src/main/resources/common/message/FetchRequest.json
index b7ad185f60b39..b4dd880faed07 100644
--- a/clients/src/main/resources/common/message/FetchRequest.json
+++ b/clients/src/main/resources/common/message/FetchRequest.json
@@ -56,7 +56,9 @@
// Version 16 is the same as version 15 (KIP-951).
//
// Version 17 adds directory id support from KIP-853
- "validVersions": "4-17",
+ //
+ // Version 18 adds high-watermark from KIP-1166
+ "validVersions": "4-18",
"flexibleVersions": "12+",
"fields": [
{ "name": "ClusterId", "type": "string", "versions": "12+", "nullableVersions": "12+", "default": "null",
@@ -103,7 +105,10 @@
{ "name": "PartitionMaxBytes", "type": "int32", "versions": "0+",
"about": "The maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored." },
{ "name": "ReplicaDirectoryId", "type": "uuid", "versions": "17+", "taggedVersions": "17+", "tag": 0, "ignorable": true,
- "about": "The directory id of the follower fetching." }
+ "about": "The directory id of the follower fetching." },
+ { "name": "HighWatermark", "type": "int64", "versions": "18+", "default": "9223372036854775807", "taggedVersions": "18+",
+ "tag": 1, "ignorable": true,
+ "about": "The high-watermark known by the replica. -1 if the high-watermark is not known and 9223372036854775807 if the feature is not supported." }
]}
]},
{ "name": "ForgottenTopicsData", "type": "[]ForgottenTopic", "versions": "7+", "ignorable": false,
diff --git a/clients/src/main/resources/common/message/FetchResponse.json b/clients/src/main/resources/common/message/FetchResponse.json
index dc8d35175661f..36dc05ff60ca4 100644
--- a/clients/src/main/resources/common/message/FetchResponse.json
+++ b/clients/src/main/resources/common/message/FetchResponse.json
@@ -48,7 +48,9 @@
// Version 16 adds the 'NodeEndpoints' field (KIP-951).
//
// Version 17 no changes to the response (KIP-853).
- "validVersions": "4-17",
+ //
+ // Version 18 no changes to the response (KIP-1166)
+ "validVersions": "4-18",
"flexibleVersions": "12+",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true,
diff --git a/clients/src/main/resources/common/message/JoinGroupRequest.json b/clients/src/main/resources/common/message/JoinGroupRequest.json
index 41d7c1acbaefe..31afdb1a32ae8 100644
--- a/clients/src/main/resources/common/message/JoinGroupRequest.json
+++ b/clients/src/main/resources/common/message/JoinGroupRequest.json
@@ -18,8 +18,6 @@
"type": "request",
"listeners": ["broker"],
"name": "JoinGroupRequest",
- // Versions 0-1 were removed in Apache Kafka 4.0, Version 2 is the new baseline.
- //
// Version 1 adds RebalanceTimeoutMs. Version 2 and 3 are the same as version 1.
//
// Starting from version 4, the client needs to issue a second request to join group
@@ -34,7 +32,7 @@
// Version 8 adds the Reason field (KIP-800).
//
// Version 9 is the same as version 8.
- "validVersions": "2-9",
+ "validVersions": "0-9",
"flexibleVersions": "6+",
"fields": [
{ "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId",
diff --git a/clients/src/main/resources/common/message/JoinGroupResponse.json b/clients/src/main/resources/common/message/JoinGroupResponse.json
index 364309596eb95..d2f016f62f66c 100644
--- a/clients/src/main/resources/common/message/JoinGroupResponse.json
+++ b/clients/src/main/resources/common/message/JoinGroupResponse.json
@@ -17,8 +17,6 @@
"apiKey": 11,
"type": "response",
"name": "JoinGroupResponse",
- // Versions 0-1 were removed in Apache Kafka 4.0, Version 2 is the new baseline.
- //
// Version 1 is the same as version 0.
//
// Version 2 adds throttle time.
@@ -37,7 +35,7 @@
// Version 8 is the same as version 7.
//
// Version 9 adds the SkipAssignment field.
- "validVersions": "2-9",
+ "validVersions": "0-9",
"flexibleVersions": "6+",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true,
diff --git a/clients/src/main/resources/common/message/StreamsGroupDescribeResponse.json b/clients/src/main/resources/common/message/StreamsGroupDescribeResponse.json
index 5dff3d7bf4449..520a1ec4aee49 100644
--- a/clients/src/main/resources/common/message/StreamsGroupDescribeResponse.json
+++ b/clients/src/main/resources/common/message/StreamsGroupDescribeResponse.json
@@ -57,9 +57,9 @@
"fields": [
{ "name": "SubtopologyId", "type": "string", "versions": "0+",
"about": "String to uniquely identify the subtopology." },
- { "name": "SourceTopics", "type": "[]string", "versions": "0+",
+ { "name": "SourceTopics", "type": "[]string", "versions": "0+", "entityType": "topicName",
"about": "The topics the subtopology reads from." },
- { "name": "RepartitionSinkTopics", "type": "[]string", "versions": "0+",
+ { "name": "RepartitionSinkTopics", "type": "[]string", "versions": "0+", "entityType": "topicName",
"about": "The repartition topics the subtopology writes to." },
{ "name": "StateChangelogTopics", "type": "[]TopicInfo", "versions": "0+",
"about": "The set of state changelog topics associated with this subtopology. Created automatically." },
@@ -154,7 +154,7 @@
"about": "value of the config" }
]},
{ "name": "TopicInfo", "versions": "0+", "fields": [
- { "name": "Name", "type": "string", "versions": "0+",
+ { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName",
"about": "The name of the topic." },
{ "name": "Partitions", "type": "int32", "versions": "0+",
"about": "The number of partitions in the topic. Can be 0 if no specific number of partitions is enforced. Always 0 for changelog topics." },
diff --git a/clients/src/main/resources/common/message/StreamsGroupHeartbeatRequest.json b/clients/src/main/resources/common/message/StreamsGroupHeartbeatRequest.json
index 247c3a68f548f..8b63e037fc0a1 100644
--- a/clients/src/main/resources/common/message/StreamsGroupHeartbeatRequest.json
+++ b/clients/src/main/resources/common/message/StreamsGroupHeartbeatRequest.json
@@ -20,7 +20,7 @@
"name": "StreamsGroupHeartbeatRequest",
"validVersions": "0",
"flexibleVersions": "0+",
- // The StreamsGroupDescribeRequest API is added as part of KIP-1071 and is still under
+ // The StreamsGroupHeartbeatRequest API is added as part of KIP-1071 and is still under
// development. Hence, the API is not exposed by default by brokers unless
// explicitly enabled.
"latestVersionUnstable": true,
@@ -50,13 +50,13 @@
"fields": [
{ "name": "SubtopologyId", "type": "string", "versions": "0+",
"about": "String to uniquely identify the subtopology. Deterministically generated from the topology" },
- { "name": "SourceTopics", "type": "[]string", "versions": "0+",
+ { "name": "SourceTopics", "type": "[]string", "versions": "0+", "entityType": "topicName",
"about": "The topics the topology reads from." },
{ "name": "SourceTopicRegex", "type": "[]string", "versions": "0+",
"about": "The regular expressions identifying topics the subtopology reads from." },
{ "name": "StateChangelogTopics", "type": "[]TopicInfo", "versions": "0+",
"about": "The set of state changelog topics associated with this subtopology. Created automatically." },
- { "name": "RepartitionSinkTopics", "type": "[]string", "versions": "0+",
+ { "name": "RepartitionSinkTopics", "type": "[]string", "versions": "0+", "entityType": "topicName",
"about": "The repartition topics the subtopology writes to." },
{ "name": "RepartitionSourceTopics", "type": "[]TopicInfo", "versions": "0+",
"about": "The set of source topics that are internally created repartition topics. Created automatically." },
@@ -104,7 +104,7 @@
"about": "value of the config" }
]},
{ "name": "TopicInfo", "versions": "0+", "fields": [
- { "name": "Name", "type": "string", "versions": "0+",
+ { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName",
"about": "The name of the topic." },
{ "name": "Partitions", "type": "int32", "versions": "0+",
"about": "The number of partitions in the topic. Can be 0 if no specific number of partitions is enforced. Always 0 for changelog topics." },
@@ -135,4 +135,4 @@
"about": "The partitions of the input topics processed by this member." }
]}
]
-}
\ No newline at end of file
+}
diff --git a/clients/src/main/resources/common/message/StreamsGroupHeartbeatResponse.json b/clients/src/main/resources/common/message/StreamsGroupHeartbeatResponse.json
index e7f7c0b768150..efeaf4525716f 100644
--- a/clients/src/main/resources/common/message/StreamsGroupHeartbeatResponse.json
+++ b/clients/src/main/resources/common/message/StreamsGroupHeartbeatResponse.json
@@ -100,7 +100,7 @@
"about": "A string representation of the status." }
]},
{ "name": "TopicPartition", "versions": "0+", "fields": [
- { "name": "Topic", "type": "string", "versions": "0+",
+ { "name": "Topic", "type": "string", "versions": "0+", "entityType": "topicName",
"about": "topic name" },
{ "name": "Partitions", "type": "[]int32", "versions": "0+",
"about": "partitions" }
@@ -118,4 +118,4 @@
"about": "port of the endpoint" }
]}
]
-}
\ No newline at end of file
+}
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMetadataTest.java
index 949bdc9aa727d..f57e93a2a15e8 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMetadataTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMetadataTest.java
@@ -17,6 +17,7 @@
package org.apache.kafka.clients.consumer.internals;
import org.apache.kafka.clients.Metadata;
+import org.apache.kafka.clients.consumer.SubscriptionPattern;
import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.ClusterResourceListener;
import org.apache.kafka.common.Node;
@@ -100,6 +101,78 @@ private void testPatternSubscription(boolean includeInternalTopics) {
assertEquals(Collections.singleton("__matching_topic"), metadata.fetch().topics());
}
+ @Test
+ public void testSubscriptionToBrokerRegexDoesNotRequestAllTopicsMetadata() {
+ // Subscribe to broker-side regex
+ subscription.subscribe(new SubscriptionPattern("__.*"), Optional.empty());
+
+ // Receive assignment from coordinator with topic IDs only
+ Uuid assignedTopicId = Uuid.randomUuid();
+ subscription.setAssignedTopicIds(Set.of(assignedTopicId));
+
+ // Metadata request should only include the assigned topic IDs
+ try (ConsumerMetadata metadata = newConsumerMetadata(false)) {
+ MetadataRequest.Builder builder = metadata.newMetadataRequestBuilder();
+ assertFalse(builder.isAllTopics(), "Should not request all topics when using broker-side regex");
+ assertEquals(List.of(assignedTopicId), builder.topicIds(), "Should only request assigned topic IDs when using broker-side regex");
+ }
+ }
+
+ @Test
+ public void testSubscriptionToBrokerRegexRetainsAssignedTopics() {
+ // Subscribe to broker-side regex
+ subscription.subscribe(new SubscriptionPattern("__.*"), Optional.empty());
+
+ // Receive assignment from coordinator with topic IDs only
+ Uuid assignedTopicId = Uuid.randomUuid();
+ subscription.setAssignedTopicIds(Set.of(assignedTopicId));
+
+ // Metadata request for assigned topic IDs
+ try (ConsumerMetadata metadata = newConsumerMetadata(false)) {
+ MetadataRequest.Builder builder = metadata.newMetadataRequestBuilder();
+ assertEquals(List.of(assignedTopicId), builder.topicIds());
+
+ // Metadata response with the assigned topic ID and name
+ Map topicIds = Map.of("__matching_topic", assignedTopicId);
+ MetadataResponse response = RequestTestUtils.metadataUpdateWithIds(1, singletonMap("__matching_topic", 1), topicIds);
+ metadata.updateWithCurrentRequestVersion(response, false, time.milliseconds());
+
+ assertEquals(Set.of("__matching_topic"), new HashSet<>(metadata.fetch().topics()));
+ assertEquals(Set.of("__matching_topic"), metadata.fetch().topics());
+ }
+ }
+
+ @Test
+ public void testSubscriptionToBrokerRegexAllowsTransientTopics() {
+ // Subscribe to broker-side regex
+ subscription.subscribe(new SubscriptionPattern("__.*"), Optional.empty());
+
+ // Receive assignment from coordinator with topic IDs only
+ Uuid assignedTopicId = Uuid.randomUuid();
+ subscription.setAssignedTopicIds(Set.of(assignedTopicId));
+
+ // Metadata request should only include the assigned topic IDs
+ try (ConsumerMetadata metadata = newConsumerMetadata(false)) {
+ MetadataRequest.Builder builder = metadata.newMetadataRequestBuilder();
+ assertFalse(builder.isAllTopics());
+ assertEquals(List.of(assignedTopicId), builder.topicIds());
+
+ // Call to offsets-related APIs starts. Metadata requests should move to requesting topic names temporarily.
+ String transientTopic = "__transient_topic";
+ metadata.addTransientTopics(Set.of(transientTopic));
+ builder = metadata.newMetadataRequestBuilder();
+ assertFalse(builder.isAllTopics());
+ // assertTrue(builder.topicIds().isEmpty());
+ assertEquals(List.of(transientTopic), builder.topics());
+
+ // Call to offsets-related APIs ends. Metadata requests should move back to requesting topic IDs for RE2J.
+ metadata.clearTransientTopics();
+ builder = metadata.newMetadataRequestBuilder();
+ assertFalse(builder.isAllTopics());
+ assertEquals(List.of(assignedTopicId), builder.topicIds());
+ }
+ }
+
@Test
public void testUserAssignment() {
subscription.assignFromUser(Set.of(
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java
index f697990b54425..eae791dc830fb 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java
@@ -26,6 +26,7 @@
import org.apache.kafka.common.IsolationLevel;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.utils.LogContext;
@@ -37,9 +38,12 @@
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
+import java.util.List;
import java.util.Optional;
import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.LongSupplier;
+import java.util.function.Predicate;
import java.util.regex.Pattern;
import static java.util.Collections.singleton;
@@ -114,6 +118,54 @@ public void partitionAssignmentChangeOnTopicSubscription() {
assertEquals(0, state.numAssignedPartitions());
}
+ @Test
+ public void testIsFetchableOnManualAssignment() {
+ state.assignFromUser(Set.of(tp0, tp1));
+ assertAssignedPartitionIsFetchable();
+ }
+
+ @Test
+ public void testIsFetchableOnAutoAssignment() {
+ state.subscribe(Set.of(topic), Optional.of(rebalanceListener));
+ state.assignFromSubscribed(Set.of(tp0, tp1));
+ assertAssignedPartitionIsFetchable();
+ }
+
+ private void assertAssignedPartitionIsFetchable() {
+ assertEquals(2, state.assignedPartitions().size());
+ assertTrue(state.assignedPartitions().contains(tp0));
+ assertTrue(state.assignedPartitions().contains(tp1));
+
+ assertFalse(state.isFetchable(tp0), "Should not be fetchable without a valid position");
+ assertFalse(state.isFetchable(tp1), "Should not be fetchable without a valid position");
+
+ state.seek(tp0, 1);
+ state.seek(tp1, 1);
+
+ assertTrue(state.isFetchable(tp0));
+ assertTrue(state.isFetchable(tp1));
+ }
+
+ @Test
+ public void testIsFetchableConsidersExplicitTopicSubscription() {
+ state.subscribe(Set.of(topic1), Optional.of(rebalanceListener));
+ state.assignFromSubscribed(Set.of(t1p0));
+ state.seek(t1p0, 1);
+
+ assertEquals(Set.of(t1p0), state.assignedPartitions());
+ assertTrue(state.isFetchable(t1p0));
+
+ // Change subscription. Assigned partitions should remain unchanged but not fetchable.
+ state.subscribe(Set.of(topic), Optional.of(rebalanceListener));
+ assertEquals(Set.of(t1p0), state.assignedPartitions());
+ assertFalse(state.isFetchable(t1p0), "Assigned partitions not in the subscription should not be fetchable");
+
+ // Unsubscribe. Assigned partitions should be cleared and not fetchable.
+ state.unsubscribe();
+ assertTrue(state.assignedPartitions().isEmpty());
+ assertFalse(state.isFetchable(t1p0));
+ }
+
@Test
public void testGroupSubscribe() {
state.subscribe(singleton(topic1), Optional.of(rebalanceListener));
@@ -273,6 +325,7 @@ public void testAssignedPartitionsAwaitingCallbackKeepPositionDefinedInCallback(
state.subscribe(singleton(topic), Optional.of(rebalanceListener));
state.assignFromSubscribedAwaitingCallback(singleton(tp0), singleton(tp0));
assertAssignmentAppliedAwaitingCallback(tp0);
+ assertEquals(singleton(tp0.topic()), state.subscription());
// Simulate callback setting position to start fetching from
state.seek(tp0, 100);
@@ -292,6 +345,7 @@ public void testAssignedPartitionsAwaitingCallbackInitializePositionsWhenCallbac
state.subscribe(singleton(topic), Optional.of(rebalanceListener));
state.assignFromSubscribedAwaitingCallback(singleton(tp0), singleton(tp0));
assertAssignmentAppliedAwaitingCallback(tp0);
+ assertEquals(singleton(tp0.topic()), state.subscription());
// Callback completed (without updating positions). Partition should require initializing
// positions, and start fetching once a valid position is set.
@@ -309,6 +363,7 @@ public void testAssignedPartitionsAwaitingCallbackDoesNotAffectPreviouslyOwnedPa
state.subscribe(singleton(topic), Optional.of(rebalanceListener));
state.assignFromSubscribedAwaitingCallback(singleton(tp0), singleton(tp0));
assertAssignmentAppliedAwaitingCallback(tp0);
+ assertEquals(singleton(tp0.topic()), state.subscription());
state.enablePartitionsAwaitingCallback(singleton(tp0));
state.seek(tp0, 100);
assertTrue(state.isFetchable(tp0));
@@ -331,7 +386,6 @@ public void testAssignedPartitionsAwaitingCallbackDoesNotAffectPreviouslyOwnedPa
private void assertAssignmentAppliedAwaitingCallback(TopicPartition topicPartition) {
assertEquals(singleton(topicPartition), state.assignedPartitions());
assertEquals(1, state.numAssignedPartitions());
- assertEquals(singleton(topicPartition.topic()), state.subscription());
assertFalse(state.isFetchable(topicPartition));
assertEquals(1, state.initializingPartitions().size());
@@ -404,6 +458,75 @@ public void testSubscribeToRe2JPattern() {
state.subscribe(new SubscriptionPattern(pattern), Optional.of(rebalanceListener));
assertTrue(state.toString().contains("type=AUTO_PATTERN_RE2J"));
assertTrue(state.toString().contains("subscribedPattern=" + pattern));
+ assertTrue(state.assignedTopicIds().isEmpty());
+ }
+
+ @Test
+ public void testIsAssignedFromRe2j() {
+ assertFalse(state.isAssignedFromRe2j(null));
+ Uuid assignedUuid = Uuid.randomUuid();
+ assertFalse(state.isAssignedFromRe2j(assignedUuid));
+
+ state.subscribe(new SubscriptionPattern("foo.*"), Optional.empty());
+ assertTrue(state.hasRe2JPatternSubscription());
+ assertFalse(state.isAssignedFromRe2j(assignedUuid));
+
+ state.setAssignedTopicIds(Set.of(assignedUuid));
+ assertTrue(state.isAssignedFromRe2j(assignedUuid));
+
+ state.unsubscribe();
+ assertFalse(state.isAssignedFromRe2j(assignedUuid));
+ assertFalse(state.hasRe2JPatternSubscription());
+
+ }
+
+ @Test
+ public void testAssignedPartitionsWithTopicIdsForRe2Pattern() {
+ state.subscribe(new SubscriptionPattern("t.*"), Optional.of(rebalanceListener));
+ assertTrue(state.assignedTopicIds().isEmpty());
+
+ TopicIdPartitionSet reconciledAssignmentFromRegex = new TopicIdPartitionSet();
+ reconciledAssignmentFromRegex.addAll(Uuid.randomUuid(), topic, Set.of(0));
+ state.assignFromSubscribedAwaitingCallback(singleton(tp0), singleton(tp0));
+ assertAssignmentAppliedAwaitingCallback(tp0);
+
+ // Simulate callback setting position to start fetching from
+ state.seek(tp0, 100);
+
+ // Callback completed. Partition should be fetchable, from the position previously defined
+ state.enablePartitionsAwaitingCallback(singleton(tp0));
+ assertEquals(0, state.initializingPartitions().size());
+ assertTrue(state.isFetchable(tp0));
+ assertTrue(state.hasAllFetchPositions());
+ assertEquals(100L, state.position(tp0).offset);
+ }
+
+ @Test
+ public void testAssignedTopicIdsPreservedWhenReconciliationCompletes() {
+ state.subscribe(new SubscriptionPattern("t.*"), Optional.of(rebalanceListener));
+ assertTrue(state.assignedTopicIds().isEmpty());
+
+ // First assignment received from coordinator
+ Uuid firstAssignedUuid = Uuid.randomUuid();
+ state.setAssignedTopicIds(Set.of(firstAssignedUuid));
+
+ // Second assignment received from coordinator (while the 1st still be reconciling)
+ Uuid secondAssignedUuid = Uuid.randomUuid();
+ state.setAssignedTopicIds(Set.of(firstAssignedUuid, secondAssignedUuid));
+
+ // First reconciliation completes and updates the subscription state
+ state.assignFromSubscribedAwaitingCallback(singleton(tp0), singleton(tp0));
+
+ // First assignment should have been applied
+ assertAssignmentAppliedAwaitingCallback(tp0);
+
+ // Assigned topic IDs should still have both topics (one reconciled, one not reconciled yet)
+ assertEquals(
+ Set.of(firstAssignedUuid, secondAssignedUuid),
+ state.assignedTopicIds(),
+ "Updating the subscription state when a reconciliation completes " +
+ "should not overwrite assigned topics that have not been reconciled yet"
+ );
}
@Test
@@ -1001,4 +1124,33 @@ public void testRequestOffsetResetIfPartitionAssigned() {
assertThrows(IllegalStateException.class, () -> state.isOffsetResetNeeded(unassignedPartition));
}
+
+ // This test ensures the "fetchablePartitions" does not run the custom predicate if the partition is not fetchable
+ // This func is used in the hot path for fetching, to find fetchable partitions that are not in the buffer,
+ // so it should avoid evaluating the predicate if not needed.
+ @Test
+ public void testFetchablePartitionsPerformsCheapChecksFirst() {
+ // Setup fetchable partition and pause it
+ state.assignFromUser(Set.of(tp0));
+ state.seek(tp0, 100);
+ assertTrue(state.isFetchable(tp0));
+ state.pause(tp0);
+
+ // Retrieve fetchable partitions with custom predicate.
+ AtomicBoolean predicateEvaluated = new AtomicBoolean(false);
+ Predicate isBuffered = tp -> {
+ predicateEvaluated.set(true);
+ return true;
+ };
+ List fetchablePartitions = state.fetchablePartitions(isBuffered);
+ assertTrue(fetchablePartitions.isEmpty());
+ assertFalse(predicateEvaluated.get(), "Custom predicate should not be evaluated when partitions are not fetchable");
+
+ // Resume partition and retrieve fetchable again
+ state.resume(tp0);
+ predicateEvaluated.set(false);
+ fetchablePartitions = state.fetchablePartitions(isBuffered);
+ assertTrue(predicateEvaluated.get());
+ assertEquals(tp0, fetchablePartitions.get(0));
+ }
}
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java
index 8460d0f4c5f37..ac63fed3c041e 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java
@@ -32,7 +32,6 @@
import org.apache.kafka.clients.producer.internals.RecordAccumulator;
import org.apache.kafka.clients.producer.internals.Sender;
import org.apache.kafka.clients.producer.internals.TransactionManager;
-import org.apache.kafka.clients.producer.internals.TransactionalRequestResult;
import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.Metric;
@@ -47,7 +46,6 @@
import org.apache.kafka.common.errors.ClusterAuthorizationException;
import org.apache.kafka.common.errors.InterruptException;
import org.apache.kafka.common.errors.InvalidTopicException;
-import org.apache.kafka.common.errors.InvalidTxnStateException;
import org.apache.kafka.common.errors.RecordTooLargeException;
import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
@@ -78,7 +76,6 @@
import org.apache.kafka.common.requests.EndTxnResponse;
import org.apache.kafka.common.requests.FindCoordinatorRequest;
import org.apache.kafka.common.requests.FindCoordinatorResponse;
-import org.apache.kafka.common.requests.InitProducerIdRequest;
import org.apache.kafka.common.requests.InitProducerIdResponse;
import org.apache.kafka.common.requests.JoinGroupRequest;
import org.apache.kafka.common.requests.MetadataResponse;
@@ -107,7 +104,6 @@
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInfo;
import org.junit.jupiter.params.ParameterizedTest;
-import org.junit.jupiter.params.provider.CsvSource;
import org.junit.jupiter.params.provider.ValueSource;
import org.mockito.MockedStatic;
import org.mockito.Mockito;
@@ -154,7 +150,6 @@
import static org.junit.jupiter.api.Assertions.assertNotEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
-import static org.junit.jupiter.api.Assertions.assertSame;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
@@ -165,7 +160,6 @@
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.ArgumentMatchers.notNull;
import static org.mockito.Mockito.atMostOnce;
-import static org.mockito.Mockito.doNothing;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.mockStatic;
import static org.mockito.Mockito.never;
@@ -1389,294 +1383,6 @@ public void testInitTransactionWhileThrottled() {
}
}
- @ParameterizedTest
- @CsvSource({
- "true, false",
- "true, true",
- "false, true"
- })
- public void testInitTransactionsWithKeepPreparedTxnAndTwoPhaseCommit(boolean keepPreparedTxn, boolean enable2PC) {
- Map configs = new HashMap<>();
- configs.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "test-txn-id");
- configs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, 10000);
- configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9000");
- if (enable2PC) {
- configs.put(ProducerConfig.TRANSACTION_TWO_PHASE_COMMIT_ENABLE_CONFIG, true);
- }
-
- Time time = new MockTime(1);
- MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap("topic", 1));
- ProducerMetadata metadata = newMetadata(0, 0, Long.MAX_VALUE);
- MockClient client = new MockClient(time, metadata);
- client.updateMetadata(initialUpdateResponse);
-
- // Capture flags from the InitProducerIdRequest
- boolean[] requestFlags = new boolean[2]; // [keepPreparedTxn, enable2Pc]
-
- client.prepareResponse(
- request -> request instanceof FindCoordinatorRequest &&
- ((FindCoordinatorRequest) request).data().keyType() == FindCoordinatorRequest.CoordinatorType.TRANSACTION.id(),
- FindCoordinatorResponse.prepareResponse(Errors.NONE, "test-txn-id", NODE));
-
- client.prepareResponse(
- request -> {
- if (request instanceof InitProducerIdRequest) {
- InitProducerIdRequest initRequest = (InitProducerIdRequest) request;
- requestFlags[0] = initRequest.data().keepPreparedTxn();
- requestFlags[1] = initRequest.data().enable2Pc();
- return true;
- }
- return false;
- },
- initProducerIdResponse(1L, (short) 5, Errors.NONE));
-
- try (Producer producer = kafkaProducer(configs, new StringSerializer(),
- new StringSerializer(), metadata, client, null, time)) {
- producer.initTransactions(keepPreparedTxn);
-
- // Verify request flags match expected values
- assertEquals(keepPreparedTxn, requestFlags[0],
- "keepPreparedTxn flag should match input parameter");
- assertEquals(enable2PC, requestFlags[1],
- "enable2Pc flag should match producer configuration");
- }
- }
-
- @Test
- public void testPrepareTransactionSuccess() throws Exception {
- StringSerializer serializer = new StringSerializer();
- KafkaProducerTestContext ctx = new KafkaProducerTestContext<>(testInfo, serializer);
-
- when(ctx.transactionManager.isTransactionV2Enabled()).thenReturn(true);
- when(ctx.transactionManager.is2PCEnabled()).thenReturn(true);
- when(ctx.sender.isRunning()).thenReturn(true);
-
- doNothing().when(ctx.transactionManager).prepareTransaction();
-
- PreparedTxnState expectedState = mock(PreparedTxnState.class);
- when(ctx.transactionManager.preparedTransactionState()).thenReturn(expectedState);
-
- try (KafkaProducer producer = ctx.newKafkaProducer()) {
- PreparedTxnState returned = producer.prepareTransaction();
- assertSame(expectedState, returned);
-
- verify(ctx.transactionManager).prepareTransaction();
- verify(ctx.accumulator).beginFlush();
- verify(ctx.accumulator).awaitFlushCompletion();
- }
- }
-
- @Test
- public void testSendNotAllowedInPreparedTransactionState() throws Exception {
- StringSerializer serializer = new StringSerializer();
- KafkaProducerTestContext ctx = new KafkaProducerTestContext<>(testInfo, serializer);
-
- String topic = "foo";
- Cluster cluster = TestUtils.singletonCluster(topic, 1);
-
- when(ctx.sender.isRunning()).thenReturn(true);
- when(ctx.metadata.fetch()).thenReturn(cluster);
-
- // Mock transaction manager to simulate being in a prepared state
- when(ctx.transactionManager.isTransactional()).thenReturn(true);
- when(ctx.transactionManager.isPrepared()).thenReturn(true);
-
- // Create record to send
- long timestamp = ctx.time.milliseconds();
- ProducerRecord record = new ProducerRecord<>(topic, 0, timestamp, "key", "value");
-
- try (KafkaProducer producer = ctx.newKafkaProducer()) {
- // Verify that sending a record throws IllegalStateException with the correct message
- IllegalStateException exception = assertThrows(
- IllegalStateException.class,
- () -> producer.send(record)
- );
-
- assertTrue(exception.getMessage().contains("Cannot perform operation while the transaction is in a prepared state"));
-
- // Verify transactionManager methods were called
- verify(ctx.transactionManager).isTransactional();
- verify(ctx.transactionManager).isPrepared();
-
- // Verify that no message was actually sent (accumulator was not called)
- verify(ctx.accumulator, never()).append(
- eq(topic),
- anyInt(),
- anyLong(),
- any(),
- any(),
- any(),
- any(),
- anyLong(),
- anyLong(),
- any()
- );
- }
- }
-
- @Test
- public void testSendOffsetsNotAllowedInPreparedTransactionState() throws Exception {
- StringSerializer serializer = new StringSerializer();
- KafkaProducerTestContext ctx = new KafkaProducerTestContext<>(testInfo, serializer);
-
- String topic = "foo";
- Cluster cluster = TestUtils.singletonCluster(topic, 1);
-
- when(ctx.sender.isRunning()).thenReturn(true);
- when(ctx.metadata.fetch()).thenReturn(cluster);
-
- // Mock transaction manager to simulate being in a prepared state
- when(ctx.transactionManager.isTransactional()).thenReturn(true);
- when(ctx.transactionManager.isPrepared()).thenReturn(true);
-
- // Create consumer group metadata
- String groupId = "test-group";
- Map offsets = new HashMap<>();
- offsets.put(new TopicPartition(topic, 0), new OffsetAndMetadata(100L));
- ConsumerGroupMetadata groupMetadata = new ConsumerGroupMetadata(groupId);
-
- try (KafkaProducer producer = ctx.newKafkaProducer()) {
- // Verify that sending offsets throws IllegalStateException with the correct message
- IllegalStateException exception = assertThrows(
- IllegalStateException.class,
- () -> producer.sendOffsetsToTransaction(offsets, groupMetadata)
- );
-
- assertTrue(exception.getMessage().contains("Cannot perform operation while the transaction is in a prepared state"));
-
- // Verify transactionManager methods were called
- verify(ctx.transactionManager).isTransactional();
- verify(ctx.transactionManager).isPrepared();
-
- // Verify that no offsets were actually sent
- verify(ctx.transactionManager, never()).sendOffsetsToTransaction(
- eq(offsets),
- eq(groupMetadata)
- );
- }
- }
-
- @Test
- public void testBeginTransactionNotAllowedInPreparedTransactionState() throws Exception {
- StringSerializer serializer = new StringSerializer();
- KafkaProducerTestContext ctx = new KafkaProducerTestContext<>(testInfo, serializer);
-
- when(ctx.sender.isRunning()).thenReturn(true);
-
- // Mock transaction manager to simulate being in a prepared state
- when(ctx.transactionManager.isTransactional()).thenReturn(true);
- when(ctx.transactionManager.isPrepared()).thenReturn(true);
-
- try (KafkaProducer producer = ctx.newKafkaProducer()) {
- // Verify that calling beginTransaction throws IllegalStateException with the correct message
- IllegalStateException exception = assertThrows(
- IllegalStateException.class,
- producer::beginTransaction
- );
-
- assertTrue(exception.getMessage().contains("Cannot perform operation while the transaction is in a prepared state"));
-
- // Verify transactionManager methods were called
- verify(ctx.transactionManager).isTransactional();
- verify(ctx.transactionManager).isPrepared();
- }
- }
-
- @Test
- public void testPrepareTransactionFailsWhen2PCDisabled() {
- StringSerializer serializer = new StringSerializer();
- KafkaProducerTestContext ctx = new KafkaProducerTestContext<>(testInfo, serializer);
-
- // Disable 2PC
- when(ctx.transactionManager.isTransactionV2Enabled()).thenReturn(true);
- when(ctx.transactionManager.is2PCEnabled()).thenReturn(false);
- when(ctx.sender.isRunning()).thenReturn(true);
-
- try (KafkaProducer producer = ctx.newKafkaProducer()) {
- assertThrows(
- InvalidTxnStateException.class,
- producer::prepareTransaction,
- "prepareTransaction() should fail if 2PC is disabled"
- );
- }
- }
-
- @Test
- public void testCompleteTransactionWithMatchingState() throws Exception {
- StringSerializer serializer = new StringSerializer();
- KafkaProducerTestContext ctx = new KafkaProducerTestContext<>(testInfo, serializer);
-
- when(ctx.transactionManager.isPrepared()).thenReturn(true);
- when(ctx.sender.isRunning()).thenReturn(true);
-
- // Create prepared states with matching values
- long producerId = 12345L;
- short epoch = 5;
- PreparedTxnState currentState = new PreparedTxnState(producerId, epoch);
- PreparedTxnState inputState = new PreparedTxnState(producerId, epoch);
-
- // Set up the transaction manager to return the prepared state
- when(ctx.transactionManager.preparedTransactionState()).thenReturn(currentState);
-
- // Should trigger commit when states match
- TransactionalRequestResult commitResult = mock(TransactionalRequestResult.class);
- when(ctx.transactionManager.beginCommit()).thenReturn(commitResult);
-
- try (KafkaProducer producer = ctx.newKafkaProducer()) {
- // Call completeTransaction with the matching state
- producer.completeTransaction(inputState);
-
- // Verify methods called in order
- verify(ctx.transactionManager).isPrepared();
- verify(ctx.transactionManager).preparedTransactionState();
- verify(ctx.transactionManager).beginCommit();
-
- // Verify abort was never called
- verify(ctx.transactionManager, never()).beginAbort();
-
- // Verify sender was woken up
- verify(ctx.sender).wakeup();
- }
- }
-
- @Test
- public void testCompleteTransactionWithNonMatchingState() throws Exception {
- StringSerializer serializer = new StringSerializer();
- KafkaProducerTestContext ctx = new KafkaProducerTestContext<>(testInfo, serializer);
-
- when(ctx.transactionManager.isPrepared()).thenReturn(true);
- when(ctx.sender.isRunning()).thenReturn(true);
-
- // Create txn prepared states with different values
- long producerId = 12345L;
- short epoch = 5;
- PreparedTxnState currentState = new PreparedTxnState(producerId, epoch);
- PreparedTxnState inputState = new PreparedTxnState(producerId + 1, epoch);
-
- // Set up the transaction manager to return the prepared state
- when(ctx.transactionManager.preparedTransactionState()).thenReturn(currentState);
-
- // Should trigger abort when states don't match
- TransactionalRequestResult abortResult = mock(TransactionalRequestResult.class);
- when(ctx.transactionManager.beginAbort()).thenReturn(abortResult);
-
- try (KafkaProducer producer = ctx.newKafkaProducer()) {
- // Call completeTransaction with the non-matching state
- producer.completeTransaction(inputState);
-
- // Verify methods called in order
- verify(ctx.transactionManager).isPrepared();
- verify(ctx.transactionManager).preparedTransactionState();
- verify(ctx.transactionManager).beginAbort();
-
- // Verify commit was never called
- verify(ctx.transactionManager, never()).beginCommit();
-
- // Verify sender was woken up
- verify(ctx.sender).wakeup();
- }
- }
-
@Test
public void testClusterAuthorizationFailure() throws Exception {
int maxBlockMs = 500;
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java
index 4668a91ed04dd..278e6c3e381d9 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java
@@ -23,7 +23,6 @@
import org.apache.kafka.clients.consumer.CommitFailedException;
import org.apache.kafka.clients.consumer.ConsumerGroupMetadata;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
-import org.apache.kafka.clients.producer.PreparedTxnState;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.Node;
@@ -4025,56 +4024,6 @@ private void verifyCommitOrAbortTransactionRetriable(TransactionResult firstTran
assertFalse(transactionManager.hasOngoingTransaction());
}
- @Test
- public void testInitializeTransactionsWithKeepPreparedTxn() {
- doInitTransactionsWith2PCEnabled(true);
- runUntil(transactionManager::hasProducerId);
-
- // Expect a bumped epoch in the response.
- assertTrue(transactionManager.hasProducerId());
- assertFalse(transactionManager.hasOngoingTransaction());
- assertEquals(ongoingProducerId, transactionManager.producerIdAndEpoch().producerId);
- assertEquals(bumpedOngoingEpoch, transactionManager.producerIdAndEpoch().epoch);
- }
-
- @Test
- public void testPrepareTransaction() {
- doInitTransactionsWith2PCEnabled(false);
- runUntil(transactionManager::hasProducerId);
-
- // Begin a transaction
- transactionManager.beginTransaction();
- assertTrue(transactionManager.hasOngoingTransaction());
-
- // Add a partition to the transaction
- transactionManager.maybeAddPartition(tp0);
-
- // Capture the current producer ID and epoch before preparing the response
- long producerId = transactionManager.producerIdAndEpoch().producerId;
- short epoch = transactionManager.producerIdAndEpoch().epoch;
-
- // Simulate a produce request
- try {
- // Prepare the response before sending to ensure it's ready
- prepareProduceResponse(Errors.NONE, producerId, epoch);
-
- appendToAccumulator(tp0);
- // Wait until the request is processed
- runUntil(() -> !client.hasPendingResponses());
- } catch (InterruptedException e) {
- fail("Unexpected interruption: " + e);
- }
-
- transactionManager.prepareTransaction();
- assertTrue(transactionManager.isPrepared());
-
- PreparedTxnState preparedState = transactionManager.preparedTransactionState();
- // Validate the state contains the correct serialized producer ID and epoch
- assertEquals(producerId + ":" + epoch, preparedState.toString());
- assertEquals(producerId, preparedState.producerId());
- assertEquals(epoch, preparedState.epoch());
- }
-
private void prepareAddPartitionsToTxn(final Map errors) {
AddPartitionsToTxnResult result = AddPartitionsToTxnResponse.resultForTransaction(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID, errors);
AddPartitionsToTxnResponseData data = new AddPartitionsToTxnResponseData().setResultsByTopicV3AndBelow(result.topicResults()).setThrottleTimeMs(0);
diff --git a/clients/src/test/java/org/apache/kafka/common/requests/JoinGroupRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/JoinGroupRequestTest.java
index 60d10a689394c..9c4110055831e 100644
--- a/clients/src/test/java/org/apache/kafka/common/requests/JoinGroupRequestTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/requests/JoinGroupRequestTest.java
@@ -19,12 +19,14 @@
import org.apache.kafka.common.errors.InvalidConfigurationException;
import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.message.JoinGroupRequestData;
+import org.apache.kafka.common.protocol.MessageUtil;
import org.apache.kafka.test.TestUtils;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
+import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.fail;
@@ -65,4 +67,20 @@ public void testRequestVersionCompatibilityFailBuild() {
.setProtocolType("consumer")
).build((short) 4));
}
+
+ @Test
+ public void testRebalanceTimeoutDefaultsToSessionTimeoutV0() {
+ int sessionTimeoutMs = 30000;
+ short version = 0;
+
+ var buffer = MessageUtil.toByteBufferAccessor(new JoinGroupRequestData()
+ .setGroupId("groupId")
+ .setMemberId("consumerId")
+ .setProtocolType("consumer")
+ .setSessionTimeoutMs(sessionTimeoutMs), version);
+
+ JoinGroupRequest request = JoinGroupRequest.parse(buffer, version);
+ assertEquals(sessionTimeoutMs, request.data().sessionTimeoutMs());
+ assertEquals(sessionTimeoutMs, request.data().rebalanceTimeoutMs());
+ }
}
diff --git a/clients/src/test/java/org/apache/kafka/common/requests/OffsetFetchResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/OffsetFetchResponseTest.java
index 23b5258a2352e..302e630950166 100644
--- a/clients/src/test/java/org/apache/kafka/common/requests/OffsetFetchResponseTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/requests/OffsetFetchResponseTest.java
@@ -17,6 +17,7 @@
package org.apache.kafka.common.requests;
import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.message.OffsetFetchRequestData;
import org.apache.kafka.common.message.OffsetFetchResponseData;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
@@ -26,6 +27,9 @@
import java.util.List;
+import static org.apache.kafka.common.record.RecordBatch.NO_PARTITION_LEADER_EPOCH;
+import static org.apache.kafka.common.requests.OffsetFetchResponse.INVALID_OFFSET;
+import static org.apache.kafka.common.requests.OffsetFetchResponse.NO_METADATA;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
@@ -243,4 +247,43 @@ public void testGroupWithSingleGroupWithTopLevelError(short version) {
new OffsetFetchResponse(data, version).group("foo")
);
}
+
+ @ParameterizedTest
+ @ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_FETCH)
+ public void testSingleGroupWithError(short version) {
+ var group = new OffsetFetchRequestData.OffsetFetchRequestGroup()
+ .setGroupId("group1")
+ .setTopics(List.of(
+ new OffsetFetchRequestData.OffsetFetchRequestTopics()
+ .setName("foo")
+ .setPartitionIndexes(List.of(0))
+ ));
+
+ if (version < 2) {
+ assertEquals(
+ new OffsetFetchResponseData.OffsetFetchResponseGroup()
+ .setGroupId("group1")
+ .setTopics(List.of(
+ new OffsetFetchResponseData.OffsetFetchResponseTopics()
+ .setName("foo")
+ .setPartitions(List.of(
+ new OffsetFetchResponseData.OffsetFetchResponsePartitions()
+ .setPartitionIndex(0)
+ .setErrorCode(Errors.INVALID_GROUP_ID.code())
+ .setCommittedOffset(INVALID_OFFSET)
+ .setMetadata(NO_METADATA)
+ .setCommittedLeaderEpoch(NO_PARTITION_LEADER_EPOCH)
+ ))
+ )),
+ OffsetFetchResponse.groupError(group, Errors.INVALID_GROUP_ID, version)
+ );
+ } else {
+ assertEquals(
+ new OffsetFetchResponseData.OffsetFetchResponseGroup()
+ .setGroupId("group1")
+ .setErrorCode(Errors.INVALID_GROUP_ID.code()),
+ OffsetFetchResponse.groupError(group, Errors.INVALID_GROUP_ID, version)
+ );
+ }
+ }
}
diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
index 6be4c35bd254d..957544e2f7305 100644
--- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
@@ -732,6 +732,14 @@ public void testFetchRequestWithMetadata() {
assertEquals(request.isolationLevel(), deserialized.isolationLevel());
}
+ @Test
+ public void testJoinGroupRequestV0RebalanceTimeout() {
+ final short version = 0;
+ JoinGroupRequest jgr = createJoinGroupRequest(version);
+ JoinGroupRequest jgr2 = JoinGroupRequest.parse(jgr.serialize(), version);
+ assertEquals(jgr2.data().rebalanceTimeoutMs(), jgr.data().rebalanceTimeoutMs());
+ }
+
@Test
public void testSerializeWithHeader() {
CreatableTopicCollection topicsToCreate = new CreatableTopicCollection(1);
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/JwtBearerJwtRetrieverTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/JwtBearerJwtRetrieverTest.java
index c466ac8368907..4a4e567dedfdf 100644
--- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/JwtBearerJwtRetrieverTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/JwtBearerJwtRetrieverTest.java
@@ -17,7 +17,6 @@
package org.apache.kafka.common.security.oauthbearer;
-import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.config.SaslConfigs;
import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerTest;
@@ -95,7 +94,7 @@ public void testConfigureWithMalformedPrivateKey() throws Exception {
List jaasConfigEntries = getJaasConfigEntries();
try (JwtBearerJwtRetriever jwtRetriever = new JwtBearerJwtRetriever()) {
- KafkaException e = assertThrows(KafkaException.class, () -> jwtRetriever.configure(configs, OAUTHBEARER_MECHANISM, jaasConfigEntries));
+ JwtRetrieverException e = assertThrows(JwtRetrieverException.class, () -> jwtRetriever.configure(configs, OAUTHBEARER_MECHANISM, jaasConfigEntries));
assertNotNull(e.getCause());
assertInstanceOf(GeneralSecurityException.class, e.getCause());
}
@@ -144,7 +143,7 @@ public void testConfigureWithInvalidPassphrase() throws Exception {
List jaasConfigEntries = getJaasConfigEntries();
try (JwtBearerJwtRetriever jwtRetriever = new JwtBearerJwtRetriever()) {
- KafkaException e = assertThrows(KafkaException.class, () -> jwtRetriever.configure(configs, OAUTHBEARER_MECHANISM, jaasConfigEntries));
+ JwtRetrieverException e = assertThrows(JwtRetrieverException.class, () -> jwtRetriever.configure(configs, OAUTHBEARER_MECHANISM, jaasConfigEntries));
assertNotNull(e.getCause());
assertInstanceOf(IOException.class, e.getCause());
}
diff --git a/committer-tools/kafka-merge-pr.py b/committer-tools/kafka-merge-pr.py
index d649181d74600..1b181ed1d68b3 100755
--- a/committer-tools/kafka-merge-pr.py
+++ b/committer-tools/kafka-merge-pr.py
@@ -70,7 +70,7 @@
DEV_BRANCH_NAME = "trunk"
-DEFAULT_FIX_VERSION = os.environ.get("DEFAULT_FIX_VERSION", "4.1.0")
+DEFAULT_FIX_VERSION = os.environ.get("DEFAULT_FIX_VERSION", "4.1.1")
ORIGINAL_HEAD = ""
diff --git a/config/log4j2.yaml b/config/log4j2.yaml
index 49bcf78d136c6..de263c57c928e 100644
--- a/config/log4j2.yaml
+++ b/config/log4j2.yaml
@@ -44,7 +44,7 @@ Configuration:
# State Change appender
- name: StateChangeAppender
fileName: "${sys:kafka.logs.dir}/state-change.log"
- filePattern: "${sys:kafka.logs.dir}/stage-change.log.%d{yyyy-MM-dd-HH}"
+ filePattern: "${sys:kafka.logs.dir}/state-change.log.%d{yyyy-MM-dd-HH}"
PatternLayout:
pattern: "${logPattern}"
TimeBasedTriggeringPolicy:
diff --git a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java
index 19567344f2d36..8f72b0cac50ba 100644
--- a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java
+++ b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java
@@ -69,6 +69,7 @@
import java.util.function.Consumer;
import java.util.stream.Collectors;
+import static java.lang.Math.min;
import static org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime.CoordinatorWriteEvent.NOT_QUEUED;
/**
@@ -758,8 +759,14 @@ private void freeCurrentBatch() {
// Cancel the linger timeout.
currentBatch.lingerTimeoutTask.ifPresent(TimerTask::cancel);
- // Release the buffer.
- bufferSupplier.release(currentBatch.buffer);
+ // Release the buffer only if it is not larger than the maxBatchSize.
+ int maxBatchSize = partitionWriter.config(tp).maxMessageSize();
+
+ if (currentBatch.builder.buffer().capacity() <= maxBatchSize) {
+ bufferSupplier.release(currentBatch.builder.buffer());
+ } else if (currentBatch.buffer.capacity() <= maxBatchSize) {
+ bufferSupplier.release(currentBatch.buffer);
+ }
currentBatch = null;
}
@@ -859,7 +866,7 @@ private void maybeAllocateNewBatch(
LogConfig logConfig = partitionWriter.config(tp);
int maxBatchSize = logConfig.maxMessageSize();
long prevLastWrittenOffset = coordinator.lastWrittenOffset();
- ByteBuffer buffer = bufferSupplier.get(maxBatchSize);
+ ByteBuffer buffer = bufferSupplier.get(min(INITIAL_BUFFER_SIZE, maxBatchSize));
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(
buffer,
@@ -1909,9 +1916,9 @@ public void onHighWatermarkUpdated(
}
/**
- * 16KB. Used for initial buffer size for write operations.
+ * 512KB. Used for initial buffer size for write operations.
*/
- static final int MIN_BUFFER_SIZE = 16384;
+ static final int INITIAL_BUFFER_SIZE = 512 * 1024;
/**
* The log prefix.
diff --git a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeMetricsImpl.java b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeMetricsImpl.java
index a95f590c5b26b..af775c7c45118 100644
--- a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeMetricsImpl.java
+++ b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeMetricsImpl.java
@@ -149,7 +149,7 @@ public CoordinatorRuntimeMetricsImpl(Metrics metrics, String metricsGroup) {
metrics.addMetric(numPartitionsActive, (Gauge) (config, now) -> numPartitionsActiveCounter.get());
metrics.addMetric(numPartitionsFailed, (Gauge) (config, now) -> numPartitionsFailedCounter.get());
- this.partitionLoadSensor = metrics.sensor("GroupPartitionLoadTime");
+ this.partitionLoadSensor = metrics.sensor(this.metricsGroup + "-PartitionLoadTime");
this.partitionLoadSensor.add(
metrics.metricName(
"partition-load-time-max",
@@ -163,7 +163,7 @@ public CoordinatorRuntimeMetricsImpl(Metrics metrics, String metricsGroup) {
"The average time it took to load the partitions in the last 30 sec."
), new Avg());
- this.threadIdleSensor = metrics.sensor("ThreadIdleRatio");
+ this.threadIdleSensor = metrics.sensor(this.metricsGroup + "-ThreadIdleRatio");
this.threadIdleSensor.add(
metrics.metricName(
"thread-idle-ratio-avg",
@@ -178,7 +178,7 @@ public CoordinatorRuntimeMetricsImpl(Metrics metrics, String metricsGroup) {
"The " + suffix + " event queue time in milliseconds"
)
);
- this.eventQueueTimeSensor = metrics.sensor("EventQueueTime");
+ this.eventQueueTimeSensor = metrics.sensor(this.metricsGroup + "-EventQueueTime");
this.eventQueueTimeSensor.add(eventQueueTimeHistogram);
KafkaMetricHistogram eventProcessingTimeHistogram = KafkaMetricHistogram.newLatencyHistogram(
@@ -187,7 +187,7 @@ public CoordinatorRuntimeMetricsImpl(Metrics metrics, String metricsGroup) {
"The " + suffix + " event processing time in milliseconds"
)
);
- this.eventProcessingTimeSensor = metrics.sensor("EventProcessingTime");
+ this.eventProcessingTimeSensor = metrics.sensor(this.metricsGroup + "-EventProcessingTime");
this.eventProcessingTimeSensor.add(eventProcessingTimeHistogram);
KafkaMetricHistogram eventPurgatoryTimeHistogram = KafkaMetricHistogram.newLatencyHistogram(
@@ -196,7 +196,7 @@ public CoordinatorRuntimeMetricsImpl(Metrics metrics, String metricsGroup) {
"The " + suffix + " event purgatory time in milliseconds"
)
);
- this.eventPurgatoryTimeSensor = metrics.sensor("EventPurgatoryTime");
+ this.eventPurgatoryTimeSensor = metrics.sensor(this.metricsGroup + "-EventPurgatoryTime");
this.eventPurgatoryTimeSensor.add(eventPurgatoryTimeHistogram);
KafkaMetricHistogram flushTimeHistogram = KafkaMetricHistogram.newLatencyHistogram(
@@ -205,7 +205,7 @@ public CoordinatorRuntimeMetricsImpl(Metrics metrics, String metricsGroup) {
"The " + suffix + " flush time in milliseconds"
)
);
- this.flushTimeSensor = metrics.sensor("FlushTime");
+ this.flushTimeSensor = metrics.sensor(this.metricsGroup + "-FlushTime");
this.flushTimeSensor.add(flushTimeHistogram);
}
diff --git a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeMetricsImplTest.java b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeMetricsImplTest.java
index ed6d269763450..68f152f2bea08 100644
--- a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeMetricsImplTest.java
+++ b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeMetricsImplTest.java
@@ -27,8 +27,8 @@
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;
-import java.util.Arrays;
-import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
import java.util.stream.IntStream;
import static org.apache.kafka.coordinator.common.runtime.CoordinatorRuntimeMetricsImpl.BATCH_FLUSH_TIME_METRIC_NAME;
@@ -39,17 +39,19 @@
import static org.apache.kafka.coordinator.common.runtime.KafkaMetricHistogram.MAX_LATENCY_MS;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
public class CoordinatorRuntimeMetricsImplTest {
private static final String METRICS_GROUP = "test-runtime-metrics";
-
+ private static final String OTHER_METRICS_GROUP = "test-runtime-metrics-2";
+
@Test
public void testMetricNames() {
Metrics metrics = new Metrics();
- HashSet expectedMetrics = new HashSet<>(Arrays.asList(
+ Set expectedMetrics = Set.of(
kafkaMetricName(metrics, NUM_PARTITIONS_METRIC_NAME, "state", "loading"),
kafkaMetricName(metrics, NUM_PARTITIONS_METRIC_NAME, "state", "active"),
kafkaMetricName(metrics, NUM_PARTITIONS_METRIC_NAME, "state", "failed"),
@@ -77,7 +79,7 @@ public void testMetricNames() {
kafkaMetricName(metrics, "batch-flush-time-ms-p95"),
kafkaMetricName(metrics, "batch-flush-time-ms-p99"),
kafkaMetricName(metrics, "batch-flush-time-ms-p999")
- ));
+ );
try (CoordinatorRuntimeMetricsImpl runtimeMetrics = new CoordinatorRuntimeMetricsImpl(metrics, METRICS_GROUP)) {
runtimeMetrics.registerEventQueueSizeGauge(() -> 0);
@@ -110,6 +112,26 @@ public void testUpdateNumPartitionsMetrics() {
}
}
+ @Test
+ public void testNumPartitionsMetricsGroupIsolation() {
+ Metrics metrics = new Metrics();
+
+ try (CoordinatorRuntimeMetricsImpl runtimeMetrics = new CoordinatorRuntimeMetricsImpl(metrics, METRICS_GROUP);
+ CoordinatorRuntimeMetricsImpl runtimeMetrics2 = new CoordinatorRuntimeMetricsImpl(metrics, OTHER_METRICS_GROUP)) {
+ IntStream.range(0, 3)
+ .forEach(__ -> runtimeMetrics.recordPartitionStateChange(CoordinatorState.INITIAL, CoordinatorState.LOADING));
+ IntStream.range(0, 2)
+ .forEach(__ -> runtimeMetrics.recordPartitionStateChange(CoordinatorState.LOADING, CoordinatorState.ACTIVE));
+ IntStream.range(0, 1)
+ .forEach(__ -> runtimeMetrics.recordPartitionStateChange(CoordinatorState.ACTIVE, CoordinatorState.FAILED));
+
+ for (String state : List.of("loading", "active", "failed")) {
+ assertMetricGauge(metrics, kafkaMetricName(metrics, NUM_PARTITIONS_METRIC_NAME, "state", state), 1);
+ assertMetricGauge(metrics, otherGroupKafkaMetricName(metrics, NUM_PARTITIONS_METRIC_NAME, "state", state), 0);
+ }
+ }
+ }
+
@Test
public void testPartitionLoadSensorMetrics() {
Time time = new MockTime();
@@ -131,6 +153,29 @@ public void testPartitionLoadSensorMetrics() {
}
}
+ @ParameterizedTest
+ @ValueSource(strings = {
+ "partition-load-time-avg",
+ "partition-load-time-max"
+ })
+ public void testPartitionLoadSensorMetricsGroupIsolation(String name) {
+ Time time = new MockTime();
+ Metrics metrics = new Metrics(time);
+
+ try (CoordinatorRuntimeMetricsImpl runtimeMetrics = new CoordinatorRuntimeMetricsImpl(metrics, METRICS_GROUP);
+ CoordinatorRuntimeMetricsImpl runtimeMetrics2 = new CoordinatorRuntimeMetricsImpl(metrics, OTHER_METRICS_GROUP)) {
+ long startTimeMs = time.milliseconds();
+ runtimeMetrics.recordPartitionLoadSensor(startTimeMs, startTimeMs + 1000);
+
+ org.apache.kafka.common.MetricName metricName = kafkaMetricName(metrics, name);
+ org.apache.kafka.common.MetricName otherGroupMetricName = otherGroupKafkaMetricName(metrics, name);
+ KafkaMetric metric = metrics.metrics().get(metricName);
+ KafkaMetric otherMetric = metrics.metrics().get(otherGroupMetricName);
+ assertNotEquals(Double.NaN, metric.metricValue());
+ assertEquals(Double.NaN, otherMetric.metricValue());
+ }
+ }
+
@Test
public void testThreadIdleSensor() {
Time time = new MockTime();
@@ -144,6 +189,22 @@ public void testThreadIdleSensor() {
assertEquals(6 / 30.0, metric.metricValue()); // 'total_ms / window_ms'
}
+ @Test
+ public void testThreadIdleSensorMetricsGroupIsolation() {
+ Time time = new MockTime();
+ Metrics metrics = new Metrics(time);
+
+ try (CoordinatorRuntimeMetricsImpl runtimeMetrics = new CoordinatorRuntimeMetricsImpl(metrics, METRICS_GROUP);
+ CoordinatorRuntimeMetricsImpl runtimeMetrics2 = new CoordinatorRuntimeMetricsImpl(metrics, OTHER_METRICS_GROUP)) {
+ runtimeMetrics.recordThreadIdleTime(1000.0);
+
+ org.apache.kafka.common.MetricName metricName = kafkaMetricName(metrics, "thread-idle-ratio-avg");
+ org.apache.kafka.common.MetricName otherGroupMetricName = otherGroupKafkaMetricName(metrics, "thread-idle-ratio-avg");
+ assertNotEquals(0.0, metrics.metrics().get(metricName).metricValue());
+ assertEquals(0.0, metrics.metrics().get(otherGroupMetricName).metricValue());
+ }
+ }
+
@Test
public void testEventQueueSize() {
Time time = new MockTime();
@@ -155,6 +216,21 @@ public void testEventQueueSize() {
}
}
+ @Test
+ public void testEventQueueSizeMetricsGroupIsolation() {
+ Time time = new MockTime();
+ Metrics metrics = new Metrics(time);
+
+ try (CoordinatorRuntimeMetricsImpl runtimeMetrics = new CoordinatorRuntimeMetricsImpl(metrics, METRICS_GROUP);
+ CoordinatorRuntimeMetricsImpl otherRuntimeMetrics = new CoordinatorRuntimeMetricsImpl(metrics, OTHER_METRICS_GROUP)) {
+ runtimeMetrics.registerEventQueueSizeGauge(() -> 5);
+ otherRuntimeMetrics.registerEventQueueSizeGauge(() -> 0);
+
+ assertMetricGauge(metrics, kafkaMetricName(metrics, "event-queue-size"), 5);
+ assertMetricGauge(metrics, otherGroupKafkaMetricName(metrics, "event-queue-size"), 0);
+ }
+ }
+
@ParameterizedTest
@ValueSource(strings = {
EVENT_QUEUE_TIME_METRIC_NAME,
@@ -205,6 +281,45 @@ public void testHistogramMetrics(String metricNamePrefix) {
assertEquals(999.0, metric.metricValue());
}
+ @ParameterizedTest
+ @ValueSource(strings = {
+ EVENT_QUEUE_TIME_METRIC_NAME,
+ EVENT_PROCESSING_TIME_METRIC_NAME,
+ EVENT_PURGATORY_TIME_METRIC_NAME,
+ BATCH_FLUSH_TIME_METRIC_NAME
+ })
+ public void testHistogramMetricsGroupIsolation(String metricNamePrefix) {
+ Time time = new MockTime();
+ Metrics metrics = new Metrics(time);
+
+ try (CoordinatorRuntimeMetricsImpl runtimeMetrics = new CoordinatorRuntimeMetricsImpl(metrics, METRICS_GROUP);
+ CoordinatorRuntimeMetricsImpl runtimeMetrics2 = new CoordinatorRuntimeMetricsImpl(metrics, OTHER_METRICS_GROUP)) {
+ switch (metricNamePrefix) {
+ case EVENT_QUEUE_TIME_METRIC_NAME:
+ runtimeMetrics.recordEventQueueTime(1000);
+ break;
+ case EVENT_PROCESSING_TIME_METRIC_NAME:
+ runtimeMetrics.recordEventProcessingTime(1000);
+ break;
+ case EVENT_PURGATORY_TIME_METRIC_NAME:
+ runtimeMetrics.recordEventPurgatoryTime(1000);
+ break;
+ case BATCH_FLUSH_TIME_METRIC_NAME:
+ runtimeMetrics.recordFlushTime(1000);
+ }
+
+ // Check metric group isolation
+ for (String suffix : List.of("-max", "-p50", "-p95", "-p99", "-p999")) {
+ org.apache.kafka.common.MetricName metricName = kafkaMetricName(metrics, metricNamePrefix + suffix);
+ org.apache.kafka.common.MetricName otherGroupMetricName = otherGroupKafkaMetricName(metrics, metricNamePrefix + suffix);
+ KafkaMetric metric = metrics.metrics().get(metricName);
+ KafkaMetric otherMetric = metrics.metrics().get(otherGroupMetricName);
+ assertNotEquals(0.0, metric.metricValue());
+ assertEquals(0.0, otherMetric.metricValue());
+ }
+ }
+ }
+
@Test
public void testRecordEventPurgatoryTimeLimit() {
Time time = new MockTime();
@@ -229,4 +344,8 @@ private static void assertMetricGauge(Metrics metrics, org.apache.kafka.common.M
private static MetricName kafkaMetricName(Metrics metrics, String name, String... keyValue) {
return metrics.metricName(name, METRICS_GROUP, "", keyValue);
}
+
+ private static MetricName otherGroupKafkaMetricName(Metrics metrics, String name, String... keyValue) {
+ return metrics.metricName(name, OTHER_METRICS_GROUP, "", keyValue);
+ }
}
diff --git a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java
index b82829e1d627d..f0d38d749fa2c 100644
--- a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java
+++ b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java
@@ -18,6 +18,7 @@
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.config.TopicConfig;
import org.apache.kafka.common.errors.NotCoordinatorException;
import org.apache.kafka.common.errors.NotEnoughReplicasException;
import org.apache.kafka.common.errors.RecordTooLargeException;
@@ -44,6 +45,7 @@
import org.mockito.ArgumentMatcher;
import java.nio.BufferOverflowException;
+import java.nio.ByteBuffer;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Arrays;
@@ -66,7 +68,7 @@
import static org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime.CoordinatorState.INITIAL;
import static org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime.CoordinatorState.LOADING;
import static org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime.HighWatermarkListener.NO_OFFSET;
-import static org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime.MIN_BUFFER_SIZE;
+import static org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime.INITIAL_BUFFER_SIZE;
import static org.apache.kafka.coordinator.common.runtime.TestUtil.endTransactionMarker;
import static org.apache.kafka.coordinator.common.runtime.TestUtil.records;
import static org.apache.kafka.coordinator.common.runtime.TestUtil.transactionalRecords;
@@ -2919,11 +2921,11 @@ public void testAppendRecordBatchSize() {
assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList());
int maxBatchSize = writer.config(TP).maxMessageSize();
- assertTrue(maxBatchSize > MIN_BUFFER_SIZE);
+ assertTrue(maxBatchSize > INITIAL_BUFFER_SIZE);
- // Generate enough records to create a batch that has 16KB < batchSize < maxBatchSize
+ // Generate enough records to create a batch that has INITIAL_BUFFER_SIZE < batchSize < maxBatchSize
List records = new ArrayList<>();
- for (int i = 0; i < 3000; i++) {
+ for (int i = 0; i < 50000; i++) {
records.add("record-" + i);
}
@@ -2937,7 +2939,210 @@ public void testAppendRecordBatchSize() {
assertFalse(write1.isCompletedExceptionally());
int batchSize = writer.entries(TP).get(0).sizeInBytes();
- assertTrue(batchSize > MIN_BUFFER_SIZE && batchSize < maxBatchSize);
+ assertTrue(batchSize > INITIAL_BUFFER_SIZE && batchSize < maxBatchSize);
+ }
+
+ @Test
+ public void testCoordinatorDoNotRetainBufferLargeThanMaxMessageSize() {
+ MockTimer timer = new MockTimer();
+ InMemoryPartitionWriter mockWriter = new InMemoryPartitionWriter(false) {
+ @Override
+ public LogConfig config(TopicPartition tp) {
+ return new LogConfig(Map.of(
+ TopicConfig.MAX_MESSAGE_BYTES_CONFIG, String.valueOf(1024 * 1024) // 1MB
+ ));
+ }
+ };
+ StringSerializer serializer = new StringSerializer();
+
+ CoordinatorRuntime runtime =
+ new CoordinatorRuntime.Builder()
+ .withTime(timer.time())
+ .withTimer(timer)
+ .withDefaultWriteTimeOut(DEFAULT_WRITE_TIMEOUT)
+ .withLoader(new MockCoordinatorLoader())
+ .withEventProcessor(new DirectEventProcessor())
+ .withPartitionWriter(mockWriter)
+ .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier())
+ .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
+ .withCoordinatorMetrics(mock(CoordinatorMetrics.class))
+ .withSerializer(serializer)
+ .withExecutorService(mock(ExecutorService.class))
+ .build();
+
+ // Schedule the loading.
+ runtime.scheduleLoadOperation(TP, 10);
+
+ // Verify the initial state.
+ CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP);
+ assertEquals(0L, ctx.coordinator.lastWrittenOffset());
+ assertEquals(0L, ctx.coordinator.lastCommittedOffset());
+ assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList());
+
+ // Generate a record larger than the maxBatchSize.
+ List largeRecords = List.of("A".repeat(100 * 1024 * 1024));
+
+ // Write #1.
+ CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, DEFAULT_WRITE_TIMEOUT,
+ state -> new CoordinatorResult<>(largeRecords, "response1", null, true, false)
+ );
+
+ // Verify that the write has not completed exceptionally.
+ // This will catch any exceptions thrown including RecordTooLargeException.
+ assertFalse(write1.isCompletedExceptionally());
+
+ // Verify that the next buffer retrieved from the bufferSupplier is the initial small one, not the large buffer.
+ assertEquals(INITIAL_BUFFER_SIZE, ctx.bufferSupplier.get(1).capacity());
+ }
+
+ @Test
+ public void testCoordinatorRetainExpandedBufferLessOrEqualToMaxMessageSize() {
+ MockTimer timer = new MockTimer();
+ InMemoryPartitionWriter mockWriter = new InMemoryPartitionWriter(false) {
+ @Override
+ public LogConfig config(TopicPartition tp) {
+ return new LogConfig(Map.of(
+ TopicConfig.MAX_MESSAGE_BYTES_CONFIG, String.valueOf(1024 * 1024 * 1024) // 1GB
+ ));
+ }
+ };
+ StringSerializer serializer = new StringSerializer();
+
+ CoordinatorRuntime runtime =
+ new CoordinatorRuntime.Builder()
+ .withTime(timer.time())
+ .withTimer(timer)
+ .withDefaultWriteTimeOut(DEFAULT_WRITE_TIMEOUT)
+ .withLoader(new MockCoordinatorLoader())
+ .withEventProcessor(new DirectEventProcessor())
+ .withPartitionWriter(mockWriter)
+ .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier())
+ .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
+ .withCoordinatorMetrics(mock(CoordinatorMetrics.class))
+ .withSerializer(serializer)
+ .withExecutorService(mock(ExecutorService.class))
+ .build();
+
+ // Schedule the loading.
+ runtime.scheduleLoadOperation(TP, 10);
+
+ // Verify the initial state.
+ CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP);
+ assertEquals(0L, ctx.coordinator.lastWrittenOffset());
+ assertEquals(0L, ctx.coordinator.lastCommittedOffset());
+ assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList());
+
+ // Generate enough records to create a batch that has INITIAL_BUFFER_SIZE < batchSize < maxBatchSize
+ List records = new ArrayList<>();
+ for (int i = 0; i < 1000000; i++) {
+ records.add("record-" + i);
+ }
+
+ // Write #1.
+ CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, DEFAULT_WRITE_TIMEOUT,
+ state -> new CoordinatorResult<>(records, "response1")
+ );
+
+ // Verify that the write has not completed exceptionally.
+ // This will catch any exceptions thrown including RecordTooLargeException.
+ assertFalse(write1.isCompletedExceptionally());
+
+ int batchSize = mockWriter.entries(TP).get(0).sizeInBytes();
+ int maxBatchSize = mockWriter.config(TP).maxMessageSize();
+ assertTrue(INITIAL_BUFFER_SIZE < batchSize && batchSize <= maxBatchSize);
+
+ // Verify that the next buffer retrieved from the bufferSupplier is the expanded buffer.
+ assertTrue(ctx.bufferSupplier.get(1).capacity() > INITIAL_BUFFER_SIZE);
+ }
+
+ @Test
+ public void testBufferShrinkWhenMaxMessageSizeReducedBelowInitialBufferSize() {
+ MockTimer timer = new MockTimer();
+ var mockWriter = new InMemoryPartitionWriter(false) {
+ private LogConfig config = new LogConfig(Map.of(
+ TopicConfig.MAX_MESSAGE_BYTES_CONFIG, String.valueOf(1024 * 1024) // 1MB
+ ));
+
+ @Override
+ public LogConfig config(TopicPartition tp) {
+ return config;
+ }
+
+ public void updateConfig(LogConfig newConfig) {
+ this.config = newConfig;
+ }
+ };
+ StringSerializer serializer = new StringSerializer();
+
+ CoordinatorRuntime runtime =
+ new CoordinatorRuntime.Builder()
+ .withTime(timer.time())
+ .withTimer(timer)
+ .withDefaultWriteTimeOut(DEFAULT_WRITE_TIMEOUT)
+ .withLoader(new MockCoordinatorLoader())
+ .withEventProcessor(new DirectEventProcessor())
+ .withPartitionWriter(mockWriter)
+ .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier())
+ .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
+ .withCoordinatorMetrics(mock(CoordinatorMetrics.class))
+ .withSerializer(serializer)
+ .withExecutorService(mock(ExecutorService.class))
+ .build();
+
+ // Schedule the loading.
+ runtime.scheduleLoadOperation(TP, 10);
+
+ // Verify the initial state.
+ CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP);
+ assertEquals(0L, ctx.coordinator.lastWrittenOffset());
+ assertEquals(0L, ctx.coordinator.lastCommittedOffset());
+ assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList());
+
+ List records = new ArrayList<>();
+ for (int i = 0; i < 1000; i++) {
+ records.add("record-" + i);
+ }
+
+ // Write #1.
+ CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, DEFAULT_WRITE_TIMEOUT,
+ state -> new CoordinatorResult<>(records, "response1")
+ );
+
+ // Verify that the write has not completed exceptionally.
+ // This will catch any exceptions thrown including RecordTooLargeException.
+ assertFalse(write1.isCompletedExceptionally());
+
+ int batchSize = mockWriter.entries(TP).get(0).sizeInBytes();
+ int maxBatchSize = mockWriter.config(TP).maxMessageSize();
+ assertTrue(batchSize <= INITIAL_BUFFER_SIZE && INITIAL_BUFFER_SIZE <= maxBatchSize);
+
+ ByteBuffer cachedBuffer = ctx.bufferSupplier.get(1);
+ assertEquals(INITIAL_BUFFER_SIZE, cachedBuffer.capacity());
+ // ctx.bufferSupplier.get(1); will clear cachedBuffer in bufferSupplier. Use release to put it back to bufferSupplier
+ ctx.bufferSupplier.release(cachedBuffer);
+
+ // Reduce max message size below initial buffer size.
+ mockWriter.updateConfig(new LogConfig(
+ Map.of(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, String.valueOf(INITIAL_BUFFER_SIZE - 66))));
+ assertEquals(INITIAL_BUFFER_SIZE - 66, mockWriter.config(TP).maxMessageSize());
+
+ // Write #2.
+ CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, DEFAULT_WRITE_TIMEOUT,
+ state -> new CoordinatorResult<>(records, "response2")
+ );
+ assertFalse(write2.isCompletedExceptionally());
+
+ // Verify that there is no cached buffer since the cached buffer size is greater than new maxMessageSize.
+ assertEquals(1, ctx.bufferSupplier.get(1).capacity());
+
+ // Write #3.
+ CompletableFuture write3 = runtime.scheduleWriteOperation("write#3", TP, DEFAULT_WRITE_TIMEOUT,
+ state -> new CoordinatorResult<>(records, "response3")
+ );
+ assertFalse(write3.isCompletedExceptionally());
+
+ // Verify that the cached buffer size is equals to new maxMessageSize that less than INITIAL_BUFFER_SIZE.
+ assertEquals(mockWriter.config(TP).maxMessageSize(), ctx.bufferSupplier.get(1).capacity());
}
@Test
diff --git a/core/src/main/java/kafka/server/share/DelayedShareFetch.java b/core/src/main/java/kafka/server/share/DelayedShareFetch.java
index fce50613becf8..2d01fb959cb10 100644
--- a/core/src/main/java/kafka/server/share/DelayedShareFetch.java
+++ b/core/src/main/java/kafka/server/share/DelayedShareFetch.java
@@ -34,6 +34,7 @@
import org.apache.kafka.server.purgatory.DelayedOperation;
import org.apache.kafka.server.share.SharePartitionKey;
import org.apache.kafka.server.share.fetch.DelayedShareFetchGroupKey;
+import org.apache.kafka.server.share.fetch.DelayedShareFetchPartitionKey;
import org.apache.kafka.server.share.fetch.PartitionMaxBytesStrategy;
import org.apache.kafka.server.share.fetch.ShareFetch;
import org.apache.kafka.server.share.fetch.ShareFetchPartitionData;
@@ -804,13 +805,22 @@ private void releasePartitionLocksAndAddToActionQueue(Set topi
}
// Releasing the lock to move ahead with the next request in queue.
releasePartitionLocks(topicIdPartitions);
- // If we have a fetch request completed for a topic-partition, we release the locks for that partition,
- // then we should check if there is a pending share fetch request for the topic-partition and complete it.
- // We add the action to delayed actions queue to avoid an infinite call stack, which could happen if
- // we directly call delayedShareFetchPurgatory.checkAndComplete
- replicaManager.addToActionQueue(() -> topicIdPartitions.forEach(topicIdPartition ->
+ replicaManager.addToActionQueue(() -> topicIdPartitions.forEach(topicIdPartition -> {
+ // If we have a fetch request completed for a share-partition, we release the locks for that partition,
+ // then we should check if there is a pending share fetch request for the share-partition and complete it.
+ // We add the action to delayed actions queue to avoid an infinite call stack, which could happen if
+ // we directly call delayedShareFetchPurgatory.checkAndComplete.
replicaManager.completeDelayedShareFetchRequest(
- new DelayedShareFetchGroupKey(shareFetch.groupId(), topicIdPartition.topicId(), topicIdPartition.partition()))));
+ new DelayedShareFetchGroupKey(shareFetch.groupId(), topicIdPartition.topicId(), topicIdPartition.partition()));
+ // As DelayedShareFetch operation is watched over multiple keys, same operation might be
+ // completed and can contain references to data fetched. Hence, if the operation is not
+ // removed from other watched keys then there can be a memory leak. The removal of the
+ // operation is dependent on the purge task by DelayedOperationPurgatory. Hence, this can
+ // also be prevented by setting smaller value for configuration {@link ShareGroupConfig#SHARE_FETCH_PURGATORY_PURGE_INTERVAL_REQUESTS_CONFIG}.
+ // However, it's best to trigger the check on all the keys that are being watched which
+ // should free the memory for the completed operation.
+ replicaManager.completeDelayedShareFetchRequest(new DelayedShareFetchPartitionKey(topicIdPartition));
+ }));
}
/**
diff --git a/core/src/main/java/kafka/server/share/SharePartition.java b/core/src/main/java/kafka/server/share/SharePartition.java
index 91a11d488f49f..09385d6c48c6c 100644
--- a/core/src/main/java/kafka/server/share/SharePartition.java
+++ b/core/src/main/java/kafka/server/share/SharePartition.java
@@ -23,7 +23,6 @@
import org.apache.kafka.common.TopicIdPartition;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.errors.CoordinatorNotAvailableException;
-import org.apache.kafka.common.errors.FencedStateEpochException;
import org.apache.kafka.common.errors.GroupIdNotFoundException;
import org.apache.kafka.common.errors.InvalidRecordStateException;
import org.apache.kafka.common.errors.InvalidRequestException;
@@ -1490,7 +1489,7 @@ private boolean initializedOrThrowException() {
String.format("Share partition failed to load %s-%s", groupId, topicIdPartition));
case INITIALIZING -> throw new LeaderNotAvailableException(
String.format("Share partition is already initializing %s-%s", groupId, topicIdPartition));
- case FENCED -> throw new FencedStateEpochException(
+ case FENCED -> throw new LeaderNotAvailableException(
String.format("Share partition is fenced %s-%s", groupId, topicIdPartition));
case EMPTY ->
// The share partition is not yet initialized.
@@ -2343,9 +2342,7 @@ private KafkaException fetchPersisterError(short errorCode, String errorMessage)
new GroupIdNotFoundException(errorMessage);
case UNKNOWN_TOPIC_OR_PARTITION ->
new UnknownTopicOrPartitionException(errorMessage);
- case FENCED_STATE_EPOCH ->
- new FencedStateEpochException(errorMessage);
- case FENCED_LEADER_EPOCH ->
+ case FENCED_LEADER_EPOCH, FENCED_STATE_EPOCH ->
new NotLeaderOrFollowerException(errorMessage);
default ->
new UnknownServerException(errorMessage);
diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala
index 30f7fb6cf8624..e57cf18d0a192 100644
--- a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala
+++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala
@@ -819,11 +819,9 @@ class TransactionCoordinator(txnConfig: TransactionConfig,
}
if (nextState == TransactionState.PREPARE_ABORT && isEpochFence) {
- // We should clear the pending state to make way for the transition to PrepareAbort and also bump
- // the epoch in the transaction metadata we are about to append.
+ // We should clear the pending state to make way for the transition to PrepareAbort
txnMetadata.pendingState = None
- txnMetadata.producerEpoch = producerEpoch
- txnMetadata.lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH
+ // For TV2+, don't manually set the epoch - let prepareAbortOrCommit handle it naturally.
}
nextProducerIdOrErrors.flatMap {
@@ -979,10 +977,10 @@ class TransactionCoordinator(txnConfig: TransactionConfig,
case Some(epochAndMetadata) =>
if (epochAndMetadata.coordinatorEpoch == coordinatorEpoch) {
- // This was attempted epoch fence that failed, so mark this state on the metadata
- epochAndMetadata.transactionMetadata.hasFailedEpochFence = true
+ // For TV2, we allow re-bumping the epoch on retry, since we don't complete the epoch bump.
+ // Therefore, we don't set hasFailedEpochFence = true.
warn(s"The coordinator failed to write an epoch fence transition for producer $transactionalId to the transaction log " +
- s"with error $error. The epoch was increased to ${newMetadata.producerEpoch} but not returned to the client")
+ s"with error $error")
}
}
}
diff --git a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala
index ae3a5d8c3c748..2368ebc21ccd2 100644
--- a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala
+++ b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala
@@ -28,7 +28,7 @@ import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.requests.{BrokerHeartbeatRequest, BrokerHeartbeatResponse, BrokerRegistrationRequest, BrokerRegistrationResponse}
import org.apache.kafka.metadata.{BrokerState, VersionRange}
import org.apache.kafka.queue.EventQueue.DeadlineFunction
-import org.apache.kafka.common.utils.{ExponentialBackoff, LogContext, Time}
+import org.apache.kafka.common.utils.{LogContext, Time}
import org.apache.kafka.queue.{EventQueue, KafkaEventQueue}
import org.apache.kafka.server.common.{ControllerRequestCompletionHandler, NodeToControllerChannelManager}
@@ -89,18 +89,6 @@ class BrokerLifecycleManager(
private val initialTimeoutNs =
MILLISECONDS.toNanos(config.initialRegistrationTimeoutMs.longValue())
- /**
- * The exponential backoff to use for resending communication.
- */
- private val resendExponentialBackoff =
- new ExponentialBackoff(100, 2, config.brokerSessionTimeoutMs.toLong / 2, 0.02)
-
- /**
- * The number of times we've tried and failed to communicate. This variable can only be
- * read or written from the BrokerToControllerRequestThread.
- */
- private var failedAttempts = 0L
-
/**
* The broker incarnation ID. This ID uniquely identifies each time we start the broker
*/
@@ -440,7 +428,6 @@ class BrokerLifecycleManager(
val message = response.responseBody().asInstanceOf[BrokerRegistrationResponse]
val errorCode = Errors.forCode(message.data().errorCode())
if (errorCode == Errors.NONE) {
- failedAttempts = 0
_brokerEpoch = message.data().brokerEpoch()
registered = true
initialRegistrationSucceeded = true
@@ -514,7 +501,6 @@ class BrokerLifecycleManager(
val errorCode = Errors.forCode(message.data().errorCode())
if (errorCode == Errors.NONE) {
val responseData = message.data()
- failedAttempts = 0
currentOfflineDirs.foreach(cur => offlineDirs.put(cur, true))
_state match {
case BrokerState.STARTING =>
@@ -577,10 +563,9 @@ class BrokerLifecycleManager(
}
private def scheduleNextCommunicationAfterFailure(): Unit = {
- val delayMs = resendExponentialBackoff.backoff(failedAttempts)
- failedAttempts = failedAttempts + 1
nextSchedulingShouldBeImmediate = false // never immediately reschedule after a failure
- scheduleNextCommunication(NANOSECONDS.convert(delayMs, MILLISECONDS))
+ scheduleNextCommunication(NANOSECONDS.convert(
+ config.brokerHeartbeatIntervalMs.longValue() , MILLISECONDS))
}
private def scheduleNextCommunicationAfterSuccess(): Unit = {
diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala
index 22c6847ded085..c56e178ca2e27 100644
--- a/core/src/main/scala/kafka/server/BrokerServer.scala
+++ b/core/src/main/scala/kafka/server/BrokerServer.scala
@@ -412,7 +412,7 @@ class BrokerServer(
config,
"heartbeat",
s"broker-${config.nodeId}-",
- config.brokerSessionTimeoutMs / 2 // KAFKA-14392
+ config.brokerHeartbeatIntervalMs
)
lifecycleManager.start(
() => sharedServer.loader.lastAppliedOffset(),
diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala
index bdd9c94a319d9..3e55df883a7e1 100755
--- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala
+++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala
@@ -238,7 +238,7 @@ object DynamicBrokerConfig {
}
}
val configHandler = new BrokerConfigHandler(config, quotaManagers)
- configHandler.processConfigChanges("", dynamicPerBrokerConfigs)
+ configHandler.processConfigChanges("", dynamicDefaultConfigs)
configHandler.processConfigChanges(config.brokerId.toString, dynamicPerBrokerConfigs)
}
}
diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala
index 3ca43aa145b52..5eb249c54d6b0 100644
--- a/core/src/main/scala/kafka/server/KafkaApis.scala
+++ b/core/src/main/scala/kafka/server/KafkaApis.scala
@@ -1016,9 +1016,11 @@ class KafkaApis(val requestChannel: RequestChannel,
groups.forEach { groupOffsetFetch =>
val isAllPartitions = groupOffsetFetch.topics == null
if (!authHelper.authorize(request.context, DESCRIBE, GROUP, groupOffsetFetch.groupId)) {
- futures += CompletableFuture.completedFuture(new OffsetFetchResponseData.OffsetFetchResponseGroup()
- .setGroupId(groupOffsetFetch.groupId)
- .setErrorCode(Errors.GROUP_AUTHORIZATION_FAILED.code))
+ futures += CompletableFuture.completedFuture(OffsetFetchResponse.groupError(
+ groupOffsetFetch,
+ Errors.GROUP_AUTHORIZATION_FAILED,
+ request.header.apiVersion()
+ ))
} else if (isAllPartitions) {
futures += fetchAllOffsetsForGroup(
request.context,
@@ -1043,33 +1045,35 @@ class KafkaApis(val requestChannel: RequestChannel,
private def fetchAllOffsetsForGroup(
requestContext: RequestContext,
- offsetFetchRequest: OffsetFetchRequestData.OffsetFetchRequestGroup,
+ groupFetchRequest: OffsetFetchRequestData.OffsetFetchRequestGroup,
requireStable: Boolean
): CompletableFuture[OffsetFetchResponseData.OffsetFetchResponseGroup] = {
val useTopicIds = OffsetFetchRequest.useTopicIds(requestContext.apiVersion)
groupCoordinator.fetchAllOffsets(
requestContext,
- offsetFetchRequest,
+ groupFetchRequest,
requireStable
- ).handle[OffsetFetchResponseData.OffsetFetchResponseGroup] { (offsetFetchResponse, exception) =>
+ ).handle[OffsetFetchResponseData.OffsetFetchResponseGroup] { (groupFetchResponse, exception) =>
if (exception != null) {
- new OffsetFetchResponseData.OffsetFetchResponseGroup()
- .setGroupId(offsetFetchRequest.groupId)
- .setErrorCode(Errors.forException(exception).code)
- } else if (offsetFetchResponse.errorCode() != Errors.NONE.code) {
- offsetFetchResponse
+ OffsetFetchResponse.groupError(
+ groupFetchRequest,
+ Errors.forException(exception),
+ requestContext.apiVersion()
+ )
+ } else if (groupFetchResponse.errorCode() != Errors.NONE.code) {
+ groupFetchResponse
} else {
// Clients are not allowed to see offsets for topics that are not authorized for Describe.
val authorizedNames = authHelper.filterByAuthorized(
requestContext,
DESCRIBE,
TOPIC,
- offsetFetchResponse.topics.asScala
+ groupFetchResponse.topics.asScala
)(_.name)
val topics = new mutable.ArrayBuffer[OffsetFetchResponseData.OffsetFetchResponseTopics]
- offsetFetchResponse.topics.forEach { topic =>
+ groupFetchResponse.topics.forEach { topic =>
if (authorizedNames.contains(topic.name)) {
if (useTopicIds) {
// If the topic is not provided by the group coordinator, we set it
@@ -1087,20 +1091,20 @@ class KafkaApis(val requestChannel: RequestChannel,
}
}
}
- offsetFetchResponse.setTopics(topics.asJava)
+ groupFetchResponse.setTopics(topics.asJava)
}
}
}
private def fetchOffsetsForGroup(
requestContext: RequestContext,
- offsetFetchRequest: OffsetFetchRequestData.OffsetFetchRequestGroup,
+ groupFetchRequest: OffsetFetchRequestData.OffsetFetchRequestGroup,
requireStable: Boolean
): CompletableFuture[OffsetFetchResponseData.OffsetFetchResponseGroup] = {
val useTopicIds = OffsetFetchRequest.useTopicIds(requestContext.apiVersion)
if (useTopicIds) {
- offsetFetchRequest.topics.forEach { topic =>
+ groupFetchRequest.topics.forEach { topic =>
if (topic.topicId != Uuid.ZERO_UUID) {
metadataCache.getTopicName(topic.topicId).ifPresent(name => topic.setName(name))
}
@@ -1112,7 +1116,7 @@ class KafkaApis(val requestChannel: RequestChannel,
requestContext,
DESCRIBE,
TOPIC,
- offsetFetchRequest.topics.asScala
+ groupFetchRequest.topics.asScala
)(_.name)
val authorizedTopics = new mutable.ArrayBuffer[OffsetFetchRequestData.OffsetFetchRequestTopics]
@@ -1134,7 +1138,7 @@ class KafkaApis(val requestChannel: RequestChannel,
topicResponse
}
- offsetFetchRequest.topics.forEach { topic =>
+ groupFetchRequest.topics.forEach { topic =>
if (useTopicIds && topic.name.isEmpty) {
errorTopics += buildErrorResponse(topic, Errors.UNKNOWN_TOPIC_ID)
} else if (!authorizedTopicNames.contains(topic.name)) {
@@ -1147,25 +1151,27 @@ class KafkaApis(val requestChannel: RequestChannel,
groupCoordinator.fetchOffsets(
requestContext,
new OffsetFetchRequestData.OffsetFetchRequestGroup()
- .setGroupId(offsetFetchRequest.groupId)
- .setMemberId(offsetFetchRequest.memberId)
- .setMemberEpoch(offsetFetchRequest.memberEpoch)
+ .setGroupId(groupFetchRequest.groupId)
+ .setMemberId(groupFetchRequest.memberId)
+ .setMemberEpoch(groupFetchRequest.memberEpoch)
.setTopics(authorizedTopics.asJava),
requireStable
- ).handle[OffsetFetchResponseData.OffsetFetchResponseGroup] { (offsetFetchResponse, exception) =>
+ ).handle[OffsetFetchResponseData.OffsetFetchResponseGroup] { (groupFetchResponse, exception) =>
if (exception != null) {
- new OffsetFetchResponseData.OffsetFetchResponseGroup()
- .setGroupId(offsetFetchRequest.groupId)
- .setErrorCode(Errors.forException(exception).code)
- } else if (offsetFetchResponse.errorCode() != Errors.NONE.code) {
- offsetFetchResponse
+ OffsetFetchResponse.groupError(
+ groupFetchRequest,
+ Errors.forException(exception),
+ requestContext.apiVersion()
+ )
+ } else if (groupFetchResponse.errorCode() != Errors.NONE.code) {
+ groupFetchResponse
} else {
val topics = new util.ArrayList[OffsetFetchResponseData.OffsetFetchResponseTopics](
- offsetFetchResponse.topics.size + errorTopics.size
+ groupFetchRequest.topics.size + errorTopics.size
)
- topics.addAll(offsetFetchResponse.topics)
+ topics.addAll(groupFetchResponse.topics)
topics.addAll(errorTopics.asJava)
- offsetFetchResponse.setTopics(topics)
+ groupFetchResponse.setTopics(topics)
}
}
}
@@ -2792,11 +2798,19 @@ class KafkaApis(val requestChannel: RequestChannel,
if (responseData.status() == null) {
responseData.setStatus(new util.ArrayList());
}
- responseData.status().add(
- new StreamsGroupHeartbeatResponseData.Status()
- .setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_INTERNAL_TOPICS.code())
- .setStatusDetail("Unauthorized to CREATE on topics " + createTopicUnauthorized.mkString(",") + ".")
- )
+ val missingInternalTopicStatus =
+ responseData.status().stream().filter(x => x.statusCode() == StreamsGroupHeartbeatResponse.Status.MISSING_INTERNAL_TOPICS.code()).findFirst()
+ if (missingInternalTopicStatus.isPresent) {
+ missingInternalTopicStatus.get().setStatusDetail(
+ missingInternalTopicStatus.get().statusDetail() + "; Unauthorized to CREATE on topics " + createTopicUnauthorized.mkString(", ") + "."
+ )
+ } else {
+ responseData.status().add(
+ new StreamsGroupHeartbeatResponseData.Status()
+ .setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_INTERNAL_TOPICS.code())
+ .setStatusDetail("Unauthorized to CREATE on topics " + createTopicUnauthorized.mkString(", ") + ".")
+ )
+ }
} else {
autoTopicCreationManager.createStreamsInternalTopics(topicsToCreate, requestContext);
}
diff --git a/core/src/test/java/kafka/server/share/SharePartitionTest.java b/core/src/test/java/kafka/server/share/SharePartitionTest.java
index ac8e9ba0fd6d8..0cc7ad4d9a18c 100644
--- a/core/src/test/java/kafka/server/share/SharePartitionTest.java
+++ b/core/src/test/java/kafka/server/share/SharePartitionTest.java
@@ -28,10 +28,10 @@
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.errors.CoordinatorNotAvailableException;
-import org.apache.kafka.common.errors.FencedStateEpochException;
import org.apache.kafka.common.errors.GroupIdNotFoundException;
import org.apache.kafka.common.errors.InvalidRecordStateException;
import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.LeaderNotAvailableException;
import org.apache.kafka.common.errors.NotLeaderOrFollowerException;
import org.apache.kafka.common.errors.UnknownServerException;
import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
@@ -758,7 +758,7 @@ public void testMaybeInitializeWithErrorPartitionResponse() {
result = sharePartition.maybeInitialize();
assertTrue(result.isDone());
assertTrue(result.isCompletedExceptionally());
- assertFutureThrows(FencedStateEpochException.class, result);
+ assertFutureThrows(NotLeaderOrFollowerException.class, result);
assertEquals(SharePartitionState.FAILED, sharePartition.partitionState());
// Mock FENCED_LEADER_EPOCH error.
@@ -788,6 +788,20 @@ public void testMaybeInitializeWithErrorPartitionResponse() {
assertTrue(result.isCompletedExceptionally());
assertFutureThrows(UnknownServerException.class, result);
assertEquals(SharePartitionState.FAILED, sharePartition.partitionState());
+
+ // Mock NETWORK_EXCEPTION error.
+ Mockito.when(readShareGroupStateResult.topicsData()).thenReturn(List.of(
+ new TopicData<>(TOPIC_ID_PARTITION.topicId(), List.of(
+ PartitionFactory.newPartitionAllData(0, 5, 10L, Errors.NETWORK_EXCEPTION.code(), Errors.NETWORK_EXCEPTION.message(),
+ List.of())))));
+ Mockito.when(persister.readState(Mockito.any())).thenReturn(CompletableFuture.completedFuture(readShareGroupStateResult));
+ sharePartition = SharePartitionBuilder.builder().withPersister(persister).build();
+
+ result = sharePartition.maybeInitialize();
+ assertTrue(result.isDone());
+ assertTrue(result.isCompletedExceptionally());
+ assertFutureThrows(UnknownServerException.class, result);
+ assertEquals(SharePartitionState.FAILED, sharePartition.partitionState());
}
@Test
@@ -935,6 +949,19 @@ public void testMaybeInitializeWithReadException() {
assertThrows(RuntimeException.class, sharePartition2::maybeInitialize);
}
+ @Test
+ public void testMaybeInitializeFencedSharePartition() {
+ SharePartition sharePartition = SharePartitionBuilder.builder().build();
+ // Mark the share partition as fenced.
+ sharePartition.markFenced();
+
+ CompletableFuture result = sharePartition.maybeInitialize();
+ assertTrue(result.isDone());
+ assertTrue(result.isCompletedExceptionally());
+ assertFutureThrows(LeaderNotAvailableException.class, result);
+ assertEquals(SharePartitionState.FENCED, sharePartition.partitionState());
+ }
+
@Test
public void testMaybeInitializeStateBatchesWithGapAtBeginning() {
Persister persister = Mockito.mock(Persister.class);
@@ -5564,7 +5591,7 @@ public void testWriteShareGroupStateFailure() {
result = sharePartition.writeShareGroupState(anyList());
assertTrue(result.isCompletedExceptionally());
- assertFutureThrows(FencedStateEpochException.class, result);
+ assertFutureThrows(NotLeaderOrFollowerException.class, result);
// Mock Write state RPC to return error response, FENCED_LEADER_EPOCH.
Mockito.when(writeShareGroupStateResult.topicsData()).thenReturn(List.of(
diff --git a/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala
index 54f6d71a27853..fafce17382c27 100644
--- a/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala
@@ -71,6 +71,7 @@ class AbstractAuthorizerIntegrationTest extends BaseRequestTest {
val brokerId: Integer = 0
val topic = "topic"
+ val sourceTopic = "source-topic"
val topicPattern = "topic.*"
val transactionalId = "transactional.id"
val producerId = 83392L
@@ -81,12 +82,15 @@ class AbstractAuthorizerIntegrationTest extends BaseRequestTest {
val logDir = "logDir"
val group = "my-group"
val shareGroup = "share-group"
+ val streamsGroup = "streams-group"
val protocolType = "consumer"
val protocolName = "consumer-range"
val clusterResource = new ResourcePattern(CLUSTER, Resource.CLUSTER_NAME, LITERAL)
val topicResource = new ResourcePattern(TOPIC, topic, LITERAL)
+ val sourceTopicResource = new ResourcePattern(TOPIC, sourceTopic, LITERAL)
val groupResource = new ResourcePattern(GROUP, group, LITERAL)
val shareGroupResource = new ResourcePattern(GROUP, shareGroup, LITERAL)
+ val streamsGroupResource = new ResourcePattern(GROUP, streamsGroup, LITERAL)
val transactionalIdResource = new ResourcePattern(TRANSACTIONAL_ID, transactionalId, LITERAL)
producerConfig.setProperty(ProducerConfig.ACKS_CONFIG, "1")
@@ -111,6 +115,7 @@ class AbstractAuthorizerIntegrationTest extends BaseRequestTest {
properties.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1")
properties.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "1")
+ properties.put(GroupCoordinatorConfig.CONSUMER_GROUP_REGEX_REFRESH_INTERVAL_MS_CONFIG, "10000")
properties.put(TransactionLogConfig.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, "1")
properties.put(TransactionLogConfig.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG, "1")
properties.put(TransactionLogConfig.TRANSACTIONS_TOPIC_MIN_ISR_CONFIG, "1")
diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
index 2c7189b5a705d..424772275ea0a 100644
--- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
@@ -17,11 +17,12 @@ import java.time.Duration
import java.util
import java.util.concurrent.{ExecutionException, Semaphore}
import java.util.regex.Pattern
-import java.util.{Comparator, Optional, Properties}
+import java.util.{Comparator, Optional, Properties, UUID}
import kafka.utils.{TestInfoUtils, TestUtils}
import kafka.utils.TestUtils.waitUntilTrue
import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, ListGroupsOptions, NewTopic}
import org.apache.kafka.clients.consumer._
+import org.apache.kafka.clients.consumer.internals.{StreamsRebalanceData, StreamsRebalanceListener}
import org.apache.kafka.clients.producer._
import org.apache.kafka.common.acl.AclOperation._
import org.apache.kafka.common.acl.AclPermissionType.{ALLOW, DENY}
@@ -37,7 +38,7 @@ import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProt
import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity
import org.apache.kafka.common.message.ListOffsetsRequestData.{ListOffsetsPartition, ListOffsetsTopic}
import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.{OffsetForLeaderPartition, OffsetForLeaderTopic, OffsetForLeaderTopicCollection}
-import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AlterPartitionReassignmentsRequestData, AlterReplicaLogDirsRequestData, AlterShareGroupOffsetsRequestData, ConsumerGroupDescribeRequestData, ConsumerGroupHeartbeatRequestData, ConsumerGroupHeartbeatResponseData, CreateAclsRequestData, CreatePartitionsRequestData, CreateTopicsRequestData, DeleteAclsRequestData, DeleteGroupsRequestData, DeleteRecordsRequestData, DeleteShareGroupOffsetsRequestData, DeleteShareGroupStateRequestData, DeleteTopicsRequestData, DescribeClusterRequestData, DescribeConfigsRequestData, DescribeGroupsRequestData, DescribeLogDirsRequestData, DescribeProducersRequestData, DescribeShareGroupOffsetsRequestData, DescribeTransactionsRequestData, FetchResponseData, FindCoordinatorRequestData, HeartbeatRequestData, IncrementalAlterConfigsRequestData, InitializeShareGroupStateRequestData, JoinGroupRequestData, ListPartitionReassignmentsRequestData, ListTransactionsRequestData, MetadataRequestData, OffsetCommitRequestData, OffsetFetchRequestData, OffsetFetchResponseData, ProduceRequestData, ReadShareGroupStateRequestData, ReadShareGroupStateSummaryRequestData, ShareAcknowledgeRequestData, ShareFetchRequestData, ShareGroupDescribeRequestData, ShareGroupHeartbeatRequestData, SyncGroupRequestData, WriteShareGroupStateRequestData, WriteTxnMarkersRequestData}
+import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AlterPartitionReassignmentsRequestData, AlterReplicaLogDirsRequestData, AlterShareGroupOffsetsRequestData, ConsumerGroupDescribeRequestData, ConsumerGroupHeartbeatRequestData, ConsumerGroupHeartbeatResponseData, CreateAclsRequestData, CreatePartitionsRequestData, CreateTopicsRequestData, DeleteAclsRequestData, DeleteGroupsRequestData, DeleteRecordsRequestData, DeleteShareGroupOffsetsRequestData, DeleteShareGroupStateRequestData, DeleteTopicsRequestData, DescribeClusterRequestData, DescribeConfigsRequestData, DescribeGroupsRequestData, DescribeLogDirsRequestData, DescribeProducersRequestData, DescribeShareGroupOffsetsRequestData, DescribeTransactionsRequestData, FetchResponseData, FindCoordinatorRequestData, HeartbeatRequestData, IncrementalAlterConfigsRequestData, InitializeShareGroupStateRequestData, JoinGroupRequestData, ListPartitionReassignmentsRequestData, ListTransactionsRequestData, MetadataRequestData, OffsetCommitRequestData, OffsetFetchRequestData, OffsetFetchResponseData, ProduceRequestData, ReadShareGroupStateRequestData, ReadShareGroupStateSummaryRequestData, ShareAcknowledgeRequestData, ShareFetchRequestData, ShareGroupDescribeRequestData, ShareGroupHeartbeatRequestData, StreamsGroupDescribeRequestData, StreamsGroupHeartbeatRequestData, StreamsGroupHeartbeatResponseData, SyncGroupRequestData, WriteShareGroupStateRequestData, WriteTxnMarkersRequestData}
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, SimpleRecord}
@@ -53,8 +54,7 @@ import org.apache.kafka.security.authorizer.AclEntry
import org.apache.kafka.security.authorizer.AclEntry.WILDCARD_HOST
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.params.ParameterizedTest
-import org.junit.jupiter.params.provider.{MethodSource, ValueSource}
-
+import org.junit.jupiter.params.provider.{CsvSource, MethodSource, ValueSource}
import org.apache.kafka.common.message.MetadataRequestData.MetadataRequestTopic
import org.apache.kafka.common.message.WriteTxnMarkersRequestData.{WritableTxnMarker, WritableTxnMarkerTopic}
import org.apache.kafka.coordinator.group.GroupConfig
@@ -76,6 +76,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
val shareGroupDeleteAcl = Map(shareGroupResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DELETE, ALLOW)))
val shareGroupDescribeConfigsAcl = Map(shareGroupResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE_CONFIGS, ALLOW)))
val shareGroupAlterConfigsAcl = Map(shareGroupResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALTER_CONFIGS, ALLOW)))
+ val streamsGroupReadAcl = Map(streamsGroupResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)))
+ val streamsGroupDescribeAcl = Map(streamsGroupResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)))
val clusterAcl = Map(clusterResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, CLUSTER_ACTION, ALLOW)))
val clusterCreateAcl = Map(clusterResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, CREATE, ALLOW)))
val clusterAlterAcl = Map(clusterResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALTER, ALLOW)))
@@ -92,6 +94,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
val topicAlterConfigsAcl = Map(topicResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALTER_CONFIGS, ALLOW)))
val transactionIdWriteAcl = Map(transactionalIdResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)))
val transactionalIdDescribeAcl = Map(transactionalIdResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)))
+ val sourceTopicDescribeAcl = Map(sourceTopicResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)))
val numRecords = 1
@@ -223,7 +226,10 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
ApiKeys.DELETE_SHARE_GROUP_OFFSETS -> ((resp: DeleteShareGroupOffsetsResponse) => Errors.forCode(
resp.data.errorCode)),
ApiKeys.ALTER_SHARE_GROUP_OFFSETS -> ((resp: AlterShareGroupOffsetsResponse) => Errors.forCode(
- resp.data.errorCode))
+ resp.data.errorCode)),
+ ApiKeys.STREAMS_GROUP_HEARTBEAT -> ((resp: StreamsGroupHeartbeatResponse) => Errors.forCode(resp.data.errorCode)),
+ ApiKeys.STREAMS_GROUP_DESCRIBE -> ((resp: StreamsGroupDescribeResponse) =>
+ Errors.forCode(resp.data.groups.asScala.find(g => streamsGroup == g.groupId).head.errorCode))
)
def findErrorForTopicId(id: Uuid, response: AbstractResponse): Errors = {
@@ -291,7 +297,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
ApiKeys.READ_SHARE_GROUP_STATE_SUMMARY -> clusterAcl,
ApiKeys.DESCRIBE_SHARE_GROUP_OFFSETS -> (shareGroupDescribeAcl ++ topicDescribeAcl),
ApiKeys.DELETE_SHARE_GROUP_OFFSETS -> (shareGroupDeleteAcl ++ topicReadAcl),
- ApiKeys.ALTER_SHARE_GROUP_OFFSETS -> (shareGroupReadAcl ++ topicReadAcl)
+ ApiKeys.ALTER_SHARE_GROUP_OFFSETS -> (shareGroupReadAcl ++ topicReadAcl),
+ ApiKeys.STREAMS_GROUP_HEARTBEAT -> (streamsGroupReadAcl ++ topicDescribeAcl),
+ ApiKeys.STREAMS_GROUP_DESCRIBE -> (streamsGroupDescribeAcl ++ topicDescribeAcl),
)
private def createMetadataRequest(allowAutoTopicCreation: Boolean) = {
@@ -825,6 +833,53 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
new AlterShareGroupOffsetsRequest.Builder(data).build(ApiKeys.ALTER_SHARE_GROUP_OFFSETS.latestVersion)
}
+ private def streamsGroupHeartbeatRequest = new StreamsGroupHeartbeatRequest.Builder(
+ new StreamsGroupHeartbeatRequestData()
+ .setGroupId(streamsGroup)
+ .setMemberId("member-id")
+ .setMemberEpoch(0)
+ .setRebalanceTimeoutMs(1000)
+ .setActiveTasks(List.empty.asJava)
+ .setStandbyTasks(List.empty.asJava)
+ .setWarmupTasks(List.empty.asJava)
+ .setTopology(new StreamsGroupHeartbeatRequestData.Topology().setSubtopologies(
+ List(new StreamsGroupHeartbeatRequestData.Subtopology()
+ .setSourceTopics(List(topic).asJava)
+ ).asJava
+ ))).build(ApiKeys.STREAMS_GROUP_HEARTBEAT.latestVersion)
+
+ private def streamsGroupHeartbeatRequest(
+ topicAsSourceTopic: Boolean,
+ topicAsRepartitionSinkTopic: Boolean,
+ topicAsRepartitionSourceTopic: Boolean,
+ topicAsStateChangelogTopics: Boolean
+ ) = new StreamsGroupHeartbeatRequest.Builder(
+ new StreamsGroupHeartbeatRequestData()
+ .setGroupId(streamsGroup)
+ .setMemberId("member-id")
+ .setMemberEpoch(0)
+ .setRebalanceTimeoutMs(1000)
+ .setActiveTasks(List.empty.asJava)
+ .setStandbyTasks(List.empty.asJava)
+ .setWarmupTasks(List.empty.asJava)
+ .setTopology(new StreamsGroupHeartbeatRequestData.Topology().setSubtopologies(
+ List(new StreamsGroupHeartbeatRequestData.Subtopology()
+ .setSourceTopics(
+ (if (topicAsSourceTopic) List(sourceTopic, topic) else List(sourceTopic)).asJava)
+ .setRepartitionSinkTopics(
+ (if (topicAsRepartitionSinkTopic) List(topic) else List.empty).asJava)
+ .setRepartitionSourceTopics(
+ (if (topicAsRepartitionSourceTopic) List(new StreamsGroupHeartbeatRequestData.TopicInfo().setName(topic).setPartitions(3)) else List.empty).asJava)
+ .setStateChangelogTopics(
+ (if (topicAsStateChangelogTopics) List(new StreamsGroupHeartbeatRequestData.TopicInfo().setName(topic)) else List.empty).asJava)
+ ).asJava
+ ))).build(ApiKeys.STREAMS_GROUP_HEARTBEAT.latestVersion)
+
+ private def streamsGroupDescribeRequest = new StreamsGroupDescribeRequest.Builder(
+ new StreamsGroupDescribeRequestData()
+ .setGroupIds(List(streamsGroup).asJava)
+ .setIncludeAuthorizedOperations(false)).build(ApiKeys.STREAMS_GROUP_DESCRIBE.latestVersion)
+
private def sendRequests(requestKeyToRequest: mutable.Map[ApiKeys, AbstractRequest], topicExists: Boolean = true,
topicNames: Map[Uuid, String] = getTopicNames()) = {
for ((key, request) <- requestKeyToRequest) {
@@ -908,6 +963,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
ApiKeys.DESCRIBE_SHARE_GROUP_OFFSETS -> describeShareGroupOffsetsRequest,
ApiKeys.DELETE_SHARE_GROUP_OFFSETS -> deleteShareGroupOffsetsRequest,
ApiKeys.ALTER_SHARE_GROUP_OFFSETS -> alterShareGroupOffsetsRequest,
+ ApiKeys.STREAMS_GROUP_HEARTBEAT -> streamsGroupHeartbeatRequest,
+ ApiKeys.STREAMS_GROUP_DESCRIBE -> streamsGroupDescribeRequest,
// Delete the topic last
ApiKeys.DELETE_TOPICS -> deleteTopicsRequest
@@ -940,7 +997,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
ApiKeys.ELECT_LEADERS -> electLeadersRequest,
ApiKeys.SHARE_FETCH -> createShareFetchRequest,
ApiKeys.SHARE_ACKNOWLEDGE -> shareAcknowledgeRequest,
- ApiKeys.DESCRIBE_SHARE_GROUP_OFFSETS -> describeShareGroupOffsetsRequest
+ ApiKeys.DESCRIBE_SHARE_GROUP_OFFSETS -> describeShareGroupOffsetsRequest,
+ ApiKeys.STREAMS_GROUP_HEARTBEAT -> streamsGroupHeartbeatRequest,
+ ApiKeys.STREAMS_GROUP_DESCRIBE -> streamsGroupDescribeRequest
)
sendRequests(requestKeyToRequest, topicExists = false, topicNames)
@@ -3076,6 +3135,29 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
sendAndReceiveRegexHeartbeat(response, listenerName, None)
}
+ @Test
+ def testConsumerGroupHeartbeatWithRegexWithTopicDescribeAclAddedAndRemoved(): Unit = {
+ createTopicWithBrokerPrincipal(topic)
+ val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW)
+ addAndVerifyAcls(Set(allowAllOpsAcl), groupResource)
+
+ val memberId = Uuid.randomUuid.toString;
+ var response = sendAndReceiveFirstRegexHeartbeat(memberId, listenerName)
+ TestUtils.tryUntilNoAssertionError() {
+ response = sendAndReceiveRegexHeartbeat(response, listenerName, Some(0), true)
+ }
+
+ addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource)
+ TestUtils.tryUntilNoAssertionError(waitTime = 25000) {
+ response = sendAndReceiveRegexHeartbeat(response, listenerName, Some(1))
+ }
+
+ removeAndVerifyAcls(topicDescribeAcl(topicResource), topicResource)
+ TestUtils.tryUntilNoAssertionError(waitTime = 25000) {
+ response = sendAndReceiveRegexHeartbeat(response, listenerName, Some(0))
+ }
+ }
+
@Test
def testConsumerGroupHeartbeatWithRegexWithDifferentMemberAcls(): Unit = {
createTopicWithBrokerPrincipal(topic, numPartitions = 2)
@@ -3093,7 +3175,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
// member permissions while computing assignments.
var member2Response = sendAndReceiveFirstRegexHeartbeat("memberWithLimitedAccess", listenerName)
member1Response = sendAndReceiveRegexHeartbeat(member1Response, interBrokerListenerName, Some(1))
- member1Response = sendAndReceiveRegexHeartbeat(member1Response, interBrokerListenerName, None, fullRequest = true)
+ member1Response = sendAndReceiveRegexHeartbeat(member1Response, interBrokerListenerName, Some(1), fullRequest = true)
member2Response = sendAndReceiveRegexHeartbeat(member2Response, listenerName, Some(1))
// Create another topic and send heartbeats on member1 to trigger regex refresh
@@ -3589,6 +3671,359 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED.code, response.data.responses.stream().findFirst().get().partitions.get(0).errorCode, s"Unexpected response $response")
}
+ @ParameterizedTest
+ @CsvSource(Array(
+ "true, false, false, false",
+ "false, true, false, false",
+ "false, false, true, false",
+ "false, false, false, true"
+ ))
+ def testStreamsGroupHeartbeatWithGroupReadAndTopicDescribeAcl(
+ topicAsSourceTopic: Boolean,
+ topicAsRepartitionSinkTopic: Boolean,
+ topicAsRepartitionSourceTopic: Boolean,
+ topicAsStateChangelogTopics: Boolean
+ ): Unit = {
+ addAndVerifyAcls(streamsGroupReadAcl(streamsGroupResource), streamsGroupResource)
+ addAndVerifyAcls(sourceTopicDescribeAcl(sourceTopicResource), sourceTopicResource) // Always added, since we need a source topic
+ addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource)
+
+ val request = streamsGroupHeartbeatRequest(
+ topicAsSourceTopic,
+ topicAsRepartitionSinkTopic,
+ topicAsRepartitionSourceTopic,
+ topicAsStateChangelogTopics
+ )
+ val resource = Set[ResourceType](GROUP, TOPIC)
+ sendRequestAndVerifyResponseError(request, resource, isAuthorized = true)
+ }
+
+ @ParameterizedTest
+ @CsvSource(Array(
+ "true, false, false, false",
+ "false, true, false, false",
+ "false, false, true, false",
+ "false, false, false, true"
+ ))
+ def testStreamsGroupHeartbeatWithOperationAll(
+ topicAsSourceTopic: Boolean,
+ topicAsRepartitionSinkTopic: Boolean,
+ topicAsRepartitionSourceTopic: Boolean,
+ topicAsStateChangelogTopics: Boolean
+ ): Unit = {
+ val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW)
+ addAndVerifyAcls(Set(allowAllOpsAcl), streamsGroupResource)
+ addAndVerifyAcls(Set(allowAllOpsAcl), topicResource)
+ addAndVerifyAcls(Set(allowAllOpsAcl), sourceTopicResource)
+
+ val request = streamsGroupHeartbeatRequest(
+ topicAsSourceTopic,
+ topicAsRepartitionSinkTopic,
+ topicAsRepartitionSourceTopic,
+ topicAsStateChangelogTopics
+ )
+ val resource = Set[ResourceType](GROUP, TOPIC)
+ sendRequestAndVerifyResponseError(request, resource, isAuthorized = true)
+ }
+
+ @ParameterizedTest
+ @CsvSource(Array(
+ "true, false, false, false",
+ "false, true, false, false",
+ "false, false, true, false",
+ "false, false, false, true"
+ ))
+ def testStreamsGroupHeartbeatWithoutGroupReadOrTopicDescribeAcl(
+ topicAsSourceTopic: Boolean,
+ topicAsRepartitionSinkTopic: Boolean,
+ topicAsRepartitionSourceTopic: Boolean,
+ topicAsStateChangelogTopics: Boolean
+ ): Unit = {
+ removeAllClientAcls()
+
+ val request = streamsGroupHeartbeatRequest(
+ topicAsSourceTopic,
+ topicAsRepartitionSinkTopic,
+ topicAsRepartitionSourceTopic,
+ topicAsStateChangelogTopics
+ )
+ val resource = Set[ResourceType](GROUP, TOPIC)
+ sendRequestAndVerifyResponseError(request, resource, isAuthorized = false)
+ }
+
+ @ParameterizedTest
+ @CsvSource(Array(
+ "true, false, false, false",
+ "false, true, false, false",
+ "false, false, true, false",
+ "false, false, false, true"
+ ))
+ def testStreamsGroupHeartbeatWithoutGroupReadAcl(
+ topicAsSourceTopic: Boolean,
+ topicAsRepartitionSinkTopic: Boolean,
+ topicAsRepartitionSourceTopic: Boolean,
+ topicAsStateChangelogTopics: Boolean
+ ): Unit = {
+ addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource)
+ addAndVerifyAcls(sourceTopicDescribeAcl(sourceTopicResource), sourceTopicResource) // Always added, since we need a source topic
+
+ val request = streamsGroupHeartbeatRequest(
+ topicAsSourceTopic,
+ topicAsRepartitionSinkTopic,
+ topicAsRepartitionSourceTopic,
+ topicAsStateChangelogTopics
+ )
+ val resource = Set[ResourceType](GROUP, TOPIC)
+ sendRequestAndVerifyResponseError(request, resource, isAuthorized = false)
+ }
+
+ @ParameterizedTest
+ @CsvSource(Array(
+ "true, false, false, false",
+ "false, true, false, false",
+ "false, false, true, false",
+ "false, false, false, true"
+ ))
+ def testStreamsGroupHeartbeatWithoutTopicDescribeAcl(
+ topicAsSourceTopic: Boolean,
+ topicAsRepartitionSinkTopic: Boolean,
+ topicAsRepartitionSourceTopic: Boolean,
+ topicAsStateChangelogTopics: Boolean
+ ): Unit = {
+ addAndVerifyAcls(streamsGroupReadAcl(streamsGroupResource), streamsGroupResource)
+ addAndVerifyAcls(sourceTopicDescribeAcl(sourceTopicResource), sourceTopicResource) // Always added, since we need a source topic
+
+ val request = streamsGroupHeartbeatRequest(
+ topicAsSourceTopic,
+ topicAsRepartitionSinkTopic,
+ topicAsRepartitionSourceTopic,
+ topicAsStateChangelogTopics
+ )
+ val resource = Set[ResourceType](GROUP, TOPIC)
+ sendRequestAndVerifyResponseError(request, resource, isAuthorized = false)
+ }
+
+ @ParameterizedTest
+ @CsvSource(Array(
+ "true, false",
+ "false, true"
+ ))
+ def testStreamsGroupHeartbeatWithoutInternalTopicCreateAcl(
+ topicAsRepartitionSourceTopic: Boolean,
+ topicAsStateChangelogTopics: Boolean
+ ): Unit = {
+ createTopicWithBrokerPrincipal(sourceTopic)
+ addAndVerifyAcls(streamsGroupReadAcl(streamsGroupResource), streamsGroupResource)
+ addAndVerifyAcls(sourceTopicDescribeAcl(sourceTopicResource), sourceTopicResource) // Always added, since we need a source topic
+ addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource)
+
+ val request = streamsGroupHeartbeatRequest(
+ topicAsSourceTopic = false,
+ topicAsRepartitionSinkTopic = false,
+ topicAsRepartitionSourceTopic = topicAsRepartitionSourceTopic,
+ topicAsStateChangelogTopics = topicAsStateChangelogTopics
+ )
+ val resource = Set[ResourceType](GROUP, TOPIC)
+
+ // Request successful, but internal topic not created.
+ val response = sendRequestAndVerifyResponseError(request, resource, isAuthorized = true).asInstanceOf[StreamsGroupHeartbeatResponse]
+ assertEquals(
+ util.List.of(new StreamsGroupHeartbeatResponseData.Status()
+ .setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_INTERNAL_TOPICS.code())
+ .setStatusDetail("Internal topics are missing: [topic]; Unauthorized to CREATE on topics topic.")),
+ response.data().status())
+ }
+
+ @ParameterizedTest
+ @CsvSource(Array(
+ "true, false",
+ "false, true"
+ ))
+ def testStreamsGroupHeartbeatWithInternalTopicCreateAcl(
+ topicAsRepartitionSourceTopic: Boolean,
+ topicAsStateChangelogTopics: Boolean
+ ): Unit = {
+ createTopicWithBrokerPrincipal(sourceTopic)
+ addAndVerifyAcls(streamsGroupReadAcl(streamsGroupResource), streamsGroupResource)
+ addAndVerifyAcls(sourceTopicDescribeAcl(sourceTopicResource), sourceTopicResource) // Always added, since we need a source topic
+ addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource)
+ addAndVerifyAcls(topicCreateAcl(topicResource), topicResource)
+
+ val request = streamsGroupHeartbeatRequest(
+ topicAsSourceTopic = false,
+ topicAsRepartitionSinkTopic = false,
+ topicAsRepartitionSourceTopic = topicAsRepartitionSourceTopic,
+ topicAsStateChangelogTopics = topicAsStateChangelogTopics
+ )
+ val resource = Set[ResourceType](GROUP, TOPIC)
+ val response = sendRequestAndVerifyResponseError(request, resource, isAuthorized = true).asInstanceOf[StreamsGroupHeartbeatResponse]
+ // Request successful, and no internal topic creation error.
+ assertEquals(
+ util.List.of(new StreamsGroupHeartbeatResponseData.Status()
+ .setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_INTERNAL_TOPICS.code())
+ .setStatusDetail("Internal topics are missing: [topic]")),
+ response.data().status())
+ }
+
+ private def createStreamsGroupToDescribe(
+ topicAsSourceTopic: Boolean,
+ topicAsRepartitionSinkTopic: Boolean,
+ topicAsRepartitionSourceTopic: Boolean,
+ topicAsStateChangelogTopics: Boolean
+ ): Unit = {
+ createTopicWithBrokerPrincipal(sourceTopic)
+ createTopicWithBrokerPrincipal(topic)
+ addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), streamsGroupResource)
+ addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), topicResource)
+ addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), sourceTopicResource)
+ streamsConsumerConfig.put(ConsumerConfig.GROUP_ID_CONFIG, streamsGroup)
+ streamsConsumerConfig.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false")
+ val consumer = createStreamsConsumer(streamsRebalanceData = new StreamsRebalanceData(
+ UUID.randomUUID(),
+ Optional.empty(),
+ util.Map.of(
+ "subtopology-0", new StreamsRebalanceData.Subtopology(
+ if (topicAsSourceTopic) util.Set.of(sourceTopic, topic) else util.Set.of(sourceTopic),
+ if (topicAsRepartitionSinkTopic) util.Set.of(topic) else util.Set.of(),
+ if (topicAsRepartitionSourceTopic)
+ util.Map.of(topic, new StreamsRebalanceData.TopicInfo(Optional.of(1), Optional.empty(), util.Map.of()))
+ else util.Map.of(),
+ if (topicAsStateChangelogTopics)
+ util.Map.of(topic, new StreamsRebalanceData.TopicInfo(Optional.of(1), Optional.empty(), util.Map.of()))
+ else util.Map.of(),
+ util.Set.of()
+ )),
+ Map.empty[String, String].asJava
+ ))
+ consumer.subscribe(
+ if (topicAsSourceTopic || topicAsRepartitionSourceTopic) util.Set.of(sourceTopic, topic) else util.Set.of(sourceTopic),
+ new StreamsRebalanceListener {
+ override def onTasksRevoked(tasks: util.Set[StreamsRebalanceData.TaskId]): Optional[Exception] =
+ Optional.empty()
+
+ override def onTasksAssigned(assignment: StreamsRebalanceData.Assignment): Optional[Exception] =
+ Optional.empty()
+
+ override def onAllTasksLost(): Optional[Exception] =
+ Optional.empty()
+ }
+ )
+ consumer.poll(Duration.ofMillis(500L))
+ removeAllClientAcls()
+ }
+
+ @ParameterizedTest
+ @CsvSource(Array(
+ "true, false, false, false",
+ "false, true, false, false",
+ "false, false, true, false",
+ "false, false, false, true"
+ ))
+ def testStreamsGroupDescribeWithGroupDescribeAndTopicDescribeAcl(
+ topicAsSourceTopic: Boolean,
+ topicAsRepartitionSinkTopic: Boolean,
+ topicAsRepartitionSourceTopic: Boolean,
+ topicAsStateChangelogTopics: Boolean
+ ): Unit = {
+ createStreamsGroupToDescribe(
+ topicAsSourceTopic,
+ topicAsRepartitionSinkTopic,
+ topicAsRepartitionSourceTopic,
+ topicAsStateChangelogTopics
+ )
+ addAndVerifyAcls(streamsGroupDescribeAcl(streamsGroupResource), streamsGroupResource)
+ addAndVerifyAcls(sourceTopicDescribeAcl(sourceTopicResource), sourceTopicResource) // Always added, since we need a source topic
+ addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource)
+
+ val request = streamsGroupDescribeRequest
+ val resource = Set[ResourceType](GROUP, TOPIC)
+ sendRequestAndVerifyResponseError(request, resource, isAuthorized = true)
+ }
+
+ @ParameterizedTest
+ @CsvSource(Array(
+ "true, false, false, false",
+ "false, true, false, false",
+ "false, false, true, false",
+ "false, false, false, true"
+ ))
+ def testStreamsGroupDescribeWithOperationAll(
+ topicAsSourceTopic: Boolean,
+ topicAsRepartitionSinkTopic: Boolean,
+ topicAsRepartitionSourceTopic: Boolean,
+ topicAsStateChangelogTopics: Boolean
+ ): Unit = {
+ createStreamsGroupToDescribe(
+ topicAsSourceTopic,
+ topicAsRepartitionSinkTopic,
+ topicAsRepartitionSourceTopic,
+ topicAsStateChangelogTopics
+ )
+
+ val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW)
+ addAndVerifyAcls(Set(allowAllOpsAcl), streamsGroupResource)
+ addAndVerifyAcls(sourceTopicDescribeAcl(sourceTopicResource), sourceTopicResource) // Always added, since we need a source topic
+ addAndVerifyAcls(Set(allowAllOpsAcl), topicResource)
+
+ val request = streamsGroupDescribeRequest
+ val resource = Set[ResourceType](GROUP, TOPIC)
+ sendRequestAndVerifyResponseError(request, resource, isAuthorized = true)
+ }
+
+ @ParameterizedTest
+ @CsvSource(Array(
+ "true, false, false, false",
+ "false, true, false, false",
+ "false, false, true, false",
+ "false, false, false, true"
+ ))
+ def testStreamsGroupDescribeWithoutGroupDescribeAcl(
+ topicAsSourceTopic: Boolean,
+ topicAsRepartitionSinkTopic: Boolean,
+ topicAsRepartitionSourceTopic: Boolean,
+ topicAsStateChangelogTopics: Boolean
+ ): Unit = {
+ createStreamsGroupToDescribe(
+ topicAsSourceTopic,
+ topicAsRepartitionSinkTopic,
+ topicAsRepartitionSourceTopic,
+ topicAsStateChangelogTopics
+ )
+ addAndVerifyAcls(sourceTopicDescribeAcl(sourceTopicResource), sourceTopicResource) // Always added, since we need a source topic
+ addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource)
+
+ val request = streamsGroupDescribeRequest
+ val resource = Set[ResourceType](GROUP, TOPIC)
+ sendRequestAndVerifyResponseError(request, resource, isAuthorized = false)
+ }
+
+ @ParameterizedTest
+ @CsvSource(Array(
+ "true, false, false, false",
+ "false, true, false, false",
+ "false, false, true, false",
+ "false, false, false, true"
+ ))
+ def testStreamsGroupDescribeWithoutGroupDescribeOrTopicDescribeAcl(
+ topicAsSourceTopic: Boolean,
+ topicAsRepartitionSinkTopic: Boolean,
+ topicAsRepartitionSourceTopic: Boolean,
+ topicAsStateChangelogTopics: Boolean
+ ): Unit = {
+ createStreamsGroupToDescribe(
+ topicAsSourceTopic,
+ topicAsRepartitionSinkTopic,
+ topicAsRepartitionSourceTopic,
+ topicAsStateChangelogTopics
+ )
+
+ val request = streamsGroupDescribeRequest
+ val resource = Set[ResourceType](GROUP, TOPIC)
+ addAndVerifyAcls(sourceTopicDescribeAcl(sourceTopicResource), sourceTopicResource) // Always added, since we need a source topic
+
+ sendRequestAndVerifyResponseError(request, resource, isAuthorized = false)
+ }
+
private def sendAndReceiveFirstRegexHeartbeat(memberId: String,
listenerName: ListenerName): ConsumerGroupHeartbeatResponseData = {
val request = new ConsumerGroupHeartbeatRequest.Builder(
@@ -3624,6 +4059,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
data = data
.setTopicPartitions(partitions.asJava)
.setSubscribedTopicRegex("^top.*")
+ .setRebalanceTimeoutMs(5 * 60 * 1000)
}
val request = new ConsumerGroupHeartbeatRequest.Builder(data).build()
val resource = Set[ResourceType](GROUP, TOPIC)
diff --git a/core/src/test/scala/integration/kafka/api/ClientOAuthIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/ClientOAuthIntegrationTest.scala
new file mode 100644
index 0000000000000..22ab6f2673c9e
--- /dev/null
+++ b/core/src/test/scala/integration/kafka/api/ClientOAuthIntegrationTest.scala
@@ -0,0 +1,261 @@
+/*
+ * 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 integration.kafka.api
+
+import com.nimbusds.jose.jwk.RSAKey
+import kafka.api.{IntegrationTestHarness, SaslSetup}
+import kafka.utils.TestInfoUtils
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.common.config.{ConfigException, SaslConfigs}
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled, TestInfo}
+
+import java.util.{Base64, Collections, Properties}
+import no.nav.security.mock.oauth2.{MockOAuth2Server, OAuth2Config}
+import no.nav.security.mock.oauth2.token.{KeyProvider, OAuth2TokenProvider}
+import org.apache.kafka.common.KafkaException
+import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
+import org.apache.kafka.common.security.auth.SecurityProtocol
+import org.apache.kafka.common.security.oauthbearer.{OAuthBearerLoginCallbackHandler, OAuthBearerLoginModule, OAuthBearerValidatorCallbackHandler}
+import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.test.TestUtils
+import org.junit.jupiter.api.Assertions.{assertDoesNotThrow, assertThrows}
+import org.junit.jupiter.params.ParameterizedTest
+import org.junit.jupiter.params.provider.MethodSource
+
+import java.io.File
+import java.nio.ByteBuffer
+import java.nio.channels.FileChannel
+import java.nio.file.StandardOpenOption
+import java.security.{KeyPairGenerator, PrivateKey}
+import java.security.interfaces.RSAPublicKey
+import java.util
+
+/**
+ * Integration tests for the consumer that cover basic usage as well as coordinator failure
+ */
+class ClientOAuthIntegrationTest extends IntegrationTestHarness with SaslSetup {
+
+ override val brokerCount = 3
+
+ override protected def securityProtocol = SecurityProtocol.SASL_PLAINTEXT
+ override protected val serverSaslProperties = Some(kafkaServerSaslProperties(kafkaServerSaslMechanisms, kafkaClientSaslMechanism))
+ override protected val clientSaslProperties = Some(kafkaClientSaslProperties(kafkaClientSaslMechanism))
+
+ protected def kafkaClientSaslMechanism = "OAUTHBEARER"
+ protected def kafkaServerSaslMechanisms = List(kafkaClientSaslMechanism)
+
+ val issuerId = "default"
+ var mockOAuthServer: MockOAuth2Server = _
+ var privateKey: PrivateKey = _
+
+ @BeforeEach
+ override def setUp(testInfo: TestInfo): Unit = {
+ // Step 1: Generate the key pair dynamically.
+ val keyGen = KeyPairGenerator.getInstance("RSA")
+ keyGen.initialize(2048)
+ val keyPair = keyGen.generateKeyPair()
+
+ privateKey = keyPair.getPrivate
+
+ // Step 2: Create the RSA JWK from key pair.
+ val rsaJWK = new RSAKey.Builder(keyPair.getPublic.asInstanceOf[RSAPublicKey])
+ .privateKey(privateKey)
+ .keyID("foo")
+ .build()
+
+ // Step 3: Create the OAuth server using the keys just created
+ val keyProvider = new KeyProvider(Collections.singletonList(rsaJWK))
+ val tokenProvider = new OAuth2TokenProvider(keyProvider)
+ val oauthConfig = new OAuth2Config(false, null, null, false, tokenProvider)
+ mockOAuthServer = new MockOAuth2Server(oauthConfig)
+
+ mockOAuthServer.start()
+ val tokenEndpointUrl = mockOAuthServer.tokenEndpointUrl(issuerId).url().toString
+ val jwksUrl = mockOAuthServer.jwksUrl(issuerId).url().toString
+ System.setProperty(BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, s"$tokenEndpointUrl,$jwksUrl")
+
+ val listenerNamePrefix = s"listener.name.${listenerName.value().toLowerCase}"
+
+ serverConfig.setProperty(s"$listenerNamePrefix.oauthbearer.${SaslConfigs.SASL_JAAS_CONFIG}", s"${classOf[OAuthBearerLoginModule].getName} required ;")
+ serverConfig.setProperty(s"$listenerNamePrefix.oauthbearer.${SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE}", issuerId)
+ serverConfig.setProperty(s"$listenerNamePrefix.oauthbearer.${SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL}", jwksUrl)
+ serverConfig.setProperty(s"$listenerNamePrefix.oauthbearer.${BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG}", classOf[OAuthBearerValidatorCallbackHandler].getName)
+
+ // create static config including client login context with credentials for JaasTestUtils 'client2'
+ startSasl(jaasSections(kafkaServerSaslMechanisms, Option(kafkaClientSaslMechanism)))
+
+ // The superuser needs the configuration in setUp because it's used to create resources before the individual
+ // test methods are invoked.
+ superuserClientConfig.putAll(defaultClientCredentialsConfigs())
+
+ super.setUp(testInfo)
+ }
+
+ @AfterEach
+ override def tearDown(): Unit = {
+ if (mockOAuthServer != null)
+ mockOAuthServer.shutdown()
+
+ closeSasl()
+ super.tearDown()
+
+ System.clearProperty(BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_FILES_CONFIG)
+ System.clearProperty(BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG)
+ }
+
+ def defaultOAuthConfigs(): Properties = {
+ val tokenEndpointUrl = mockOAuthServer.tokenEndpointUrl(issuerId).url().toString
+
+ val configs = new Properties()
+ configs.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, securityProtocol.name)
+ configs.put(SaslConfigs.SASL_JAAS_CONFIG, jaasClientLoginModule(kafkaClientSaslMechanism))
+ configs.put(SaslConfigs.SASL_LOGIN_CALLBACK_HANDLER_CLASS, classOf[OAuthBearerLoginCallbackHandler].getName)
+ configs.put(SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, tokenEndpointUrl)
+ configs
+ }
+
+ def defaultClientCredentialsConfigs(): Properties = {
+ val configs = defaultOAuthConfigs()
+ configs.put(SaslConfigs.SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_ID, "test-client")
+ configs.put(SaslConfigs.SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_SECRET, "test-secret")
+ configs
+ }
+
+ def defaultJwtBearerConfigs(): Properties = {
+ val configs = defaultOAuthConfigs()
+ configs.put(SaslConfigs.SASL_JAAS_CONFIG, jaasClientLoginModule(kafkaClientSaslMechanism))
+ configs.put(SaslConfigs.SASL_LOGIN_CALLBACK_HANDLER_CLASS, classOf[OAuthBearerLoginCallbackHandler].getName)
+ configs.put(SaslConfigs.SASL_OAUTHBEARER_JWT_RETRIEVER_CLASS, "org.apache.kafka.common.security.oauthbearer.JwtBearerJwtRetriever")
+ configs
+ }
+
+ @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames)
+ @MethodSource(Array("getTestGroupProtocolParametersAll"))
+ def testBasicClientCredentials(groupProtocol: String): Unit = {
+ val configs = defaultClientCredentialsConfigs()
+ assertDoesNotThrow(() => createProducer(configOverrides = configs))
+ assertDoesNotThrow(() => createConsumer(configOverrides = configs))
+ assertDoesNotThrow(() => createAdminClient(configOverrides = configs))
+ }
+
+ @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames)
+ @MethodSource(Array("getTestGroupProtocolParametersAll"))
+ def testBasicJwtBearer(groupProtocol: String): Unit = {
+ val jwt = mockOAuthServer.issueToken(issuerId, "jdoe", "someaudience", Collections.singletonMap("scope", "test"))
+ val assertionFile = TestUtils.tempFile(jwt.serialize())
+ System.setProperty(BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_FILES_CONFIG, assertionFile.getAbsolutePath)
+
+ val configs = defaultJwtBearerConfigs()
+ configs.put(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_FILE, assertionFile.getAbsolutePath)
+
+ assertDoesNotThrow(() => createProducer(configOverrides = configs))
+ assertDoesNotThrow(() => createConsumer(configOverrides = configs))
+ assertDoesNotThrow(() => createAdminClient(configOverrides = configs))
+ }
+
+ @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames)
+ @MethodSource(Array("getTestGroupProtocolParametersAll"))
+ def testBasicJwtBearer2(groupProtocol: String): Unit = {
+ val privateKeyFile = generatePrivateKeyFile()
+ System.setProperty(BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_FILES_CONFIG, privateKeyFile.getAbsolutePath)
+
+ val configs = defaultJwtBearerConfigs()
+ configs.put(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_FILE, privateKeyFile.getPath)
+ configs.put(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_AUD, "default")
+ configs.put(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_SUB, "kafka-client-test-sub")
+ configs.put(SaslConfigs.SASL_OAUTHBEARER_SCOPE, "default")
+ // configs.put(SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME, "aud")
+
+ assertDoesNotThrow(() => createProducer(configOverrides = configs))
+ assertDoesNotThrow(() => createConsumer(configOverrides = configs))
+ assertDoesNotThrow(() => createAdminClient(configOverrides = configs))
+ }
+
+ @Disabled("KAFKA-19394: Failure in ConsumerNetworkThread.initializeResources() can cause hangs on AsyncKafkaConsumer.close()")
+ @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames)
+ @MethodSource(Array("getTestGroupProtocolParametersAll"))
+ def testJwtBearerWithMalformedAssertionFile(groupProtocol: String): Unit = {
+ // Create the assertion file, but fill it with non-JWT garbage.
+ val assertionFile = TestUtils.tempFile("CQEN*)Q#F)&)^#QNC")
+ System.setProperty(BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_FILES_CONFIG, assertionFile.getAbsolutePath)
+
+ val configs = defaultJwtBearerConfigs()
+ configs.put(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_FILE, assertionFile.getAbsolutePath)
+
+ assertThrows(classOf[KafkaException], () => createProducer(configOverrides = configs))
+ assertThrows(classOf[KafkaException], () => createConsumer(configOverrides = configs))
+ assertThrows(classOf[KafkaException], () => createAdminClient(configOverrides = configs))
+ }
+
+ @Disabled("KAFKA-19394: Failure in ConsumerNetworkThread.initializeResources() can cause hangs on AsyncKafkaConsumer.close()")
+ @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames)
+ @MethodSource(Array("getTestGroupProtocolParametersAll"))
+ def testJwtBearerWithEmptyAssertionFile(groupProtocol: String): Unit = {
+ // Create the assertion file, but leave it empty.
+ val assertionFile = TestUtils.tempFile()
+ System.setProperty(BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_FILES_CONFIG, assertionFile.getAbsolutePath)
+
+ val configs = defaultJwtBearerConfigs()
+ configs.put(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_FILE, assertionFile.getAbsolutePath)
+
+ assertThrows(classOf[KafkaException], () => createProducer(configOverrides = configs))
+ assertThrows(classOf[KafkaException], () => createConsumer(configOverrides = configs))
+ assertThrows(classOf[KafkaException], () => createAdminClient(configOverrides = configs))
+ }
+
+ @Disabled("KAFKA-19394: Failure in ConsumerNetworkThread.initializeResources() can cause hangs on AsyncKafkaConsumer.close()")
+ @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames)
+ @MethodSource(Array("getTestGroupProtocolParametersAll"))
+ def testJwtBearerWithMissingAssertionFile(groupProtocol: String): Unit = {
+ val missingFileName = "/this/does/not/exist.txt"
+
+ val configs = defaultJwtBearerConfigs()
+ configs.put(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_FILE, missingFileName)
+
+ assertThrows(classOf[KafkaException], () => createProducer(configOverrides = configs))
+ assertThrows(classOf[KafkaException], () => createConsumer(configOverrides = configs))
+ assertThrows(classOf[KafkaException], () => createAdminClient(configOverrides = configs))
+ }
+
+ @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames)
+ @MethodSource(Array("getTestGroupProtocolParametersAll"))
+ def testUnsupportedJwtRetriever(groupProtocol: String): Unit = {
+ val className = "org.apache.kafka.common.security.oauthbearer.ThisIsNotARealJwtRetriever"
+
+ val configs = defaultOAuthConfigs()
+ configs.put(SaslConfigs.SASL_OAUTHBEARER_JWT_RETRIEVER_CLASS, className)
+
+ assertThrows(classOf[ConfigException], () => createProducer(configOverrides = configs))
+ assertThrows(classOf[ConfigException], () => createConsumer(configOverrides = configs))
+ assertThrows(classOf[ConfigException], () => createAdminClient(configOverrides = configs))
+ }
+
+ def generatePrivateKeyFile(): File = {
+ val file = File.createTempFile("private-", ".key")
+ val bytes = Base64.getEncoder.encode(privateKey.getEncoded)
+ var channel: FileChannel = null
+
+ try {
+ channel = FileChannel.open(file.toPath, util.EnumSet.of(StandardOpenOption.WRITE))
+ Utils.writeFully(channel, ByteBuffer.wrap(bytes))
+ } finally {
+ channel.close()
+ }
+
+ file
+ }
+}
\ No newline at end of file
diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
index e062dcc09fa08..7c08dd9c3fe7d 100644
--- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
+++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
@@ -22,14 +22,16 @@ import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, KafkaConsume
import kafka.utils.TestUtils
import kafka.utils.Implicits._
-import java.util.Properties
+import java.util.{Optional, Properties}
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig}
import kafka.server.KafkaConfig
import kafka.integration.KafkaServerTestHarness
import kafka.security.JaasTestUtils
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig}
+import org.apache.kafka.clients.consumer.internals.{AsyncKafkaConsumer, StreamsRebalanceData}
import org.apache.kafka.common.network.{ConnectionMode, ListenerName}
import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySerializer, Deserializer, Serializer}
+import org.apache.kafka.common.utils.Utils
import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.raft.MetadataLogConfig
import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs}
@@ -49,6 +51,7 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness {
val producerConfig = new Properties
val consumerConfig = new Properties
val shareConsumerConfig = new Properties
+ val streamsConsumerConfig = new Properties
val adminClientConfig = new Properties
val superuserClientConfig = new Properties
val serverConfig = new Properties
@@ -56,6 +59,7 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness {
private val consumers = mutable.Buffer[Consumer[_, _]]()
private val shareConsumers = mutable.Buffer[ShareConsumer[_, _]]()
+ private val streamsConsumers = mutable.Buffer[Consumer[_, _]]()
private val producers = mutable.Buffer[KafkaProducer[_, _]]()
private val adminClients = mutable.Buffer[Admin]()
@@ -148,7 +152,12 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness {
shareConsumerConfig.putIfAbsent(ConsumerConfig.GROUP_ID_CONFIG, "group")
shareConsumerConfig.putIfAbsent(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, classOf[ByteArrayDeserializer].getName)
shareConsumerConfig.putIfAbsent(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, classOf[ByteArrayDeserializer].getName)
-
+
+ streamsConsumerConfig.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers())
+ streamsConsumerConfig.putIfAbsent(ConsumerConfig.GROUP_ID_CONFIG, "group")
+ streamsConsumerConfig.putIfAbsent(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, classOf[ByteArrayDeserializer].getName)
+ streamsConsumerConfig.putIfAbsent(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, classOf[ByteArrayDeserializer].getName)
+
adminClientConfig.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers())
doSuperuserSetup(testInfo)
@@ -207,6 +216,25 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness {
shareConsumer
}
+ def createStreamsConsumer[K, V](keyDeserializer: Deserializer[K] = new ByteArrayDeserializer,
+ valueDeserializer: Deserializer[V] = new ByteArrayDeserializer,
+ configOverrides: Properties = new Properties,
+ configsToRemove: List[String] = List(),
+ streamsRebalanceData: StreamsRebalanceData): AsyncKafkaConsumer[K, V] = {
+ val props = new Properties
+ props ++= streamsConsumerConfig
+ props ++= configOverrides
+ configsToRemove.foreach(props.remove(_))
+ val streamsConsumer = new AsyncKafkaConsumer[K, V](
+ new ConsumerConfig(ConsumerConfig.appendDeserializerToConfig(Utils.propsToMap(props), keyDeserializer, valueDeserializer)),
+ keyDeserializer,
+ valueDeserializer,
+ Optional.of(streamsRebalanceData)
+ )
+ streamsConsumers += streamsConsumer
+ streamsConsumer
+ }
+
def createAdminClient(
listenerName: ListenerName = listenerName,
configOverrides: Properties = new Properties
@@ -239,11 +267,14 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness {
consumers.foreach(_.close(Duration.ZERO))
shareConsumers.foreach(_.wakeup())
shareConsumers.foreach(_.close(Duration.ZERO))
+ streamsConsumers.foreach(_.wakeup())
+ streamsConsumers.foreach(_.close(Duration.ZERO))
adminClients.foreach(_.close(Duration.ZERO))
producers.clear()
consumers.clear()
shareConsumers.clear()
+ streamsConsumers.clear()
adminClients.clear()
} finally {
super.tearDown()
diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
index c7a45fc8c80d7..59eba1eb18666 100644
--- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
@@ -52,6 +52,7 @@ import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEX
import org.apache.kafka.coordinator.group.{GroupConfig, GroupCoordinatorConfig}
import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.security.authorizer.AclEntry
+import org.apache.kafka.server.common.{EligibleLeaderReplicasVersion, MetadataVersion}
import org.apache.kafka.server.config.{QuotaConfig, ServerConfigs, ServerLogConfigs}
import org.apache.kafka.server.logger.LoggingController
import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig, LogFileUtils}
@@ -60,7 +61,7 @@ import org.apache.logging.log4j.core.config.Configurator
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{BeforeEach, Test, TestInfo, Timeout}
import org.junit.jupiter.params.ParameterizedTest
-import org.junit.jupiter.params.provider.{MethodSource}
+import org.junit.jupiter.params.provider.MethodSource
import org.slf4j.LoggerFactory
import java.util.AbstractMap.SimpleImmutableEntry
@@ -3002,6 +3003,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
def testElectUncleanLeadersForOnePartition(): Unit = {
// Case: unclean leader election with one topic partition
client = createAdminClient
+ disableEligibleLeaderReplicas(client)
val broker1 = 1
val broker2 = 2
@@ -3029,6 +3031,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
def testElectUncleanLeadersForManyPartitions(): Unit = {
// Case: unclean leader election with many topic partitions
client = createAdminClient
+ disableEligibleLeaderReplicas(client)
val broker1 = 1
val broker2 = 2
@@ -3068,6 +3071,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
def testElectUncleanLeadersForAllPartitions(): Unit = {
// Case: noop unclean leader election and valid unclean leader election for all partitions
client = createAdminClient
+ disableEligibleLeaderReplicas(client)
val broker1 = 1
val broker2 = 2
@@ -3107,6 +3111,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
def testElectUncleanLeadersForUnknownPartitions(): Unit = {
// Case: unclean leader election for unknown topic
client = createAdminClient
+ disableEligibleLeaderReplicas(client)
val broker1 = 1
val broker2 = 2
@@ -3132,6 +3137,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
def testElectUncleanLeadersWhenNoLiveBrokers(): Unit = {
// Case: unclean leader election with no live brokers
client = createAdminClient
+ disableEligibleLeaderReplicas(client)
val broker1 = 1
val broker2 = 2
@@ -3160,6 +3166,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
def testElectUncleanLeadersNoop(): Unit = {
// Case: noop unclean leader election with explicit topic partitions
client = createAdminClient
+ disableEligibleLeaderReplicas(client)
val broker1 = 1
val broker2 = 2
@@ -3187,6 +3194,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
def testElectUncleanLeadersAndNoop(): Unit = {
// Case: one noop unclean leader election and one valid unclean leader election
client = createAdminClient
+ disableEligibleLeaderReplicas(client)
val broker1 = 1
val broker2 = 2
@@ -3878,6 +3886,14 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
testAppendConfig(props, "0:0", "1:1,0:0")
}
+ private def disableEligibleLeaderReplicas(admin: Admin): Unit = {
+ if (metadataVersion.isAtLeast(MetadataVersion.IBP_4_1_IV0)) {
+ admin.updateFeatures(
+ util.Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, new FeatureUpdate(0, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE)),
+ new UpdateFeaturesOptions()).all().get()
+ }
+ }
+
private def testAppendConfig(props: Properties, append: String, expected: String): Unit = {
client = createAdminClient
createTopic(topic, topicConfig = props)
diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala b/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala
deleted file mode 100644
index e93c8c99e6302..0000000000000
--- a/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala
+++ /dev/null
@@ -1,158 +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.api
-
-import kafka.utils.TestUtils
-import org.apache.kafka.clients.admin.{Admin, NewPartitionReassignment, TopicDescription}
-import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord, RecordMetadata}
-import org.apache.kafka.common.TopicPartition
-import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
-import org.junit.jupiter.api.Assertions.{assertEquals, assertNotEquals}
-import org.junit.jupiter.api.Test
-
-import java.nio.charset.StandardCharsets
-import java.util
-import java.util.Optional
-
-class ProducerSendWhileDeletionTest extends IntegrationTestHarness {
- val producerCount: Int = 1
- val brokerCount: Int = 2
- val defaultLingerMs: Int = 5
-
- serverConfig.put(ServerLogConfigs.NUM_PARTITIONS_CONFIG, 2.toString)
- serverConfig.put(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, 2.toString)
- serverConfig.put(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG, false.toString)
-
- producerConfig.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, 5000L.toString)
- producerConfig.put(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG, 10000.toString)
- producerConfig.put(ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG, (10000 + defaultLingerMs).toString)
-
- /**
- * Tests that Producer gets self-recovered when a topic is deleted mid-way of produce.
- *
- * Producer will attempt to send messages to the partition specified in each record, and should
- * succeed as long as the partition is included in the metadata.
- */
- @Test
- def testSendWithTopicDeletionMidWay(): Unit = {
- val numRecords = 10
- val topic = "topic"
-
- // Create topic with leader as 0 for the 2 partitions.
- createTopicWithAssignment(topic, Map(0 -> Seq(0, 1), 1 -> Seq(0, 1)))
-
- val reassignment = util.Map.of(
- new TopicPartition(topic, 0), Optional.of(new NewPartitionReassignment(util.List.of(1, 0))),
- new TopicPartition(topic, 1), Optional.of(new NewPartitionReassignment(util.List.of(1, 0)))
- )
-
- // Change leader to 1 for both the partitions to increase leader epoch from 0 -> 1
- val admin = createAdminClient()
- admin.alterPartitionReassignments(reassignment).all().get()
-
- val producer = createProducer()
-
- (1 to numRecords).foreach { i =>
- val resp = producer.send(new ProducerRecord(topic, null, ("value" + i).getBytes(StandardCharsets.UTF_8))).get
- assertEquals(topic, resp.topic())
- }
-
- // Start topic deletion
- deleteTopic(topic, listenerName)
-
- // Verify that the topic is deleted when no metadata request comes in
- TestUtils.verifyTopicDeletion(topic, 2, brokers)
-
- // Producer should be able to send messages even after topic gets deleted and auto-created
- assertEquals(topic, producer.send(new ProducerRecord(topic, null, "value".getBytes(StandardCharsets.UTF_8))).get.topic())
- }
-
- /**
- * Tests that Producer produce to new topic id after recreation.
- *
- * Producer will attempt to send messages to the partition specified in each record, and should
- * succeed as long as the metadata has been updated with new topic id.
- */
- @Test
- def testSendWithRecreatedTopic(): Unit = {
- val numRecords = 10
- val topic = "topic"
- createTopic(topic)
- val admin = createAdminClient()
- val topicId = topicMetadata(admin, topic).topicId()
- val producer = createProducer()
-
- (1 to numRecords).foreach { i =>
- val resp = producer.send(new ProducerRecord(topic, null, ("value" + i).getBytes(StandardCharsets.UTF_8))).get
- assertEquals(topic, resp.topic())
- }
- // Start topic deletion
- deleteTopic(topic, listenerName)
-
- // Verify that the topic is deleted when no metadata request comes in
- TestUtils.verifyTopicDeletion(topic, 2, brokers)
- createTopic(topic)
- assertNotEquals(topicId, topicMetadata(admin, topic).topicId())
-
- // Producer should be able to send messages even after topic gets recreated
- val recordMetadata: RecordMetadata = producer.send(new ProducerRecord(topic, null, "value".getBytes(StandardCharsets.UTF_8))).get
- assertEquals(topic, recordMetadata.topic())
- assertEquals(0, recordMetadata.offset())
- }
-
- /**
- * Tests that Producer produce to topic during reassignment where topic metadata change on broker side.
- *
- * Producer will attempt to send messages to the partition specified in each record, and should
- * succeed as long as the metadata cache on the leader includes the partition topic id.
- */
- @Test
- def testSendWithTopicReassignmentIsMidWay(): Unit = {
- val numRecords = 10
- val topic = "topic"
- val partition0: TopicPartition = new TopicPartition(topic, 0)
- val admin: Admin = createAdminClient()
-
- // Create topic with leader as 0 for the 1 partition.
- createTopicWithAssignment(topic, Map(0 -> Seq(0)))
- TestUtils.assertLeader(admin, partition0, 0)
-
- val topicDetails = topicMetadata(admin, topic)
- val producer = createProducer()
-
- (1 to numRecords).foreach { i =>
- val resp = producer.send(new ProducerRecord(topic, null, ("value" + i).getBytes(StandardCharsets.UTF_8))).get
- assertEquals(topic, resp.topic())
- }
-
- val reassignment = util.Map.of(
- partition0, Optional.of(new NewPartitionReassignment(util.List.of(1))),
- )
-
- // Change replica assignment from 0 to 1. Leadership moves to 1.
- admin.alterPartitionReassignments(reassignment).all().get()
- TestUtils.assertLeader(admin, partition0, 1)
- assertEquals(topicDetails.topicId(), topicMetadata(admin, topic).topicId())
-
- // Producer should be able to send messages even after topic gets reassigned
- assertEquals(topic, producer.send(new ProducerRecord(topic, null, "value".getBytes(StandardCharsets.UTF_8))).get.topic())
- }
-
- private def topicMetadata(admin: Admin, topic: String): TopicDescription = {
- admin.describeTopics(util.List.of(topic)).allTopicNames().get().get(topic)
- }
-}
diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala
index 5bbe7e54d5503..f12286476033a 100644
--- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala
+++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala
@@ -1099,9 +1099,13 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames)
@MethodSource(Array("getTestGroupProtocolParametersAll"))
def testServersCanStartWithInvalidStaticConfigsAndValidDynamicConfigs(groupProtocol: String): Unit = {
+ TestNumReplicaFetcherMetricsReporter.testReporters.clear()
+
// modify snapshot interval config to explicitly take snapshot on a broker with valid dynamic configs
val props = defaultStaticConfig(numServers)
props.put(MetadataLogConfig.METADATA_SNAPSHOT_MAX_INTERVAL_MS_CONFIG, "10000")
+ props.put(MetricConfigs.METRIC_REPORTER_CLASSES_CONFIG, classOf[TestNumReplicaFetcherMetricsReporter].getName)
+ props.put(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, "1")
val kafkaConfig = KafkaConfig.fromProps(props)
val newBroker = createBroker(kafkaConfig).asInstanceOf[BrokerServer]
@@ -1109,6 +1113,15 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
alterSslKeystoreUsingConfigCommand(sslProperties1, listenerPrefix(SecureExternal))
+ // Add num.replica.fetchers to the cluster-level config.
+ val clusterLevelProps = new Properties
+ clusterLevelProps.put(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, "2")
+ reconfigureServers(clusterLevelProps, perBrokerConfig = false, (ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, "2"))
+
+ // Wait for the metrics reporter to be configured
+ val initialReporter = TestNumReplicaFetcherMetricsReporter.waitForReporters(1).head
+ initialReporter.verifyState(reconfigureCount = 1, numFetcher = 2)
+
TestUtils.ensureConsistentKRaftMetadata(servers, controllerServer)
TestUtils.waitUntilTrue(
@@ -1121,11 +1134,19 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
newBroker.shutdown()
newBroker.awaitShutdown()
+ // Clean up the test reporter
+ TestNumReplicaFetcherMetricsReporter.testReporters.clear()
+
val invalidStaticConfigs = defaultStaticConfig(newBroker.config.brokerId)
invalidStaticConfigs.putAll(securityProps(invalidSslConfigs, KEYSTORE_PROPS, listenerPrefix(SecureExternal)))
newBroker.config.updateCurrentConfig(KafkaConfig.fromProps(invalidStaticConfigs))
newBroker.startup()
+
+ // Verify that the custom MetricsReporter is not reconfigured after restart.
+ // If readDynamicBrokerConfigsFromSnapshot works correctly, the reporter should maintain its state.
+ val reporterAfterRestart = TestNumReplicaFetcherMetricsReporter.waitForReporters(1).head
+ reporterAfterRestart.verifyState(reconfigureCount = 0, numFetcher = 2)
}
private def awaitInitialPositions(consumer: Consumer[_, _]): Unit = {
@@ -1634,6 +1655,64 @@ class TestMetricsReporter extends MetricsReporter with Reconfigurable with Close
}
}
+object TestNumReplicaFetcherMetricsReporter {
+ val testReporters = new ConcurrentLinkedQueue[TestNumReplicaFetcherMetricsReporter]()
+
+ def waitForReporters(count: Int): List[TestNumReplicaFetcherMetricsReporter] = {
+ TestUtils.waitUntilTrue(() => testReporters.size == count, msg = "Metrics reporters size not matched. Expected: " + count + ", actual: " + testReporters.size())
+
+ val reporters = testReporters.asScala.toList
+ TestUtils.waitUntilTrue(() => reporters.forall(_.configureCount == 1), msg = "Metrics reporters not configured")
+ reporters
+ }
+}
+
+
+class TestNumReplicaFetcherMetricsReporter extends MetricsReporter {
+ import TestNumReplicaFetcherMetricsReporter._
+ @volatile var configureCount = 0
+ @volatile var reconfigureCount = 0
+ @volatile var numFetchers: Int = 1
+ testReporters.add(this)
+
+ override def init(metrics: util.List[KafkaMetric]): Unit = {
+ }
+
+ override def configure(configs: util.Map[String, _]): Unit = {
+ configureCount += 1
+ numFetchers = configs.get(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG).toString.toInt
+ }
+
+ override def metricChange(metric: KafkaMetric): Unit = {
+ }
+
+ override def metricRemoval(metric: KafkaMetric): Unit = {
+ }
+
+ override def reconfigurableConfigs(): util.Set[String] = {
+ util.Set.of(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG)
+ }
+
+ override def validateReconfiguration(configs: util.Map[String, _]): Unit = {
+ val numFetchers = configs.get(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG).toString.toInt
+ if (numFetchers <= 0)
+ throw new ConfigException(s"Invalid num.replica.fetchers $numFetchers")
+ }
+
+ override def reconfigure(configs: util.Map[String, _]): Unit = {
+ reconfigureCount += 1
+ numFetchers = configs.get(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG).toString.toInt
+ }
+
+ override def close(): Unit = {
+ }
+
+ def verifyState(reconfigureCount: Int, numFetcher: Int = 1): Unit = {
+ assertEquals(reconfigureCount, this.reconfigureCount)
+ assertEquals(numFetcher, this.numFetchers)
+ }
+}
+
class MockFileConfigProvider extends FileConfigProvider {
@throws(classOf[IOException])
diff --git a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala
index 6c491e739e3e7..3d5837b92d0d7 100755
--- a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala
+++ b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala
@@ -40,7 +40,7 @@ import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.queue.KafkaEventQueue
import org.apache.kafka.raft.{MetadataLogConfig, QuorumConfig}
import org.apache.kafka.server.{ClientMetricsManager, ServerSocketFactory}
-import org.apache.kafka.server.common.{EligibleLeaderReplicasVersion, MetadataVersion, TransactionVersion}
+import org.apache.kafka.server.common.{MetadataVersion, TransactionVersion}
import org.apache.kafka.server.config.{KRaftConfigs, ServerConfigs, ServerLogConfigs}
import org.apache.kafka.server.fault.{FaultHandler, MockFaultHandler}
import org.apache.kafka.server.util.timer.SystemTimer
@@ -284,12 +284,6 @@ abstract class QuorumTestHarness extends Logging {
} else TransactionVersion.TV_1.featureLevel()
formatter.setFeatureLevel(TransactionVersion.FEATURE_NAME, transactionVersion)
- val elrVersion =
- if (TestInfoUtils.isEligibleLeaderReplicasV1Enabled(testInfo)) {
- EligibleLeaderReplicasVersion.ELRV_1.featureLevel()
- } else EligibleLeaderReplicasVersion.ELRV_0.featureLevel()
- formatter.setFeatureLevel(EligibleLeaderReplicasVersion.FEATURE_NAME, elrVersion)
-
addFormatterSettings(formatter)
formatter.run()
val bootstrapMetadata = formatter.bootstrapMetadata()
diff --git a/core/src/test/scala/kafka/utils/TestInfoUtils.scala b/core/src/test/scala/kafka/utils/TestInfoUtils.scala
index 5b6a2239c930b..e6c70b6e8fe49 100644
--- a/core/src/test/scala/kafka/utils/TestInfoUtils.scala
+++ b/core/src/test/scala/kafka/utils/TestInfoUtils.scala
@@ -50,12 +50,4 @@ object TestInfoUtils {
def isTransactionV2Enabled(testInfo: TestInfo): Boolean = {
!testInfo.getDisplayName.contains("isTV2Enabled=false")
}
-
- /**
- * Returns whether eligible leader replicas version 1 is enabled.
- * When no parameter is provided, the default returned is false.
- */
- def isEligibleLeaderReplicasV1Enabled(testInfo: TestInfo): Boolean = {
- testInfo.getDisplayName.contains("isELRV1Enabled=true")
- }
}
diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala
index eea5db86bc628..26675fca74793 100644
--- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala
@@ -33,6 +33,7 @@ import org.junit.jupiter.params.provider.{CsvSource, ValueSource}
import org.mockito.ArgumentMatchers.{any, anyBoolean, anyInt}
import org.mockito.Mockito._
import org.mockito.{ArgumentCaptor, ArgumentMatchers}
+import org.mockito.Mockito.doAnswer
import scala.collection.mutable
import scala.jdk.CollectionConverters._
@@ -1267,6 +1268,142 @@ class TransactionCoordinatorTest {
any())
}
+ @Test
+ def shouldNotCauseEpochOverflowWhenInitPidDuringOngoingTxnV2(): Unit = {
+ // When InitProducerId is called with an ongoing transaction at epoch 32766 (Short.MaxValue - 1),
+ // it should not cause an epoch overflow by incrementing twice.
+ // The only true increment happens in prepareAbortOrCommit
+ val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID,
+ (Short.MaxValue - 1).toShort, (Short.MaxValue - 2).toShort, txnTimeoutMs, TransactionState.ONGOING, partitions, time.milliseconds(), time.milliseconds(), TV_2)
+
+ when(transactionManager.validateTransactionTimeoutMs(anyBoolean(), anyInt()))
+ .thenReturn(true)
+ when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId)))
+ .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata))))
+ when(transactionManager.transactionVersionLevel()).thenReturn(TV_2)
+
+ // Capture the transition metadata to verify epoch increments
+ val capturedTxnTransitMetadata: ArgumentCaptor[TxnTransitMetadata] = ArgumentCaptor.forClass(classOf[TxnTransitMetadata])
+ when(transactionManager.appendTransactionToLog(
+ ArgumentMatchers.eq(transactionalId),
+ ArgumentMatchers.eq(coordinatorEpoch),
+ capturedTxnTransitMetadata.capture(),
+ capturedErrorsCallback.capture(),
+ any(),
+ any())
+ ).thenAnswer(invocation => {
+ val transitMetadata = invocation.getArgument[TxnTransitMetadata](2)
+ // Simulate the metadata update that would happen in the real appendTransactionToLog
+ txnMetadata.completeTransitionTo(transitMetadata)
+ capturedErrorsCallback.getValue.apply(Errors.NONE)
+ })
+
+ // Handle InitProducerId with ongoing transaction at epoch 32766
+ coordinator.handleInitProducerId(
+ transactionalId,
+ txnTimeoutMs,
+ enableTwoPCFlag = false,
+ keepPreparedTxn = false,
+ None,
+ initProducerIdMockCallback
+ )
+
+ // Verify that the epoch did not overflow (should be Short.MaxValue = 32767, not negative)
+ assertEquals(Short.MaxValue, txnMetadata.producerEpoch)
+ assertEquals(TransactionState.PREPARE_ABORT, txnMetadata.state)
+
+ verify(transactionManager).validateTransactionTimeoutMs(anyBoolean(), anyInt())
+ verify(transactionManager, times(3)).getTransactionState(ArgumentMatchers.eq(transactionalId))
+ verify(transactionManager).appendTransactionToLog(
+ ArgumentMatchers.eq(transactionalId),
+ ArgumentMatchers.eq(coordinatorEpoch),
+ any[TxnTransitMetadata],
+ any(),
+ any(),
+ any())
+ }
+
+ @Test
+ def shouldHandleTimeoutAtEpochOverflowBoundaryCorrectlyTV2(): Unit = {
+ // Test the scenario where we have an ongoing transaction at epoch 32766 (Short.MaxValue - 1)
+ // and the producer crashes/times out. This test verifies that the timeout handling
+ // correctly manages the epoch overflow scenario without causing failures.
+
+ val epochAtMaxBoundary = (Short.MaxValue - 1).toShort // 32766
+ val now = time.milliseconds()
+
+ // Create transaction metadata at the epoch boundary that would cause overflow IFF double-incremented
+ val txnMetadata = new TransactionMetadata(
+ transactionalId = transactionalId,
+ producerId = producerId,
+ prevProducerId = RecordBatch.NO_PRODUCER_ID,
+ nextProducerId = RecordBatch.NO_PRODUCER_ID,
+ producerEpoch = epochAtMaxBoundary,
+ lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH,
+ txnTimeoutMs = txnTimeoutMs,
+ state = TransactionState.ONGOING,
+ topicPartitions = partitions,
+ txnStartTimestamp = now,
+ txnLastUpdateTimestamp = now,
+ clientTransactionVersion = TV_2
+ )
+ assertTrue(txnMetadata.isProducerEpochExhausted)
+
+ // Mock the transaction manager to return our test transaction as timed out
+ when(transactionManager.timedOutTransactions())
+ .thenReturn(List(TransactionalIdAndProducerIdEpoch(transactionalId, producerId, epochAtMaxBoundary)))
+ when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId)))
+ .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata))))
+ when(transactionManager.transactionVersionLevel()).thenReturn(TV_2)
+
+ // Mock the append operation to simulate successful write and update the metadata
+ when(transactionManager.appendTransactionToLog(
+ ArgumentMatchers.eq(transactionalId),
+ ArgumentMatchers.eq(coordinatorEpoch),
+ any[TxnTransitMetadata],
+ capturedErrorsCallback.capture(),
+ any(),
+ any())
+ ).thenAnswer(invocation => {
+ val transitMetadata = invocation.getArgument[TxnTransitMetadata](2)
+ // Simulate the metadata update that would happen in the real appendTransactionToLog
+ txnMetadata.completeTransitionTo(transitMetadata)
+ capturedErrorsCallback.getValue.apply(Errors.NONE)
+ })
+
+ // Track the actual behavior
+ var callbackInvoked = false
+ var resultError: Errors = null
+ var resultProducerId: Long = -1
+ var resultEpoch: Short = -1
+
+ def checkOnEndTransactionComplete(txnIdAndPidEpoch: TransactionalIdAndProducerIdEpoch)
+ (error: Errors, newProducerId: Long, newProducerEpoch: Short): Unit = {
+ callbackInvoked = true
+ resultError = error
+ resultProducerId = newProducerId
+ resultEpoch = newProducerEpoch
+ }
+
+ // Execute the timeout abort process
+ coordinator.abortTimedOutTransactions(checkOnEndTransactionComplete)
+
+ assertTrue(callbackInvoked, "Callback should have been invoked")
+ assertEquals(Errors.NONE, resultError, "Expected no errors in the callback")
+ assertEquals(producerId, resultProducerId, "Expected producer ID to match")
+ assertEquals(Short.MaxValue, resultEpoch, "Expected producer epoch to be Short.MaxValue (32767) single epoch bump")
+
+ // Verify the transaction metadata was correctly updated to the final epoch
+ assertEquals(Short.MaxValue, txnMetadata.producerEpoch,
+ s"Expected transaction metadata producer epoch to be ${Short.MaxValue} " +
+ s"after timeout handling, but was ${txnMetadata.producerEpoch}"
+ )
+
+ // Verify the basic flow was attempted
+ verify(transactionManager).timedOutTransactions()
+ verify(transactionManager, atLeast(1)).getTransactionState(ArgumentMatchers.eq(transactionalId))
+ }
+
@Test
def testInitProducerIdWithNoLastProducerData(): Unit = {
// If the metadata doesn't include the previous producer data (for example, if it was written to the log by a broker
@@ -1842,4 +1979,153 @@ class TransactionCoordinatorTest {
else
producerEpoch
}
+
+ @Test
+ def testTV2AllowsEpochReBumpingAfterFailedWrite(): Unit = {
+ // Test the complete TV2 flow: failed write → epoch fence → abort → retry with epoch bump
+ // This demonstrates that TV2 allows epoch re-bumping after failed writes (unlike TV1)
+ val producerEpoch = 1.toShort
+ val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID,
+ producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, TransactionState.ONGOING, partitions, time.milliseconds(), time.milliseconds(), TV_2)
+
+ when(transactionManager.validateTransactionTimeoutMs(anyBoolean(), anyInt()))
+ .thenReturn(true)
+ when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId)))
+ .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata))))
+ when(transactionManager.transactionVersionLevel()).thenReturn(TV_2)
+
+ // First attempt fails with COORDINATOR_NOT_AVAILABLE
+ when(transactionManager.appendTransactionToLog(
+ ArgumentMatchers.eq(transactionalId),
+ ArgumentMatchers.eq(coordinatorEpoch),
+ any(),
+ any(),
+ any(),
+ any()
+ )).thenAnswer(invocation => {
+ val callback = invocation.getArgument[Errors => Unit](3)
+
+ // Simulate the real TransactionStateManager behavior: reset pendingState on failure
+ // since handleInitProducerId doesn't provide a custom retryOnError function
+ txnMetadata.pendingState = None
+
+ // For TV2, hasFailedEpochFence is NOT set to true, allowing epoch bumps on retry
+ // The epoch remains at its original value (1) since completeTransitionTo was never called
+
+ callback.apply(Errors.COORDINATOR_NOT_AVAILABLE)
+ })
+
+ coordinator.handleInitProducerId(
+ transactionalId,
+ txnTimeoutMs,
+ enableTwoPCFlag = false,
+ keepPreparedTxn = false,
+ None,
+ initProducerIdMockCallback
+ )
+ assertEquals(InitProducerIdResult(-1, -1, Errors.COORDINATOR_NOT_AVAILABLE), result)
+
+ // After the first failed attempt, the state should be:
+ // - hasFailedEpochFence = false (NOT set for TV2)
+ // - pendingState = None (reset by TransactionStateManager)
+ // - producerEpoch = 1 (unchanged since completeTransitionTo was never called)
+ // - transaction still ONGOING
+
+ // Second attempt: Should abort the ongoing transaction
+ reset(transactionManager)
+ when(transactionManager.validateTransactionTimeoutMs(anyBoolean(), anyInt()))
+ .thenReturn(true)
+ when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId)))
+ .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata))))
+ when(transactionManager.transactionVersionLevel()).thenReturn(TV_2)
+
+ // Mock the appendTransactionToLog to succeed for the endTransaction call
+ when(transactionManager.appendTransactionToLog(
+ ArgumentMatchers.eq(transactionalId),
+ ArgumentMatchers.eq(coordinatorEpoch),
+ any(),
+ any(),
+ any(),
+ any()
+ )).thenAnswer(invocation => {
+ val newMetadata = invocation.getArgument[TxnTransitMetadata](2)
+ val callback = invocation.getArgument[Errors => Unit](3)
+
+ // Complete the transition and call the callback with success
+ txnMetadata.completeTransitionTo(newMetadata)
+ callback.apply(Errors.NONE)
+ })
+
+ // Mock the transactionMarkerChannelManager to simulate the second write (PREPARE_ABORT -> COMPLETE_ABORT)
+ doAnswer(invocation => {
+ val newMetadata = invocation.getArgument[TxnTransitMetadata](3)
+ // Simulate the completion of transaction markers and the second write
+ // This would normally happen asynchronously after markers are sent
+ txnMetadata.completeTransitionTo(newMetadata) // This transitions to COMPLETE_ABORT
+ txnMetadata.pendingState = None
+
+ null
+ }).when(transactionMarkerChannelManager).addTxnMarkersToSend(
+ ArgumentMatchers.eq(coordinatorEpoch),
+ ArgumentMatchers.eq(TransactionResult.ABORT),
+ ArgumentMatchers.eq(txnMetadata),
+ any()
+ )
+
+ coordinator.handleInitProducerId(
+ transactionalId,
+ txnTimeoutMs,
+ enableTwoPCFlag = false,
+ keepPreparedTxn = false,
+ None,
+ initProducerIdMockCallback
+ )
+
+ // The second attempt should return CONCURRENT_TRANSACTIONS (this is intentional)
+ assertEquals(InitProducerIdResult(-1, -1, Errors.CONCURRENT_TRANSACTIONS), result)
+
+ // The transactionMarkerChannelManager mock should have completed the transition to COMPLETE_ABORT
+ // Verify that hasFailedEpochFence was never set to true for TV2, allowing future epoch bumps
+ assertFalse(txnMetadata.hasFailedEpochFence)
+
+ // Third attempt: Client retries after CONCURRENT_TRANSACTIONS
+ reset(transactionManager)
+ when(transactionManager.validateTransactionTimeoutMs(anyBoolean(), anyInt()))
+ .thenReturn(true)
+ when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId)))
+ .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata))))
+ when(transactionManager.transactionVersionLevel()).thenReturn(TV_2)
+
+ when(transactionManager.appendTransactionToLog(
+ ArgumentMatchers.eq(transactionalId),
+ ArgumentMatchers.eq(coordinatorEpoch),
+ any(),
+ any(),
+ any(),
+ any()
+ )).thenAnswer(invocation => {
+ val newMetadata = invocation.getArgument[TxnTransitMetadata](2)
+ val callback = invocation.getArgument[Errors => Unit](3)
+
+ // Complete the transition and call the callback with success
+ txnMetadata.completeTransitionTo(newMetadata)
+ callback.apply(Errors.NONE)
+ })
+
+ coordinator.handleInitProducerId(
+ transactionalId,
+ txnTimeoutMs,
+ enableTwoPCFlag = false,
+ keepPreparedTxn = false,
+ None,
+ initProducerIdMockCallback
+ )
+
+ // The third attempt should succeed with epoch 3 (2 + 1)
+ // This demonstrates that TV2 allows epoch re-bumping after failed writes
+ assertEquals(InitProducerIdResult(producerId, 3.toShort, Errors.NONE), result)
+
+ // Final verification that hasFailedEpochFence was never set to true for TV2
+ assertFalse(txnMetadata.hasFailedEpochFence)
+ }
}
diff --git a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala
index e7a8e10d80e9f..52d9b01ff4aa3 100755
--- a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala
+++ b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala
@@ -31,7 +31,7 @@ import org.apache.kafka.common.errors.{InvalidConfigurationException, TimeoutExc
import org.apache.kafka.common.serialization.StringDeserializer
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.security.auth.SecurityProtocol
-import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, AlterConfigOp, AlterConfigsResult, ConfigEntry}
+import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, AlterConfigOp, AlterConfigsResult, ConfigEntry, FeatureUpdate, UpdateFeaturesOptions}
import org.apache.kafka.metadata.MetadataCache
import org.apache.kafka.server.config.ReplicationConfigs
import org.apache.kafka.server.metrics.KafkaYammerMetrics
@@ -42,6 +42,7 @@ import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.MethodSource
import com.yammer.metrics.core.Meter
import org.apache.kafka.metadata.LeaderConstants
+import org.apache.kafka.server.common.MetadataVersion
import org.apache.logging.log4j.core.config.Configurator
class UncleanLeaderElectionTest extends QuorumTestHarness {
@@ -119,6 +120,14 @@ class UncleanLeaderElectionTest extends QuorumTestHarness {
admin = TestUtils.createAdminClient(brokers, ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT), adminConfigs)
}
+ private def disableEligibleLeaderReplicas(): Unit = {
+ if (metadataVersion.isAtLeast(MetadataVersion.IBP_4_1_IV0)) {
+ admin.updateFeatures(
+ java.util.Map.of("eligible.leader.replicas.version", new FeatureUpdate(0, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE)),
+ new UpdateFeaturesOptions()).all().get()
+ }
+ }
+
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames)
@MethodSource(Array("getTestGroupProtocolParametersAll"))
def testUncleanLeaderElectionEnabled(groupProtocol: String): Unit = {
@@ -126,6 +135,7 @@ class UncleanLeaderElectionTest extends QuorumTestHarness {
configProps1.put("unclean.leader.election.enable", "true")
configProps2.put("unclean.leader.election.enable", "true")
startBrokers(Seq(configProps1, configProps2))
+ disableEligibleLeaderReplicas()
// create topic with 1 partition, 2 replicas, one on each broker
TestUtils.createTopicWithAdmin(admin, topic, brokers, controllerServers, replicaAssignment = Map(partitionId -> Seq(brokerId1, brokerId2)))
@@ -137,6 +147,7 @@ class UncleanLeaderElectionTest extends QuorumTestHarness {
def testUncleanLeaderElectionDisabled(groupProtocol: String): Unit = {
// unclean leader election is disabled by default
startBrokers(Seq(configProps1, configProps2))
+ disableEligibleLeaderReplicas()
// create topic with 1 partition, 2 replicas, one on each broker
TestUtils.createTopicWithAdmin(admin, topic, brokers, controllerServers, replicaAssignment = Map(partitionId -> Seq(brokerId1, brokerId2)))
@@ -151,6 +162,7 @@ class UncleanLeaderElectionTest extends QuorumTestHarness {
configProps1.put("unclean.leader.election.enable", "false")
configProps2.put("unclean.leader.election.enable", "false")
startBrokers(Seq(configProps1, configProps2))
+ disableEligibleLeaderReplicas()
// create topic with 1 partition, 2 replicas, one on each broker, and unclean leader election enabled
val topicProps = new Properties()
@@ -167,6 +179,7 @@ class UncleanLeaderElectionTest extends QuorumTestHarness {
configProps1.put("unclean.leader.election.enable", "true")
configProps2.put("unclean.leader.election.enable", "true")
startBrokers(Seq(configProps1, configProps2))
+ disableEligibleLeaderReplicas()
// create topic with 1 partition, 2 replicas, one on each broker, and unclean leader election disabled
val topicProps = new Properties()
@@ -180,6 +193,7 @@ class UncleanLeaderElectionTest extends QuorumTestHarness {
@MethodSource(Array("getTestGroupProtocolParametersAll"))
def testUncleanLeaderElectionInvalidTopicOverride(groupProtocol: String): Unit = {
startBrokers(Seq(configProps1))
+ disableEligibleLeaderReplicas()
// create topic with an invalid value for unclean leader election
val topicProps = new Properties()
@@ -328,6 +342,7 @@ class UncleanLeaderElectionTest extends QuorumTestHarness {
def testTopicUncleanLeaderElectionEnableWithAlterTopicConfigs(groupProtocol: String): Unit = {
// unclean leader election is disabled by default
startBrokers(Seq(configProps1, configProps2))
+ disableEligibleLeaderReplicas()
// create topic with 1 partition, 2 replicas, one on each broker
TestUtils.createTopicWithAdmin(admin, topic, brokers, controllerServers, replicaAssignment = Map(partitionId -> Seq(brokerId1, brokerId2)))
diff --git a/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestTest.scala b/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestTest.scala
index 33bd1c174defa..0f55feccb46a0 100644
--- a/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestTest.scala
@@ -207,6 +207,20 @@ class ConsumerGroupDescribeRequestTest(cluster: ClusterInstance) extends GroupCo
)
assertEquals(expected, actual)
+
+ val unknownGroupResponse = consumerGroupDescribe(
+ groupIds = List("grp-unknown"),
+ includeAuthorizedOperations = true,
+ version = version.toShort,
+ )
+ assertEquals(Errors.GROUP_ID_NOT_FOUND.code, unknownGroupResponse.head.errorCode())
+
+ val emptyGroupResponse = consumerGroupDescribe(
+ groupIds = List(""),
+ includeAuthorizedOperations = true,
+ version = version.toShort,
+ )
+ assertEquals(Errors.INVALID_GROUP_ID.code, emptyGroupResponse.head.errorCode())
}
} finally {
admin.close()
diff --git a/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala b/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala
index baf13cde2f0cf..506d0007924bb 100644
--- a/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala
@@ -301,6 +301,48 @@ class ConsumerGroupHeartbeatRequestTest(cluster: ClusterInstance) extends GroupC
}
}
+ @ClusterTest
+ def testEmptyConsumerGroupId(): Unit = {
+ val admin = cluster.admin()
+
+ // Creates the __consumer_offsets topics because it won't be created automatically
+ // in this test because it does not use FindCoordinator API.
+ try {
+ TestUtils.createOffsetsTopicWithAdmin(
+ admin = admin,
+ brokers = cluster.brokers.values().asScala.toSeq,
+ controllers = cluster.controllers().values().asScala.toSeq
+ )
+
+ // Heartbeat request to join the group. Note that the member subscribes
+ // to an nonexistent topic.
+ val consumerGroupHeartbeatRequest = new ConsumerGroupHeartbeatRequest.Builder(
+ new ConsumerGroupHeartbeatRequestData()
+ .setGroupId("")
+ .setMemberId(Uuid.randomUuid().toString)
+ .setMemberEpoch(0)
+ .setRebalanceTimeoutMs(5 * 60 * 1000)
+ .setSubscribedTopicNames(List("foo").asJava)
+ .setTopicPartitions(List.empty.asJava),
+ true
+ ).build()
+
+ // Send the request until receiving a successful response. There is a delay
+ // here because the group coordinator is loaded in the background.
+ var consumerGroupHeartbeatResponse: ConsumerGroupHeartbeatResponse = null
+ TestUtils.waitUntilTrue(() => {
+ consumerGroupHeartbeatResponse = connectAndReceive[ConsumerGroupHeartbeatResponse](consumerGroupHeartbeatRequest)
+ consumerGroupHeartbeatResponse.data.errorCode == Errors.INVALID_REQUEST.code
+ }, msg = s"Did not receive the expected error. Last response $consumerGroupHeartbeatResponse.")
+
+ // Verify the response.
+ assertEquals(Errors.INVALID_REQUEST.code, consumerGroupHeartbeatResponse.data.errorCode)
+ assertEquals("GroupId can't be empty.", consumerGroupHeartbeatResponse.data.errorMessage)
+ } finally {
+ admin.close()
+ }
+ }
+
@ClusterTest
def testConsumerGroupHeartbeatWithEmptySubscription(): Unit = {
val admin = cluster.admin()
diff --git a/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala
index fe4501e640a31..d945c02b6f5f5 100644
--- a/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala
@@ -78,8 +78,8 @@ class DeleteGroupsRequestTest(cluster: ClusterInstance) extends GroupCoordinator
)
deleteGroups(
- groupIds = List("grp-non-empty", "grp"),
- expectedErrors = List(Errors.NON_EMPTY_GROUP, Errors.NONE),
+ groupIds = List("grp-non-empty", "grp", ""),
+ expectedErrors = List(Errors.NON_EMPTY_GROUP, Errors.NONE, Errors.GROUP_ID_NOT_FOUND),
version = version.toShort
)
diff --git a/core/src/test/scala/unit/kafka/server/DescribeGroupsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DescribeGroupsRequestTest.scala
index d16ea1fd9e56c..67db0449ffe72 100644
--- a/core/src/test/scala/unit/kafka/server/DescribeGroupsRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/DescribeGroupsRequestTest.scala
@@ -93,10 +93,15 @@ class DescribeGroupsRequestTest(cluster: ClusterInstance) extends GroupCoordinat
.setGroupId("grp-unknown")
.setGroupState(ClassicGroupState.DEAD.toString) // Return DEAD group when the group does not exist.
.setErrorCode(if (version >= 6) Errors.GROUP_ID_NOT_FOUND.code() else Errors.NONE.code())
- .setErrorMessage(if (version >= 6) "Group grp-unknown not found." else null)
+ .setErrorMessage(if (version >= 6) "Group grp-unknown not found." else null),
+ new DescribedGroup()
+ .setGroupId("")
+ .setGroupState(ClassicGroupState.DEAD.toString) // Return DEAD group when the group does not exist.
+ .setErrorCode(if (version >= 6) Errors.GROUP_ID_NOT_FOUND.code() else Errors.NONE.code())
+ .setErrorMessage(if (version >= 6) "Group not found." else null)
),
describeGroups(
- groupIds = List("grp-1", "grp-2", "grp-unknown"),
+ groupIds = List("grp-1", "grp-2", "grp-unknown", ""),
version = version.toShort
)
)
diff --git a/core/src/test/scala/unit/kafka/server/HeartbeatRequestTest.scala b/core/src/test/scala/unit/kafka/server/HeartbeatRequestTest.scala
index 0bc22194dc4f8..3026cdecb2754 100644
--- a/core/src/test/scala/unit/kafka/server/HeartbeatRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/HeartbeatRequestTest.scala
@@ -179,6 +179,15 @@ class HeartbeatRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBas
expectedError = Errors.UNKNOWN_MEMBER_ID,
version = version.toShort
)
+
+ // Heartbeat with empty group id.
+ heartbeat(
+ groupId = "",
+ memberId = leaderMemberId,
+ generationId = -1,
+ expectedError = Errors.INVALID_GROUP_ID,
+ version = version.toShort
+ )
}
}
}
diff --git a/core/src/test/scala/unit/kafka/server/JoinGroupRequestTest.scala b/core/src/test/scala/unit/kafka/server/JoinGroupRequestTest.scala
index b6c21af8abfe2..0f2ab3669c90d 100644
--- a/core/src/test/scala/unit/kafka/server/JoinGroupRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/JoinGroupRequestTest.scala
@@ -139,6 +139,17 @@ class JoinGroupRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBas
)
)
+ // Join with an empty group id.
+ verifyJoinGroupResponseDataEquals(
+ new JoinGroupResponseData()
+ .setErrorCode(Errors.INVALID_GROUP_ID.code)
+ .setProtocolName(if (version >= 7) null else ""),
+ sendJoinRequest(
+ groupId = "",
+ version = version.toShort
+ )
+ )
+
// Join with an inconsistent protocolType.
verifyJoinGroupResponseDataEquals(
new JoinGroupResponseData()
diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
index 034c49c5c0775..a6c2658963515 100644
--- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
+++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
@@ -3924,7 +3924,7 @@ class KafkaApisTest extends Logging {
MetadataCacheTest.updateCache(metadataCache, partitionRecords)
// 4. Send TopicMetadataReq using topicId
- val metadataReqByTopicId = new MetadataRequest.Builder(util.List.of(authorizedTopicId, unauthorizedTopicId)).build()
+ val metadataReqByTopicId = MetadataRequest.Builder.forTopicIds(util.Set.of(authorizedTopicId, unauthorizedTopicId)).build()
val repByTopicId = buildRequest(metadataReqByTopicId, plaintextListener)
when(clientRequestQuotaManager.maybeRecordAndGetThrottleTimeMs(any[RequestChannel.Request](),
any[Long])).thenReturn(0)
diff --git a/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala
index 326f02ec7e293..75bf82ef155d7 100644
--- a/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala
@@ -233,6 +233,42 @@ class OffsetFetchRequestTest(cluster: ClusterInstance) extends GroupCoordinatorB
)
)
+ // Fetch with empty group id.
+ assertEquals(
+ new OffsetFetchResponseData.OffsetFetchResponseGroup()
+ .setGroupId("")
+ .setTopics(List(
+ new OffsetFetchResponseData.OffsetFetchResponseTopics()
+ .setName(if (version < 10) "foo" else "")
+ .setTopicId(if (version >= 10) topicId else Uuid.ZERO_UUID)
+ .setPartitions(List(
+ new OffsetFetchResponseData.OffsetFetchResponsePartitions()
+ .setPartitionIndex(0)
+ .setCommittedOffset(-1L),
+ new OffsetFetchResponseData.OffsetFetchResponsePartitions()
+ .setPartitionIndex(1)
+ .setCommittedOffset(-1L),
+ new OffsetFetchResponseData.OffsetFetchResponsePartitions()
+ .setPartitionIndex(5)
+ .setCommittedOffset(-1L)
+ ).asJava)
+ ).asJava),
+ fetchOffsets(
+ group = new OffsetFetchRequestData.OffsetFetchRequestGroup()
+ .setGroupId("")
+ .setMemberId(memberId)
+ .setMemberEpoch(memberEpoch)
+ .setTopics(List(
+ new OffsetFetchRequestData.OffsetFetchRequestTopics()
+ .setName("foo")
+ .setTopicId(topicId)
+ .setPartitionIndexes(List[Integer](0, 1, 5).asJava) // 5 does not exist.
+ ).asJava),
+ requireStable = requireStable,
+ version = version.toShort
+ )
+ )
+
// Fetch with stale member epoch.
assertEquals(
new OffsetFetchResponseData.OffsetFetchResponseGroup()
@@ -611,4 +647,53 @@ class OffsetFetchRequestTest(cluster: ClusterInstance) extends GroupCoordinatorB
)
}
}
+
+ @ClusterTest
+ def testGroupErrors(): Unit = {
+ val topicId = createTopic(
+ topic = "foo",
+ numPartitions = 3
+ )
+
+ for (version <- ApiKeys.OFFSET_FETCH.oldestVersion() to ApiKeys.OFFSET_FETCH.latestVersion(isUnstableApiEnabled)) {
+ assertEquals(
+ if (version >= 2) {
+ new OffsetFetchResponseData.OffsetFetchResponseGroup()
+ .setGroupId("unknown")
+ .setErrorCode(Errors.NOT_COORDINATOR.code)
+ } else {
+ // Version 1 does not support group level errors. Hence, the error is
+ // returned at the partition level.
+ new OffsetFetchResponseData.OffsetFetchResponseGroup()
+ .setGroupId("unknown")
+ .setTopics(List(
+ new OffsetFetchResponseData.OffsetFetchResponseTopics()
+ .setName("foo")
+ .setPartitions(List(
+ new OffsetFetchResponseData.OffsetFetchResponsePartitions()
+ .setPartitionIndex(0)
+ .setErrorCode(Errors.NOT_COORDINATOR.code)
+ .setCommittedOffset(-1)
+ .setCommittedLeaderEpoch(-1)
+ .setMetadata("")
+ ).asJava)
+ ).asJava)
+ },
+ fetchOffsets(
+ group = new OffsetFetchRequestData.OffsetFetchRequestGroup()
+ .setGroupId("unknown")
+ .setMemberId("")
+ .setMemberEpoch(0)
+ .setTopics(List(
+ new OffsetFetchRequestData.OffsetFetchRequestTopics()
+ .setName("foo")
+ .setTopicId(topicId)
+ .setPartitionIndexes(List[Integer](0).asJava)
+ ).asJava),
+ requireStable = false,
+ version = version.toShort
+ )
+ )
+ }
+ }
}
diff --git a/core/src/test/scala/unit/kafka/server/SyncGroupRequestTest.scala b/core/src/test/scala/unit/kafka/server/SyncGroupRequestTest.scala
index 3b95d1c9d4170..09ed807db8ecb 100644
--- a/core/src/test/scala/unit/kafka/server/SyncGroupRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/SyncGroupRequestTest.scala
@@ -64,6 +64,17 @@ class SyncGroupRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBas
version = version.toShort
)
+ // Sync with empty group id.
+ verifySyncGroupWithOldProtocol(
+ groupId = "",
+ memberId = "member-id",
+ generationId = -1,
+ expectedProtocolType = null,
+ expectedProtocolName = null,
+ expectedError = Errors.INVALID_GROUP_ID,
+ version = version.toShort
+ )
+
val metadata = ConsumerProtocol.serializeSubscription(
new ConsumerPartitionAssignor.Subscription(Collections.singletonList("foo"))
).array
diff --git a/docker/native/native-image-configs/reflect-config.json b/docker/native/native-image-configs/reflect-config.json
index f4263a08898f2..c953ea9fe1d8e 100644
--- a/docker/native/native-image-configs/reflect-config.json
+++ b/docker/native/native-image-configs/reflect-config.json
@@ -1023,6 +1023,12 @@
"name":"org.apache.kafka.common.security.kerberos.KerberosLogin",
"methods":[{"name":"","parameterTypes":[] }]
},
+{
+ "name":"org.apache.kafka.common.security.oauthbearer.DefaultJwtRetriever"
+},
+{
+ "name":"org.apache.kafka.common.security.oauthbearer.DefaultJwtValidator"
+},
{
"name":"org.apache.kafka.common.security.plain.PlainLoginModule",
"methods":[{"name":"","parameterTypes":[] }]
@@ -1067,6 +1073,18 @@
"name":"org.apache.kafka.metadata.authorizer.StandardAuthorizer",
"methods":[{"name":"","parameterTypes":[] }]
},
+{
+ "name":"org.apache.kafka.server.logger.LoggingController",
+ "queryAllPublicConstructors":true
+},
+{
+ "name":"org.apache.kafka.server.logger.LoggingControllerMBean",
+ "queryAllPublicMethods":true
+},
+{
+ "name":"org.apache.kafka.server.share.persister.DefaultStatePersister",
+ "methods":[{"name":"","parameterTypes":["org.apache.kafka.server.share.persister.PersisterStateManager"] }]
+},
{
"name":"org.apache.kafka.storage.internals.checkpoint.CleanShutdownFileHandler$Content",
"allDeclaredFields":true,
diff --git a/docs/configuration.html b/docs/configuration.html
index c922d92f10556..c060071518785 100644
--- a/docs/configuration.html
+++ b/docs/configuration.html
@@ -263,6 +263,16 @@
+ SCRAM, or
+ non-production/production OAUTHBEARER for example broker configurations.
JAAS configuration for Kafka clients
@@ -579,8 +579,8 @@ See GSSAPI (Kerberos),
PLAIN,
- SCRAM or
- OAUTHBEARER for example configurations.
+ SCRAM, or
+ non-production/production OAUTHBEARER for example client configurations.
JAAS configuration using static config file
To configure SASL authentication on the clients using static JAAS config file:
@@ -589,8 +589,8 @@ KafkaClient for the selected mechanism as described in the examples
for setting up GSSAPI (Kerberos),
PLAIN,
- SCRAM or
- OAUTHBEARER.
+ SCRAM, or
+ non-production/production OAUTHBEARER.
For example, GSSAPI
credentials may be configured as:
KafkaClient {
@@ -905,10 +905,13 @@ RFC 7628.
The default OAUTHBEARER implementation in Kafka creates and validates Unsecured JSON Web Tokens
and is only suitable for use in non-production Kafka installations. Refer to Security Considerations
- for more details.
+ for more details. Recent versions of Apache Kafka have added production-ready OAUTHBEARER implementations that support interaction with an OAuth 2.0-standards
+ compliant identity provider. Both modes are described in the following, noted where applicable.
Under the default implementation of principal.builder.class
, the principalName of OAuthBearerToken is used as the authenticated Principal
for configuration of ACLs etc.
- Configuring Kafka Brokers
+ Configuring Non-production Kafka Brokers
+ The default implementation of SASL/OAUTHBEARER in Kafka creates and validates Unsecured JSON Web Tokens.
+ While suitable only for non-production use, it does provide the flexibility to create arbitrary tokens in a DEV or TEST environment.
- Add a suitably modified JAAS file similar to the one below to each Kafka broker's config directory, let's call it kafka_server_jaas.conf for this example:
KafkaServer {
@@ -917,7 +920,40 @@
The property unsecuredLoginStringClaim_sub
in the KafkaServer
section is used by
the broker when it initiates connections to other brokers. In this example, admin will appear in the
- subject (sub
) claim and will be the user for inter-broker communication.
+ subject (sub
) claim and will be the user for inter-broker communication.
+
+ Here are the various supported JAAS module options on the broker side for Unsecured JSON Web Token validation:
+
+
+ JAAS Module Option for Unsecured Token Validation |
+ Documentation |
+
+
+ unsecuredValidatorPrincipalClaimName="value" |
+ Set to a non-empty value if you wish a particular String claim
+ holding a principal name to be checked for existence; the default is to check
+ for the existence of the 'sub ' claim. |
+
+
+ unsecuredValidatorScopeClaimName="value" |
+ Set to a custom claim name if you wish the name of the String or
+ String List claim holding any token scope to be something other than
+ 'scope '. |
+
+
+ unsecuredValidatorRequiredScope="value" |
+ Set to a space-delimited list of scope values if you wish the
+ String/String List claim holding the token scope to be checked to
+ make sure it contains certain values. |
+
+
+ unsecuredValidatorAllowableClockSkewMs="value" |
+ Set to a positive integer value if you wish to allow up to some number of
+ positive milliseconds of clock skew (the default is 0). |
+
+
+
+
- Pass the JAAS config file location as JVM parameter to each Kafka broker:
-Djava.security.auth.login.config=/etc/kafka/kafka_server_jaas.conf
- Configure SASL port and SASL mechanisms in server.properties as described here. For example:
@@ -927,8 +963,40 @@
+ Configuring Production Kafka Brokers
+
+ - Add a suitably modified JAAS file similar to the one below to each Kafka broker's config directory, let's call it kafka_server_jaas.conf for this example:
+
KafkaServer {
+ org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule required ;
+};
+ - Pass the JAAS config file location as JVM parameter to each Kafka broker:
+
-Djava.security.auth.login.config=/etc/kafka/kafka_server_jaas.conf
+ - Configure SASL port and SASL mechanisms in server.properties as described here. For example:
+
listeners=SASL_SSL://host.name:port
+security.inter.broker.protocol=SASL_SSL
+sasl.mechanism.inter.broker.protocol=OAUTHBEARER
+sasl.enabled.mechanisms=OAUTHBEARER
+listener.name.<listener name>.oauthbearer.sasl.server.callback.handler.class=org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallbackHandler
+listener.name.<listener name>.oauthbearer.sasl.oauthbearer.jwks.endpoint.url=https://example.com/oauth2/v1/keys
+
+ The OAUTHBEARER broker configuration includes:
+
+
+ - sasl.oauthbearer.clock.skew.seconds
+ - sasl.oauthbearer.expected.audience
+ - sasl.oauthbearer.expected.issuer
+ - sasl.oauthbearer.jwks.endpoint.refresh.ms
+ - sasl.oauthbearer.jwks.endpoint.retry.backoff.max.ms
+ - sasl.oauthbearer.jwks.endpoint.retry.backoff.ms
+ - sasl.oauthbearer.jwks.endpoint.url
+ - sasl.oauthbearer.scope.claim.name
+ - sasl.oauthbearer.sub.claim.name
+
+
+
+
- Configuring Kafka Clients
+ Configuring Non-production Kafka Clients
To configure SASL authentication on the clients:
- Configure the JAAS configuration property for each client in producer.properties or consumer.properties.
@@ -943,6 +1011,61 @@
sub)
claims in sasl.jaas.config
.
+ The default implementation of SASL/OAUTHBEARER in Kafka creates and validates Unsecured JSON Web Tokens.
+ While suitable only for non-production use, it does provide the flexibility to create arbitrary tokens in a DEV or TEST environment.
+ Here are the various supported JAAS module options on the client side (and on the broker side if OAUTHBEARER is the inter-broker protocol):
+
+
+ JAAS Module Option for Unsecured Token Creation |
+ Documentation |
+
+
+ unsecuredLoginStringClaim_<claimname>="value" |
+ Creates a String claim with the given name and value. Any valid
+ claim name can be specified except 'iat ' and 'exp ' (these are
+ automatically generated). |
+
+
+ unsecuredLoginNumberClaim_<claimname>="value" |
+ Creates a Number claim with the given name and value. Any valid
+ claim name can be specified except 'iat ' and 'exp ' (these are
+ automatically generated). |
+
+
+ unsecuredLoginListClaim_<claimname>="value" |
+ Creates a String List claim with the given name and values parsed
+ from the given value where the first character is taken as the delimiter. For
+ example: unsecuredLoginListClaim_fubar="|value1|value2" . Any valid
+ claim name can be specified except 'iat ' and 'exp ' (these are
+ automatically generated). |
+
+
+ unsecuredLoginExtension_<extensionname>="value" |
+ Creates a String extension with the given name and value.
+ For example: unsecuredLoginExtension_traceId="123" . A valid extension name
+ is any sequence of lowercase or uppercase alphabet characters. In addition, the "auth" extension name is reserved.
+ A valid extension value is any combination of characters with ASCII codes 1-127.
+ |
+
+ unsecuredLoginPrincipalClaimName |
+ Set to a custom claim name if you wish the name of the String
+ claim holding the principal name to be something other than 'sub '. |
+
+
+ unsecuredLoginLifetimeSeconds |
+ Set to an integer value if the token expiration is to be set to something
+ other than the default value of 3600 seconds (which is 1 hour). The
+ 'exp ' claim will be set to reflect the expiration time. |
+
+
+ unsecuredLoginScopeClaimName |
+ Set to a custom claim name if you wish the name of the String or
+ String List claim holding any token scope to be something other than
+ 'scope '. |
+
+
+
+
JAAS configuration for clients may alternatively be specified as a JVM parameter similar to brokers
as described here. Clients use the login section named
KafkaClient
. This option allows only one user for all client connections from a JVM.
@@ -953,101 +1076,64 @@
- Unsecured Token Creation Options for SASL/OAUTHBEARER
-
- - The default implementation of SASL/OAUTHBEARER in Kafka creates and validates Unsecured JSON Web Tokens.
- While suitable only for non-production use, it does provide the flexibility to create arbitrary tokens in a DEV or TEST environment.
- - Here are the various supported JAAS module options on the client side (and on the broker side if OAUTHBEARER is the inter-broker protocol):
-
-
- JAAS Module Option for Unsecured Token Creation |
- Documentation |
-
-
- unsecuredLoginStringClaim_<claimname>="value" |
- Creates a String claim with the given name and value. Any valid
- claim name can be specified except 'iat ' and 'exp ' (these are
- automatically generated). |
-
-
- unsecuredLoginNumberClaim_<claimname>="value" |
- Creates a Number claim with the given name and value. Any valid
- claim name can be specified except 'iat ' and 'exp ' (these are
- automatically generated). |
-
-
- unsecuredLoginListClaim_<claimname>="value" |
- Creates a String List claim with the given name and values parsed
- from the given value where the first character is taken as the delimiter. For
- example: unsecuredLoginListClaim_fubar="|value1|value2" . Any valid
- claim name can be specified except 'iat ' and 'exp ' (these are
- automatically generated). |
-
-
- unsecuredLoginExtension_<extensionname>="value" |
- Creates a String extension with the given name and value.
- For example: unsecuredLoginExtension_traceId="123" . A valid extension name
- is any sequence of lowercase or uppercase alphabet characters. In addition, the "auth" extension name is reserved.
- A valid extension value is any combination of characters with ASCII codes 1-127.
- |
-
- unsecuredLoginPrincipalClaimName |
- Set to a custom claim name if you wish the name of the String
- claim holding the principal name to be something other than 'sub '. |
-
-
- unsecuredLoginLifetimeSeconds |
- Set to an integer value if the token expiration is to be set to something
- other than the default value of 3600 seconds (which is 1 hour). The
- 'exp ' claim will be set to reflect the expiration time. |
-
-
- unsecuredLoginScopeClaimName |
- Set to a custom claim name if you wish the name of the String or
- String List claim holding any token scope to be something other than
- 'scope '. |
-
-
-
-
-
- Unsecured Token Validation Options for SASL/OAUTHBEARER
-
- - Here are the various supported JAAS module options on the broker side for Unsecured JSON Web Token validation:
-
-
- JAAS Module Option for Unsecured Token Validation |
- Documentation |
-
-
- unsecuredValidatorPrincipalClaimName="value" |
- Set to a non-empty value if you wish a particular String claim
- holding a principal name to be checked for existence; the default is to check
- for the existence of the 'sub ' claim. |
-
-
- unsecuredValidatorScopeClaimName="value" |
- Set to a custom claim name if you wish the name of the String or
- String List claim holding any token scope to be something other than
- 'scope '. |
-
-
- unsecuredValidatorRequiredScope="value" |
- Set to a space-delimited list of scope values if you wish the
- String/String List claim holding the token scope to be checked to
- make sure it contains certain values. |
-
-
- unsecuredValidatorAllowableClockSkewMs="value" |
- Set to a positive integer value if you wish to allow up to some number of
- positive milliseconds of clock skew (the default is 0). |
-
-
+ Configuring Production Kafka Clients
+ To configure SASL authentication on the clients:
+
+ - Configure the JAAS configuration property for each client in producer.properties or consumer.properties.
+ The login module describes how the clients like producer and consumer can connect to the Kafka Broker.
+ The following is an example configuration for a client for the OAUTHBEARER mechanisms:
+
sasl.jaas.config=org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule required ;
+
+ JAAS configuration for clients may alternatively be specified as a JVM parameter similar to brokers
+ as described here. Clients use the login section named
+ KafkaClient
. This option allows only one user for all client connections from a JVM.
+ - Configure the following properties in producer.properties or consumer.properties. For example, if using the OAuth
client_credentials
grant type
+ to communicate with the OAuth identity provider, the configuration might look like this:
+ security.protocol=SASL_SSL
+sasl.mechanism=OAUTHBEARER
+sasl.oauthbearer.jwt.retriever.class=org.apache.kafka.common.security.oauthbearer.ClientCredentialsJwtRetriever
+sasl.oauthbearer.client.credentials.client.id=jdoe
+sasl.oauthbearer.client.credentials.client.secret=$3cr3+
+sasl.oauthbearer.scope=my-application-scope
+sasl.oauthbearer.token.endpoint.url=https://example.com/oauth2/v1/token
+
+ Or, if using the OAuth urn:ietf:params:oauth:grant-type:jwt-bearer
grant type
+ to communicate with the OAuth identity provider, the configuration might look like this:
+ security.protocol=SASL_SSL
+sasl.mechanism=OAUTHBEARER
+sasl.oauthbearer.jwt.retriever.class=org.apache.kafka.common.security.oauthbearer.JwtBearerJwtRetriever
+sasl.oauthbearer.assertion.private.key.file=/path/to/private.key
+sasl.oauthbearer.assertion.algorithm=RS256
+sasl.oauthbearer.assertion.claim.exp.seconds=600
+sasl.oauthbearer.assertion.template.file=/path/to/template.json
+sasl.oauthbearer.scope=my-application-scope
+sasl.oauthbearer.token.endpoint.url=https://example.com/oauth2/v1/token
+ The OAUTHBEARER client configuration includes:
+
+
+ - sasl.oauthbearer.assertion.algorithm
+ - sasl.oauthbearer.assertion.claim.aud
+ - sasl.oauthbearer.assertion.claim.exp.seconds
+ - sasl.oauthbearer.assertion.claim.iss
+ - sasl.oauthbearer.assertion.claim.jti.include
+ - sasl.oauthbearer.assertion.claim.nbf.seconds
+ - sasl.oauthbearer.assertion.claim.sub
+ - sasl.oauthbearer.assertion.file
+ - sasl.oauthbearer.assertion.private.key.file
+ - sasl.oauthbearer.assertion.private.key.passphrase
+ - sasl.oauthbearer.assertion.template.file
+ - sasl.oauthbearer.client.credentials.client.id
+ - sasl.oauthbearer.client.credentials.client.secret
+ - sasl.oauthbearer.header.urlencode
+ - sasl.oauthbearer.jwt.retriever.class
+ - sasl.oauthbearer.jwt.validator.class
+ - sasl.oauthbearer.scope
+ - sasl.oauthbearer.token.endpoint.url
+
- - The default unsecured SASL/OAUTHBEARER implementation may be overridden (and must be overridden in production environments)
- using custom login and SASL Server callback handlers.
- - For more details on security considerations, refer to RFC 6749, Section 10.
-
+ The default implementation of SASL/OAUTHBEARER depends on the jackson-databind library.
+ Since it's an optional dependency, users have to configure it as a dependency via their build tool.
+
Token Refresh for SASL/OAUTHBEARER
Kafka periodically refreshes any token before it expires so that the client can continue to make
@@ -1125,7 +1211,7 @@
Follow the mechanism-specific steps in GSSAPI (Kerberos),
PLAIN,
- SCRAM and OAUTHBEARER
+ SCRAM, and non-production/production OAUTHBEARER
to configure SASL for the enabled mechanisms.
diff --git a/docs/streams/core-concepts.html b/docs/streams/core-concepts.html
index c400ca08453c6..a2d1b7209b551 100644
--- a/docs/streams/core-concepts.html
+++ b/docs/streams/core-concepts.html
@@ -279,7 +279,7 @@ Lambda Architecture.
+ to the stream processing pipeline, known as the Lambda Architecture.
Prior to 0.11.0.0, Kafka only provides at-least-once delivery guarantees and hence any stream processing systems that leverage it as the backend storage could not guarantee end-to-end exactly-once semantics.
In fact, even for those stream processing systems that claim to support exactly-once processing, as long as they are reading from / writing to Kafka as the source / sink, their applications cannot actually guarantee that
no duplicates will be generated throughout the pipeline.
diff --git a/docs/streams/developer-guide/config-streams.html b/docs/streams/developer-guide/config-streams.html
index c1f23e994e167..8c45d0c7976d0 100644
--- a/docs/streams/developer-guide/config-streams.html
+++ b/docs/streams/developer-guide/config-streams.html
@@ -72,14 +72,15 @@
Optional configuration parameters
Kafka consumers and producer configuration parameters
@@ -297,12 +297,12 @@ num.standby.replicascommit.interval.ms |
Low |
The frequency in milliseconds with which to save the position (offsets in source topics) of tasks. |
- 30000 (30 seconds) |
+ 30000 (30 seconds) (at-least-once) / 100 (exactly-once) |
default.deserialization.exception.handler (Deprecated. Use deserialization.exception.handler instead.) |
Medium |
Exception handling class that implements the DeserializationExceptionHandler interface. |
- LogAndContinueExceptionHandler |
+ LogAndFailExceptionHandler |
default.key.serde |
Medium |
@@ -327,11 +327,10 @@ num.standby.replicas
- default.dsl.store |
+
default.dsl.store (Deprecated. Use dsl.store.suppliers.class instead.) |
Low |
- [DEPRECATED] The default state store type used by DSL operators. Deprecated in
- favor of dsl.store.suppliers.class
+ The default state store type used by DSL operators.
|
"ROCKS_DB" |
@@ -491,58 +490,68 @@ num.standby.replicas