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> 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: *
    *
  1. 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, *
  2. Gets the internal producer id and epoch, used in all future transactional * messages issued by the producer.
  3. *
- * - *

- * 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: - *

    - *
  1. Prepare the transaction by calling this method. This returns a {@link PreparedTxnState} if successful.
  2. - *
  3. Make any external system changes that need to be atomic with this transaction.
  4. - *
  5. Complete the transaction by calling {@link #commitTransaction()}, {@link #abortTransaction()} or - * completeTransaction(PreparedTxnState).
  6. - *
- * - * @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 @@

+
  • +

    org.apache.kafka.sasl.oauthbearer.allowed.files

    +

    This system property is used to determine which files, if any, are allowed to be read by the SASL OAUTHBEARER plugin. This property accepts comma-separated list of files. By default the value is an empty list. +

    If users want to enable some files, users need to explicitly set the system property like below. +

    -Dorg.apache.kafka.sasl.oauthbearer.allowed.files=/tmp/token,/tmp/private_key.pem
    + + + +
    Since:4.1.0
    Default Value:
    +
  • org.apache.kafka.sasl.oauthbearer.allowed.urls

    This system property is used to set the allowed URLs as SASL OAUTHBEARER token or jwks endpoints. This property accepts comma-separated list of URLs. By default the value is an empty list. diff --git a/docs/documentation.html b/docs/documentation.html index b142263ac856a..e13042ddd48d1 100644 --- a/docs/documentation.html +++ b/docs/documentation.html @@ -33,7 +33,7 @@

    Documentation

    -

    Kafka 4.0 Documentation

    +

    Kafka 4.1 Documentation

    Prior releases: 0.7.x, 0.8.0, 0.8.1.X, @@ -64,6 +64,7 @@

    Kafka 4.0 Documentation

    3.7.X, 3.8.X, 3.9.X. + 4.0.X.

    1. Getting Started

    1.1 Introduction

    diff --git a/docs/js/templateData.js b/docs/js/templateData.js index cb834035f44e1..7c79371798476 100644 --- a/docs/js/templateData.js +++ b/docs/js/templateData.js @@ -19,6 +19,6 @@ limitations under the License. var context={ "version": "41", "dotVersion": "4.1", - "fullDotVersion": "4.1.0", + "fullDotVersion": "4.1.1-SNAPSHOT", "scalaVersion": "2.13" }; diff --git a/docs/ops.html b/docs/ops.html index 09842ee20848e..be0bfe89e8bf9 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -3983,20 +3983,56 @@
    Provisioning Nodes
    +

    Upgrade

    + +

    Apache Kafka 4.1 added support for upgrading a cluster from a static controller configuration to a dynamic controller configuration. Dynamic controller configuration allows users to add controller to and remove controller from the cluster. See the Controller membership changes section for more details.

    + +

    This feature upgrade is done by upgrading the KRaft feature version and updating the nodes' configuration.

    + +
    Describe KRaft Version
    + +

    Dynamic controller cluster was added in kraft.version=1 or release-version 4.1. To determine which kraft feature version the cluster is using you can execute the following CLI command:

    + +
    $ bin/kafka-features.sh --bootstrap-controller localhost:9093 describe
    +...
    +Feature: kraft.version  SupportedMinVersion: 0  SupportedMaxVersion: 1  FinalizedVersionLevel: 0        Epoch: 7
    +Feature: metadata.version       SupportedMinVersion: 3.3-IV3    SupportedMaxVersion: 4.0-IV3    FinalizedVersionLevel: 4.0-IV3  Epoch: 7
    + +

    If the FinalizedVersionLevel for Feature: kraft.version is 0, the version needs to be upgraded to at least 1 to support a dynamic controller cluster.

    + +
    Upgrade KRaft Version
    + +

    The KRaft feature version can be upgraded to support dynamic controller clusters by using the kafka-feature CLI command. To upgrade all of the feature versions to the latest version:

    + +
    $ bin/kafka-features.sh --bootstrap-server localhost:9092 upgrade --release-version 4.1
    + +

    To upgrade just the KRaft feature version:

    + +
    $ bin/kafka-features.sh --bootstrap-server localhost:9092 upgrade --feature kraft.version=1
    + +
    Update KRaft Config
    + +

    KRaft version 1 deprecated the controller.quorum.voters property and added the controller.quorum.bootstrap.servers property. After checking that the KRaft version has been successfully upgraded to at least version 1, remove the controller.quorum.voters property and add the controller.quorum.bootstrap.servers to all of the nodes (controllers and brokers) in the cluster.

    + +
    process.roles=...
    +node.id=...
    +controller.quorum.bootstrap.servers=controller1.example.com:9093,controller2.example.com:9093,controller3.example.com:9093
    +controller.listener.names=CONTROLLER
    + +

    Provisioning Nodes

    The bin/kafka-storage.sh random-uuid command can be used to generate a cluster ID for your new cluster. This cluster ID must be used when formatting each server in the cluster with the bin/kafka-storage.sh format command.

    This is different from how Kafka has operated in the past. Previously, Kafka would format blank storage directories automatically, and also generate a new cluster ID automatically. One reason for the change is that auto-formatting can sometimes obscure an error condition. This is particularly important for the metadata log maintained by the controller and broker servers. If a majority of the controllers were able to start with an empty log directory, a leader might be able to be elected with missing committed data.

    -
    Bootstrap a Standalone Controller
    +
    Bootstrap a Standalone Controller
    The recommended method for creating a new KRaft controller cluster is to bootstrap it with one voter and dynamically add the rest of the controllers. Bootstrapping the first controller can be done with the following CLI command:
    $ bin/kafka-storage.sh format --cluster-id <CLUSTER_ID> --standalone --config config/controller.properties
    This command will 1) create a meta.properties file in metadata.log.dir with a randomly generated directory.id, 2) create a snapshot at 00000000000000000000-0000000000.checkpoint with the necessary control records (KRaftVersionRecord and VotersRecord) to make this Kafka node the only voter for the quorum. -
    Bootstrap with Multiple Controllers
    +
    Bootstrap with Multiple Controllers
    The KRaft cluster metadata partition can also be bootstrapped with more than one voter. This can be done by using the --initial-controllers flag:
    CLUSTER_ID="$(bin/kafka-storage.sh random-uuid)"
    @@ -4013,7 +4049,7 @@ 
    In the replica description 0@controller-0:1234:3Db5QLSqSZieL3rJBUUegA, 0 is the replica id, 3Db5QLSqSZieL3rJBUUegA is the replica directory id, controller-0 is the replica's host and 1234 is the replica's port. -
    Formatting Brokers and New Controllers
    +
    Formatting Brokers and New Controllers
    When provisioning new broker and controller nodes that we want to add to an existing Kafka cluster, use the kafka-storage.sh format command with the --no-initial-controllers flag.
    $ bin/kafka-storage.sh format --cluster-id <CLUSTER_ID> --config config/server.properties --no-initial-controllers
    @@ -4077,7 +4113,7 @@
    Add New Controller
    - If a dynamic controller cluster already exists, it can be expanded by first provisioning a new controller using the kafka-storage.sh tool and starting the controller. + If a dynamic controller cluster already exists, it can be expanded by first provisioning a new controller using the kafka-storage.sh tool and starting the controller. After starting the controller, the replication to the new controller can be monitored using the bin/kafka-metadata-quorum.sh describe --replication command. Once the new controller has caught up to the active controller, it can be added to the cluster using the bin/kafka-metadata-quorum.sh add-controller command. @@ -4463,9 +4499,16 @@

    Tool

    -

    The ELR fields can be checked through the API DescribeTopicPartitions. The admin client can fetch the ELR info by describing the topics. - Also note that, if min.insync.replicas is updated for a topic, the ELR field will be cleaned. If cluster default min ISR is updated, - all the ELR fields will be cleaned.

    +

    The ELR fields can be checked through the API DescribeTopicPartitions. The admin client can fetch the ELR info by describing the topics.

    +

    Note that when the ELR feature is enabled:

    +
      +
    • The cluster-level min.insync.replicas config will be added if there is not any. The value is the same as the static config in the active controller.
    • +
    • The removal of min.insync.replicas config at the cluster-level is not allowed.
    • +
    • If the cluster-level min.insync.replicas is updated, even if the value is unchanged, all the ELR state will be cleaned.
    • +
    • The previously set min.insync.replicas value at the broker-level config will be removed. Please set at the cluster-level if necessary.
    • +
    • The alteration of min.insync.replicas config at the broker-level is not allowed.
    • +
    • If min.insync.replicas is updated for a topic, the ELR state will be cleaned.
    • +
    diff --git a/docs/security.html b/docs/security.html index a2deee2b3a38a..5940fc3cda669 100644 --- a/docs/security.html +++ b/docs/security.html @@ -556,8 +556,8 @@

    See GSSAPI (Kerberos), PLAIN, - SCRAM or - OAUTHBEARER for example broker configurations.

  • + 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.
      -
    1. Configuring Kafka Brokers
      +
    2. 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.

      1. 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.
      2. + 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 ValidationDocumentation
        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).
        +

        +
      3. Pass the JAAS config file location as JVM parameter to each Kafka broker:
        -Djava.security.auth.login.config=/etc/kafka/kafka_server_jaas.conf
      4. Configure SASL port and SASL mechanisms in server.properties as described here. For example: @@ -927,8 +963,40 @@

  • +
  • Configuring Production Kafka Brokers
    +
      +
    1. 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 ;
      +};
    2. +
    3. Pass the JAAS config file location as JVM parameter to each Kafka broker: +
      -Djava.security.auth.login.config=/etc/kafka/kafka_server_jaas.conf
    4. +
    5. 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
      • +
      +
    6. +
    +
  • -
  • Configuring Kafka Clients
    +
  • Configuring Non-production Kafka Clients
    To configure SASL authentication on the clients:
    1. 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 CreationDocumentation
      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. +
      unsecuredLoginPrincipalClaimNameSet to a custom claim name if you wish the name of the String + claim holding the principal name to be something other than 'sub'.
      unsecuredLoginLifetimeSecondsSet 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.
      unsecuredLoginScopeClaimNameSet 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.

    2. @@ -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 CreationDocumentation
      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. -
      unsecuredLoginPrincipalClaimNameSet to a custom claim name if you wish the name of the String - claim holding the principal name to be something other than 'sub'.
      unsecuredLoginLifetimeSecondsSet 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.
      unsecuredLoginScopeClaimNameSet 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 ValidationDocumentation
      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: +
        +
      1. 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.

      2. +
      3. 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
        • +
      4. -
      5. The default unsecured SASL/OAUTHBEARER implementation may be overridden (and must be overridden in production environments) - using custom login and SASL Server callback handlers.
      6. -
      7. For more details on security considerations, refer to RFC 6749, Section 10.
      8. -
    +
  • 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.replicasnull - 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-1 - retry.backoff.ms + repartition.purge.interval.ms + Low + The frequency in milliseconds with which to delete fully consumed records from repartition topics. Purging will occur after at least this value since the last purge, but may be delayed until later. + 30000 (30 seconds) + + retry.backoff.ms Low The amount of time in milliseconds, before a request is retried. 100 - rocksdb.config.setter + rocksdb.config.setter Medium The RocksDB configuration. null - state.cleanup.delay.ms + state.cleanup.delay.ms Low The amount of time in milliseconds to wait before deleting state when a partition has migrated. 600000 (10 minutes) - state.dir + state.dir High Directory location for state stores. /${java.io.tmpdir}/kafka-streams - task.assignor.class + task.assignor.class Medium A task assignor class or class name implementing the TaskAssignor interface. The high-availability task assignor. - task.timeout.ms + task.timeout.ms Medium The maximum amount of time in milliseconds a task might stall due to internal errors and retries until an error is raised. For a timeout of 0 ms, a task would raise an error for the first internal error. For any timeout larger than 0 ms, a task will retry at least once before an error is raised. 300000 (5 minutes) - topology.optimization + topology.optimization Medium A configuration telling Kafka Streams if it should optimize the topology and what optimizations to apply. Acceptable values are: StreamsConfig.NO_OPTIMIZATION (none), StreamsConfig.OPTIMIZE (all) or a comma separated list of specific optimizations: StreamsConfig.REUSE_KTABLE_SOURCE_TOPICS (reuse.ktable.source.topics), StreamsConfig.MERGE_REPARTITION_TOPICS (merge.repartition.topics), StreamsConfig.SINGLE_STORE_SELF_JOIN (single.store.self.join). "NO_OPTIMIZATION" - upgrade.from + upgrade.from Medium The version you are upgrading from during a rolling upgrade. See Upgrade From null - windowstore.changelog.additional.retention.ms + windowstore.changelog.additional.retention.ms Low Added to a windows maintainMs to ensure data is not deleted from the log prematurely. Allows for clock drift. 86400000 (1 day) - window.size.ms + window.size.ms (Deprecated. See Window Serdes for alternatives.) Low Sets window size for the deserializer in order to calculate window end times. null + windowed.inner.class.serde (Deprecated. See Window Serdes for alternatives.) + Low + Serde for the inner class of a windowed record. Must implement the Serde interface. + null + +
    +

    group.protocol

    +
    +
    +

    + The group protocol used by the Kafka Streams client used for coordination. + It determines how the client will communicate with the Kafka brokers and other clients in the same group. + The default value is "classic", which is the classic consumer group protocol. + Can be set to "streams" (requires broker-side enablement) to enable the new Kafka Streams group protocol. + Note that the "streams" rebalance protocol is an Early Access feature and should not be used in production. +

    +
    +
    +

    rack.aware.assignment.non_overlap_cost

    @@ -1205,18 +1228,6 @@

    topology.optimization -

    windowed.inner.class.serde

    -
    -
    -

    - Serde for the inner class of a windowed record. Must implement the org.apache.kafka.common.serialization.Serde interface. -

    -

    - Note that this config is only used by plain consumer/producer clients that set a windowed de/serializer type via configs. For Kafka Streams applications that deal with windowed types, you must pass in the inner serde type when you instantiate the windowed serde object for your topology. -

    -
    -

    upgrade.from

    diff --git a/docs/streams/developer-guide/datatypes.html b/docs/streams/developer-guide/datatypes.html index afa6397c5ef40..2bc2d7d5d0ef3 100644 --- a/docs/streams/developer-guide/datatypes.html +++ b/docs/streams/developer-guide/datatypes.html @@ -48,9 +48,10 @@ -
  • Kafka Streams DSL for Scala Implicit Serdes
  • +
  • Kafka Streams DSL for Scala Implicit Serdes
  • Configuring Serdes

    @@ -103,7 +104,7 @@

    Primitive and basic types<dependency> <groupId>org.apache.kafka</groupId> <artifactId>kafka-clients</artifactId> - <version>2.8.0</version> + <version>{{fullDotVersion}}</version> </dependency>

    This artifact provides the following serde implementations under the package org.apache.kafka.common.serialization, which you can leverage when e.g., defining default serializers in your Streams configuration.

    @@ -163,6 +164,76 @@

    JSONAs shown in the example, you can use JSONSerdes inner classes Serdes.serdeFrom(<serializerInstance>, <deserializerInstance>) to construct JSON compatible serializers and deserializers.

    +
    +

    Window Serdes

    +

    Apache Kafka Streams includes serde implementations for windowed types in + its kafka-streams Maven artifact:

    +
    <dependency>
    +    <groupId>org.apache.kafka</groupId>
    +    <artifactId>kafka-streams</artifactId>
    +    <version>{{fullDotVersion}}</version>
    +</dependency>
    +

    This artifact provides the following windowed serde implementations under the package org.apache.kafka.streams.kstream:

    + +

    Serdes:

    +
      +
    • WindowedSerdes.TimeWindowedSerde<T>
    • +
    • WindowedSerdes.SessionWindowedSerde<T>
    • +
    + +

    Serializers:

    +
      +
    • TimeWindowedSerializer<T>
    • +
    • SessionWindowedSerializer<T>
    • +
    + +

    Deserializers:

    +
      +
    • TimeWindowedDeserializer<T>
    • +
    • SessionWindowedDeserializer<T>
    • +
    +

    Usage in Code

    +

    When using windowed serdes in your application code, you typically create instances via constructors or factory methods:

    +
    // Time windowed serde - using factory method
    +Serde<Windowed<String>> timeWindowedSerde = 
    +    WindowedSerdes.timeWindowedSerdeFrom(String.class, 500L);
    +
    +// Time windowed serde - using constructor
    +Serde<Windowed<String>> timeWindowedSerde2 = 
    +    new WindowedSerdes.TimeWindowedSerde<>(Serdes.String(), 500L);
    +
    +// Session windowed serde - using factory method
    +Serde<Windowed<String>> sessionWindowedSerde = 
    +    WindowedSerdes.sessionWindowedSerdeFrom(String.class);
    +
    +// Session windowed serde - using constructor  
    +Serde<Windowed<String>> sessionWindowedSerde2 = 
    +    new WindowedSerdes.SessionWindowedSerde<>(Serdes.String());
    +
    +// Using individual serializers/deserializers
    +TimeWindowedSerializer<String> serializer = new TimeWindowedSerializer<>(Serdes.String().serializer());
    +TimeWindowedDeserializer<String> deserializer = new TimeWindowedDeserializer<>(Serdes.String().deserializer(), 500L);
    + +

    Usage in Command Line

    +

    When using command-line tools (like bin/kafka-console-consumer.sh), you can configure windowed deserializers by passing the inner class and window size via configuration properties. The property names use a prefix pattern:

    +
    # Time windowed deserializer configuration
    +--property print.key=true \
    +--property key.deserializer=org.apache.kafka.streams.kstream.TimeWindowedDeserializer \
    +--property key.deserializer.windowed.inner.deserializer.class=org.apache.kafka.common.serialization.StringDeserializer \
    +--property key.deserializer.window.size.ms=500
    +
    +# Session windowed deserializer configuration  
    +--property print.key=true \
    +--property key.deserializer=org.apache.kafka.streams.kstream.SessionWindowedDeserializer \
    +--property key.deserializer.windowed.inner.deserializer.class=org.apache.kafka.common.serialization.StringDeserializer
    + +

    Deprecated Configs

    +

    The following StreamsConfig parameters are deprecated in favor of passing parameters directly to serializer/deserializer constructors:

    +
      +
    • StreamsConfig.WINDOWED_INNER_CLASS_SERDE is deprecated in favor of TimeWindowedSerializer.WINDOWED_INNER_SERIALIZER_CLASS and TimeWindowedDeserializer.WINDOWED_INNER_DESERIALIZER_CLASS
    • +
    • StreamsConfig.WINDOW_SIZE_MS_CONFIG is deprecated in favor of TimeWindowedDeserializer.WINDOW_SIZE_MS_CONFIG
    • +
    +

    Implementing custom Serdes

    If you need to implement custom Serdes, your best starting point is to take a look at the source code references of diff --git a/docs/streams/developer-guide/dsl-api.html b/docs/streams/developer-guide/dsl-api.html index 8e8a36f76c075..4de5389ac75f4 100644 --- a/docs/streams/developer-guide/dsl-api.html +++ b/docs/streams/developer-guide/dsl-api.html @@ -3130,15 +3130,20 @@

    Operations and concepts

    Processor (provided by a given ProcessorSupplier);
  • KStream#processValues: Process all records in a stream, one record at a time, by applying a - FixedKeyProcessor (provided by a given FixedKeyProcessorSupplier); + FixedKeyProcessor (provided by a given FixedKeyProcessorSupplier) + [CAUTION: If you are deploying a new Kafka Streams application, and you are using the + "merge repartition topics" optimization, you should enable the fix for + KAFKA-19668 to avoid compatibility + issues for future upgrades to newer versions of Kafka Streams; + For more details, see the migration guide below];
  • Processor: A processor of key-value pair records;
  • ContextualProcessor: An abstract implementation of Processor that manages the - ProcessorContext instance. + ProcessorContext instance;
  • FixedKeyProcessor: A processor of key-value pair records where keys are immutable;
  • ContextualFixedKeyProcessor: An abstract implementation of FixedKeyProcessor that - manages the FixedKeyProcessorContext instance. + manages the FixedKeyProcessorContext instance;
  • ProcessorSupplier: A processor supplier that can create one or more Processor instances; and
  • @@ -3456,6 +3461,25 @@

    The Processor API now serves as a unified replacement for all these methods. It simplifies the API surface while maintaining support for both stateless and stateful operations.

    + +

    CAUTION: If you are using KStream.transformValues() and you have the "merge repartition topics" + optimization enabled, rewriting your program to KStream.processValues() might not be safe due to + KAFKA-19668. For this case, you should not upgrade + to Kafka Streams 4.0.0 or 4.1.0, but use Kafka Streams 4.0.1 instead, which contains a fix. + Note, that the fix is not enabled by default for backward compatibility reasons, and you would need to + enable the fix by setting config __enable.process.processValue.fix__ = true and pass it + into StreamsBuilder() constructor.

    +
    final Properties properties = new Properties();
    +properties.put(StreamsConfig.APPLICATION_ID_CONFIG, ...);
    +properties.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, ...);
    +properties.put(TopologyConfig.InternalConfig.ENABLE_PROCESS_PROCESSVALUE_FIX, true);
    +
    +final StreamsBuilder builder = new StreamsBuilder(new TopologyConfig(new StreamsConfig(properties)));
    + +

    It is recommended, that you compare the output of Topology.describe() for the old and new topology, + to verify if the rewrite to processValues() is correct, and that it does not introduce any incompatibilities. + You should also test the upgrade in a non-production environment.

    +

    Migration Examples

    To migrate from the deprecated transform, transformValues, flatTransform, and flatTransformValues methods to the Processor API (PAPI) in Kafka Streams, let's resume the diff --git a/docs/streams/upgrade-guide.html b/docs/streams/upgrade-guide.html index a203190756340..d5c926923e029 100644 --- a/docs/streams/upgrade-guide.html +++ b/docs/streams/upgrade-guide.html @@ -141,6 +141,110 @@

    <

    Streams API changes in 4.1.0

    +

    Early Access of the Streams Rebalance Protocol

    + +

    + The Streams Rebalance Protocol is a broker-driven rebalancing system designed specifically for Kafka + Streams applications. Following the pattern of KIP-848, which moved rebalance coordination of plain consumers + from clients to brokers, KIP-1071 extends this model to Kafka Streams workloads. Instead of clients + computing new assignments on the client during rebalance events involving all members of the group, assignments are + computed continuously on the broker. Instead of using a consumer group, the streams application registers as a + streams group with the broker, which manages and exposes all metadata required for coordination of the + streams application instances. +

    + +

    + This Early Access release covers a subset of the functionality detailed in + KIP-1071. + Do not use the new protocol in production. The API is subject to change in future + releases. +

    + +

    What's Included in Early Access

    + +
      +
    • Core Streams Group Rebalance Protocol: The group.protocol=streams configuration + enables the dedicated streams rebalance protocol. This separates streams groups from consumer groups and + provides a streams-specific group membership lifecycle and metadata management on the broker.
    • +
    • Sticky Task Assignor: A basic task assignment strategy that minimizes task movement + during rebalances is included.
    • +
    • Interactive Query Support: IQ operations are compatible with the new streams protocol.
    • +
    • New Admin RPC: The StreamsGroupDescribe RPC provides streams-specific metadata + separate from consumer group information, with corresponding access via the Admin client.
    • +
    • CLI Integration: You can list, describe, and delete streams groups via the kafka-streams-groups.sh script.
    • +
    + +

    What's Not Included in Early Access

    + +
      +
    • Static Membership: Setting a client `instance.id` will be rejected.
    • +
    • Topology Updates: If a topology is changed significantly (e.g., by adding new source topics + or changing the number of sub-topologies), a new streams group must be created.
    • +
    • High Availability Assignor: Only the sticky assignor is supported.
    • +
    • Regular Expressions: Pattern-based topic subscription is not supported.
    • +
    • Reset Operations: CLI offset reset operations are not supported.
    • +
    • Protocol Migration: Group migration is not available between the classic and new streams protocols.
    • +
    + +

    Why Use the Streams Rebalance Protocol?

    + +
      +
    • + Broker-Driven Coordination: + Centralizes task assignment logic on brokers instead of the client. This provides consistent, + authoritative task assignment decisions from a single coordination point and reduces the potential for + split-brain scenarios. +
    • +
    • + Faster, More Stable Rebalances: + Reduces rebalance duration and impact by removing the global synchronization point. This minimizes + application downtime during membership changes or failures. +
    • +
    • + Better Observability: + Provides dedicated metrics and admin interfaces that separate streams from consumer groups, leading to + clearer troubleshooting with broker-side observability. +
    • +
    + +

    + Enabling the protocol requires the brokers and clients are running Apache Kafka 4.1. It should be enabled + only on new clusters for testing purposes. + Set unstable.feature.versions.enable=true for controllers and brokers, and + set unstable.api.versions.enable=true on the brokers as well. In your Kafka Streams application + configuration, set group.protocol=streams. + After the new feature is configured, check + kafka-features.sh --bootstrap-server localhost:9092 describe + and `streams.version` should now have FinalizedVersionLevel 1. +

    + +

    + Migration between the classic consumer group protocol and the Streams Rebalance Protocol is not supported in + either direction. An application using this protocol must use a new application.id that has not + been used by any application on the classic protocol. Furthermore, this ID must not be in use as a + group.id by any consumer ("classic" or "consumer") nor share-group application. + It is also possible to delete a previous consumer group using kafka-consumer-groups.sh before + starting the application with the new protocol, which will however also delete all offsets for that group. +

    + +

    + To operate the new streams groups, explore the options of kafka-streams-groups.sh to list, + describe, and delete streams groups. In the new protocol, session.timeout.ms, + heartbeat.interval.ms and num.standby.replicas are group-level configurations, + which are ignored when they are set on the client side. Use the kafka-configs.sh tool to set + these configurations, for example: + kafka-configs.sh --bootstrap-server localhost:9092 --alter --entity-type groups + --entity-name wordcount --add-config streams.num.standby.replicas=1. +

    + +

    + Please provide feedback on this feature via the + Kafka mailing lists or by filing + JIRA issues. +

    + +

    Other changes

    +

    The introduction of KIP-1111 enables you to enforce explicit naming for all internal resources of the topology, including internal topics (e.g., changelog and repartition topics) and their associated state stores. @@ -158,7 +262,15 @@

    Streams API +
  • + The filename for rotated state-change.log files incorrectly rotates to stage-change.log.[date] (changing state to stage). This issue is corrected in 4.0.1. + See KAFKA-19576 for details. +
  • diff --git a/gradle.properties b/gradle.properties index 52bf115819de5..9ec5bd79799b3 100644 --- a/gradle.properties +++ b/gradle.properties @@ -23,7 +23,7 @@ group=org.apache.kafka # - streams/quickstart/pom.xml # - streams/quickstart/java/src/main/resources/archetype-resources/pom.xml # - streams/quickstart/java/pom.xml -version=4.1.0-SNAPSHOT +version=4.1.1-SNAPSHOT scalaVersion=2.13.16 # Adding swaggerVersion in gradle.properties to have a single version in place for swagger swaggerVersion=2.2.25 diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index cf519c4af8f07..10173814e53d4 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -57,6 +57,8 @@ versions += [ caffeine: "3.2.0", bndlib: "7.1.0", checkstyle: project.hasProperty('checkstyleVersion') ? checkstyleVersion : "10.20.2", + commonsBeanutils: "1.11.0", + commonsLang: "3.18.0", commonsValidator: "1.9.0", classgraph: "4.8.179", gradle: "8.14.1", @@ -122,6 +124,7 @@ versions += [ slf4j: "1.7.36", snappy: "1.1.10.7", spotbugs: "4.8.6", + mockOAuth2Server: "2.2.1", zinc: "1.9.2", // When updating the zstd version, please do as well in docker/native/native-image-configs/resource-config.json // Also make sure the compression levels in org.apache.kafka.common.record.CompressionType are still valid @@ -147,6 +150,8 @@ libs += [ bndlib:"biz.aQute.bnd:biz.aQute.bndlib:$versions.bndlib", caffeine: "com.github.ben-manes.caffeine:caffeine:$versions.caffeine", classgraph: "io.github.classgraph:classgraph:$versions.classgraph", + commonsBeanutils: "commons-beanutils:commons-beanutils:$versions.commonsBeanutils", + commonsLang: "org.apache.commons:commons-lang3:$versions.commonsLang", commonsValidator: "commons-validator:commons-validator:$versions.commonsValidator", jacksonAnnotations: "com.fasterxml.jackson.core:jackson-annotations:$versions.jackson", jacksonDatabind: "com.fasterxml.jackson.core:jackson-databind:$versions.jackson", @@ -222,6 +227,7 @@ libs += [ snappy: "org.xerial.snappy:snappy-java:$versions.snappy", spotbugs: "com.github.spotbugs:spotbugs-annotations:$versions.spotbugs", swaggerAnnotations: "io.swagger.core.v3:swagger-annotations:$swaggerVersion", + mockOAuth2Server: "no.nav.security:mock-oauth2-server:$versions.mockOAuth2Server", jfreechart: "jfreechart:jfreechart:$versions.jfreechart", mavenArtifact: "org.apache.maven:maven-artifact:$versions.mavenArtifact", zstd: "com.github.luben:zstd-jni:$versions.zstd", diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java index 5c72eba50713c..c4650145ebc14 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java @@ -207,6 +207,11 @@ public class GroupCoordinatorConfig { ConsumerGroupMigrationPolicy.DOWNGRADE + ": only downgrade from consumer group to classic group is enabled, " + ConsumerGroupMigrationPolicy.DISABLED + ": neither upgrade nor downgrade is enabled."; + public static final String CONSUMER_GROUP_REGEX_REFRESH_INTERVAL_MS_CONFIG = "group.consumer.regex.refresh.interval.ms"; + public static final String CONSUMER_GROUP_REGEX_REFRESH_INTERVAL_MS_DOC = "The interval at which the group coordinator will refresh " + + "the topics matching the group subscribed regexes. This is only applicable to consumer groups using the consumer group protocol. "; + public static final int CONSUMER_GROUP_REGEX_REFRESH_INTERVAL_MS_DEFAULT = 10 * 60 * 1000; // 10 minutes + /// /// Share group configs /// @@ -318,6 +323,8 @@ public class GroupCoordinatorConfig { .define(CONSUMER_GROUP_MAX_SIZE_CONFIG, INT, CONSUMER_GROUP_MAX_SIZE_DEFAULT, atLeast(1), MEDIUM, CONSUMER_GROUP_MAX_SIZE_DOC) .define(CONSUMER_GROUP_ASSIGNORS_CONFIG, LIST, CONSUMER_GROUP_ASSIGNORS_DEFAULT, null, MEDIUM, CONSUMER_GROUP_ASSIGNORS_DOC) .define(CONSUMER_GROUP_MIGRATION_POLICY_CONFIG, STRING, CONSUMER_GROUP_MIGRATION_POLICY_DEFAULT, ConfigDef.CaseInsensitiveValidString.in(Utils.enumOptions(ConsumerGroupMigrationPolicy.class)), MEDIUM, CONSUMER_GROUP_MIGRATION_POLICY_DOC) + // Interval config used for testing purposes. + .defineInternal(CONSUMER_GROUP_REGEX_REFRESH_INTERVAL_MS_CONFIG, INT, CONSUMER_GROUP_REGEX_REFRESH_INTERVAL_MS_DEFAULT, atLeast(10 * 1000), MEDIUM, CONSUMER_GROUP_REGEX_REFRESH_INTERVAL_MS_DOC) // Share group configs .define(SHARE_GROUP_SESSION_TIMEOUT_MS_CONFIG, INT, SHARE_GROUP_SESSION_TIMEOUT_MS_DEFAULT, atLeast(1), MEDIUM, SHARE_GROUP_SESSION_TIMEOUT_MS_DOC) @@ -370,6 +377,7 @@ public class GroupCoordinatorConfig { private final int consumerGroupMaxSessionTimeoutMs; private final int consumerGroupMinHeartbeatIntervalMs; private final int consumerGroupMaxHeartbeatIntervalMs; + private final int consumerGroupRegexRefreshIntervalMs; // Share group configurations private final int shareGroupMaxSize; private final int shareGroupSessionTimeoutMs; @@ -419,6 +427,7 @@ public GroupCoordinatorConfig(AbstractConfig config) { this.consumerGroupMaxSessionTimeoutMs = config.getInt(GroupCoordinatorConfig.CONSUMER_GROUP_MAX_SESSION_TIMEOUT_MS_CONFIG); this.consumerGroupMinHeartbeatIntervalMs = config.getInt(GroupCoordinatorConfig.CONSUMER_GROUP_MIN_HEARTBEAT_INTERVAL_MS_CONFIG); this.consumerGroupMaxHeartbeatIntervalMs = config.getInt(GroupCoordinatorConfig.CONSUMER_GROUP_MAX_HEARTBEAT_INTERVAL_MS_CONFIG); + this.consumerGroupRegexRefreshIntervalMs = config.getInt(GroupCoordinatorConfig.CONSUMER_GROUP_REGEX_REFRESH_INTERVAL_MS_CONFIG); // Share group configurations this.shareGroupSessionTimeoutMs = config.getInt(GroupCoordinatorConfig.SHARE_GROUP_SESSION_TIMEOUT_MS_CONFIG); this.shareGroupMinSessionTimeoutMs = config.getInt(GroupCoordinatorConfig.SHARE_GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG); @@ -810,6 +819,13 @@ public int consumerGroupMaxHeartbeatIntervalMs() { return consumerGroupMaxHeartbeatIntervalMs; } + /** + * The consumer group regex batch refresh max interval in milliseconds. + */ + public int consumerGroupRegexRefreshIntervalMs() { + return consumerGroupRegexRefreshIntervalMs; + } + /** * The share group session timeout in milliseconds. */ diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java index 7871ae6002947..4e0e03265a6ce 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java @@ -72,6 +72,7 @@ import org.apache.kafka.common.requests.DescribeGroupsRequest; import org.apache.kafka.common.requests.DescribeShareGroupOffsetsRequest; import org.apache.kafka.common.requests.OffsetCommitRequest; +import org.apache.kafka.common.requests.OffsetFetchResponse; import org.apache.kafka.common.requests.ShareGroupDescribeRequest; import org.apache.kafka.common.requests.ShareGroupHeartbeatRequest; import org.apache.kafka.common.requests.StreamsGroupDescribeRequest; @@ -95,6 +96,7 @@ import org.apache.kafka.coordinator.group.streams.StreamsGroupHeartbeatResult; import org.apache.kafka.image.MetadataDelta; import org.apache.kafka.image.MetadataImage; +import org.apache.kafka.image.TopicImage; import org.apache.kafka.server.authorizer.AuthorizableRequestContext; import org.apache.kafka.server.authorizer.Authorizer; import org.apache.kafka.server.record.BrokerCompressionType; @@ -141,6 +143,7 @@ import static org.apache.kafka.coordinator.group.Utils.throwIfEmptyString; import static org.apache.kafka.coordinator.group.Utils.throwIfNotEmptyCollection; import static org.apache.kafka.coordinator.group.Utils.throwIfNotNull; +import static org.apache.kafka.coordinator.group.Utils.throwIfNotNullOrEmpty; import static org.apache.kafka.coordinator.group.Utils.throwIfNull; /** @@ -540,6 +543,26 @@ private static void throwIfStreamsGroupHeartbeatRequestIsInvalid( } } + /** + * Validates the request. Specifically, throws if any not-yet-supported features are used. + * + * @param request The request to validate. + * @throws InvalidRequestException if the request is not valid. + */ + private static void throwIfStreamsGroupHeartbeatRequestIsUsingUnsupportedFeatures( + StreamsGroupHeartbeatRequestData request + ) throws InvalidRequestException { + throwIfNotNull(request.instanceId(), "Static membership is not yet supported."); + throwIfNotNull(request.taskOffsets(), "TaskOffsets are not supported yet."); + throwIfNotNull(request.taskEndOffsets(), "TaskEndOffsets are not supported yet."); + throwIfNotNullOrEmpty(request.warmupTasks(), "WarmupTasks are not supported yet."); + if (request.topology() != null) { + for (StreamsGroupHeartbeatRequestData.Subtopology subtopology : request.topology().subtopologies()) { + throwIfNotEmptyCollection(subtopology.sourceTopicRegex(), "Regular expressions for source topics are not supported yet."); + } + } + } + /** * See * {@link GroupCoordinator#streamsGroupHeartbeat(AuthorizableRequestContext, StreamsGroupHeartbeatRequestData)}. @@ -559,6 +582,7 @@ public CompletableFuture streamsGroupHeartbeat( } try { + throwIfStreamsGroupHeartbeatRequestIsUsingUnsupportedFeatures(request); throwIfStreamsGroupHeartbeatRequestIsInvalid(request); } catch (Throwable ex) { ApiError apiError = ApiError.fromThrowable(ex); @@ -1076,7 +1100,7 @@ public CompletableFuture> } else { futures.add(CompletableFuture.completedFuture(List.of( new ConsumerGroupDescribeResponseData.DescribedGroup() - .setGroupId(null) + .setGroupId("") .setErrorCode(Errors.INVALID_GROUP_ID.code()) ))); } @@ -1128,7 +1152,7 @@ public CompletableFuture> } else { futures.add(CompletableFuture.completedFuture(List.of( new StreamsGroupDescribeResponseData.DescribedGroup() - .setGroupId(null) + .setGroupId("") .setErrorCode(Errors.INVALID_GROUP_ID.code()) ))); } @@ -1180,7 +1204,7 @@ public CompletableFuture> shareGroupDescribe( } else { futures.add(CompletableFuture.completedFuture(List.of( new ShareGroupDescribeResponseData.DescribedGroup() - .setGroupId(null) + .setGroupId("") .setErrorCode(Errors.INVALID_GROUP_ID.code()) ))); } @@ -1262,7 +1286,7 @@ public CompletableFuture> descri if (groupId == null) { futures.add(CompletableFuture.completedFuture(List.of( new DescribeGroupsResponseData.DescribedGroup() - .setGroupId(null) + .setGroupId("") .setErrorCode(Errors.INVALID_GROUP_ID.code()) ))); } else { @@ -1529,18 +1553,20 @@ public CompletableFuture fetch boolean requireStable ) { if (!isActive.get()) { - return CompletableFuture.completedFuture(new OffsetFetchResponseData.OffsetFetchResponseGroup() - .setGroupId(request.groupId()) - .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code()) - ); + return CompletableFuture.completedFuture(OffsetFetchResponse.groupError( + request, + Errors.COORDINATOR_NOT_AVAILABLE, + context.requestVersion() + )); } // For backwards compatibility, we support fetch commits for the empty group id. if (request.groupId() == null) { - return CompletableFuture.completedFuture(new OffsetFetchResponseData.OffsetFetchResponseGroup() - .setGroupId(request.groupId()) - .setErrorCode(Errors.INVALID_GROUP_ID.code()) - ); + return CompletableFuture.completedFuture(OffsetFetchResponse.groupError( + request, + Errors.INVALID_GROUP_ID, + context.requestVersion() + )); } // The require stable flag when set tells the broker to hold on returning unstable @@ -1562,6 +1588,7 @@ public CompletableFuture fetch ) ).exceptionally(exception -> handleOffsetFetchException( "fetch-offsets", + context, request, exception )); @@ -1584,18 +1611,20 @@ public CompletableFuture fetch boolean requireStable ) { if (!isActive.get()) { - return CompletableFuture.completedFuture(new OffsetFetchResponseData.OffsetFetchResponseGroup() - .setGroupId(request.groupId()) - .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code()) - ); + return CompletableFuture.completedFuture(OffsetFetchResponse.groupError( + request, + Errors.COORDINATOR_NOT_AVAILABLE, + context.requestVersion() + )); } // For backwards compatibility, we support fetch commits for the empty group id. if (request.groupId() == null) { - return CompletableFuture.completedFuture(new OffsetFetchResponseData.OffsetFetchResponseGroup() - .setGroupId(request.groupId()) - .setErrorCode(Errors.INVALID_GROUP_ID.code()) - ); + return CompletableFuture.completedFuture(OffsetFetchResponse.groupError( + request, + Errors.INVALID_GROUP_ID, + context.requestVersion() + )); } // The require stable flag when set tells the broker to hold on returning unstable @@ -1617,6 +1646,7 @@ public CompletableFuture fetch ) ).exceptionally(exception -> handleOffsetFetchException( "fetch-all-offsets", + context, request, exception )); @@ -2108,10 +2138,18 @@ public void onPartitionsDeleted( ).get(); // At this point the metadata will not have been updated - // with the deleted topics. - Set topicIds = topicPartitions.stream() - .map(tp -> metadataImage.topics().getTopic(tp.topic()).id()) - .collect(Collectors.toSet()); + // with the deleted topics, but we must guard against it. + if (metadataImage == null || metadataImage.equals(MetadataImage.EMPTY)) { + return; + } + + Set topicIds = new HashSet<>(); + for (TopicPartition tp : topicPartitions) { + TopicImage image = metadataImage.topics().getTopic(tp.topic()); + if (image != null) { + topicIds.add(image.id()); + } + } CompletableFuture.allOf( FutureUtils.mapExceptionally( @@ -2244,12 +2282,14 @@ private static boolean isGroupIdNotEmpty(String groupId) { * The handler also handles and logs unexpected errors. * * @param operationName The name of the operation. + * @param context The request context. * @param request The OffsetFetchRequestGroup request. * @param exception The exception to handle. * @return The OffsetFetchRequestGroup response. */ private OffsetFetchResponseData.OffsetFetchResponseGroup handleOffsetFetchException( String operationName, + AuthorizableRequestContext context, OffsetFetchRequestData.OffsetFetchRequestGroup request, Throwable exception ) { @@ -2268,18 +2308,22 @@ private OffsetFetchResponseData.OffsetFetchResponseGroup handleOffsetFetchExcept // NOT_ENOUGH_REPLICAS and REQUEST_TIMED_OUT to COORDINATOR_NOT_AVAILABLE, // COORDINATOR_NOT_AVAILABLE is also not handled by consumers on versions prior to // 3.9. - return new OffsetFetchResponseData.OffsetFetchResponseGroup() - .setGroupId(request.groupId()) - .setErrorCode(Errors.NOT_COORDINATOR.code()); + return OffsetFetchResponse.groupError( + request, + Errors.NOT_COORDINATOR, + context.requestVersion() + ); default: return handleOperationException( operationName, request, exception, - (error, __) -> new OffsetFetchResponseData.OffsetFetchResponseGroup() - .setGroupId(request.groupId()) - .setErrorCode(error.code()), + (error, __) -> OffsetFetchResponse.groupError( + request, + error, + context.requestVersion() + ), log ); } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java index 3d293a1ddca4b..080fa265221ff 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java @@ -248,6 +248,7 @@ import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.STABLE; import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.CLASSIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME; import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.CONSUMER_GROUP_REBALANCES_SENSOR_NAME; +import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.SHARE_GROUP_REBALANCES_SENSOR_NAME; import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.STREAMS_GROUP_REBALANCES_SENSOR_NAME; import static org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroupMember.hasAssignedPartitionsChanged; import static org.apache.kafka.coordinator.group.streams.StreamsCoordinatorRecordHelpers.convertToStreamsGroupTopologyRecord; @@ -402,8 +403,10 @@ GroupMetadataManager build() { /** * The minimum amount of time between two consecutive refreshes of * the regular expressions within a single group. + * + * Package private for setting the lower limit of the refresh interval. */ - private static final long REGEX_BATCH_REFRESH_INTERVAL_MS = 10_000L; + static final long REGEX_BATCH_REFRESH_MIN_INTERVAL_MS = 10_000L; /** * The log context. @@ -2608,6 +2611,7 @@ private CoordinatorResult records ) { + final long currentTimeMs = time.milliseconds(); String groupId = group.groupId(); String memberId = updatedMember.memberId(); String oldSubscribedTopicRegex = member.subscribedTopicRegex(); @@ -3113,11 +3118,12 @@ private boolean maybeUpdateRegularExpressions( } // Conditions to trigger a refresh: - // 0. The group is subscribed to regular expressions. - // 1. There is no ongoing refresh for the group. - // 2. The last refresh is older than 10s. - // 3. The group has unresolved regular expressions. - // 4. The metadata image has new topics. + // 0. The group is subscribed to regular expressions. + // 1. There is no ongoing refresh for the group. + // 2. The last refresh is older than 10s. + // 3.1 The group has unresolved regular expressions. + // 3.2 Or the metadata image has new topics. + // 3.3 Or the last refresh is older than the batch refresh max interval. // 0. The group is subscribed to regular expressions. We also take the one // that the current may have just introduced. @@ -3134,11 +3140,11 @@ private boolean maybeUpdateRegularExpressions( // 2. The last refresh is older than 10s. If the group does not have any regular // expressions but the current member just brought a new one, we should continue. long lastRefreshTimeMs = group.lastResolvedRegularExpressionRefreshTimeMs(); - if (time.milliseconds() <= lastRefreshTimeMs + REGEX_BATCH_REFRESH_INTERVAL_MS) { + if (currentTimeMs <= lastRefreshTimeMs + REGEX_BATCH_REFRESH_MIN_INTERVAL_MS) { return bumpGroupEpoch; } - // 3. The group has unresolved regular expressions. + // 3.1 The group has unresolved regular expressions. Map subscribedRegularExpressions = new HashMap<>(group.subscribedRegularExpressions()); if (isNotEmpty(oldSubscribedTopicRegex)) { subscribedRegularExpressions.compute(oldSubscribedTopicRegex, Utils::decValue); @@ -3149,9 +3155,12 @@ private boolean maybeUpdateRegularExpressions( requireRefresh |= subscribedRegularExpressions.size() != group.numResolvedRegularExpressions(); - // 4. The metadata has new topics that we must consider. + // 3.2 The metadata has new topics that we must consider. requireRefresh |= group.lastResolvedRegularExpressionVersion() < lastMetadataImageWithNewTopics; + // 3.3 The last refresh is older than the batch refresh max interval. + requireRefresh |= currentTimeMs > lastRefreshTimeMs + config.consumerGroupRegexRefreshIntervalMs(); + if (requireRefresh && !subscribedRegularExpressions.isEmpty()) { Set regexes = Collections.unmodifiableSet(subscribedRegularExpressions.keySet()); executor.schedule( @@ -5349,16 +5358,26 @@ public void replay( String groupId = key.groupId(); String memberId = key.memberId(); - ShareGroup shareGroup = getOrMaybeCreatePersistedShareGroup(groupId, value != null); + ShareGroup shareGroup; + ShareGroupMember oldMember; + try { + shareGroup = getOrMaybeCreatePersistedShareGroup(groupId, value != null); + oldMember = shareGroup.getOrMaybeCreateMember(memberId, value != null); + } catch (GroupIdNotFoundException ex) { + log.debug("ShareGroupMemberMetadata tombstone without group - {}", groupId, ex); + return; + } catch (UnknownMemberIdException ex) { + log.debug("ShareGroupMemberMetadata tombstone for groupId - {} without member - {}", groupId, memberId, ex); + return; + } + Set oldSubscribedTopicNames = new HashSet<>(shareGroup.subscribedTopicNames().keySet()); if (value != null) { - ShareGroupMember oldMember = shareGroup.getOrMaybeCreateMember(memberId, true); shareGroup.updateMember(new ShareGroupMember.Builder(oldMember) .updateWith(value) .build()); } else { - ShareGroupMember oldMember = shareGroup.getOrMaybeCreateMember(memberId, false); if (oldMember.memberEpoch() != LEAVE_GROUP_MEMBER_EPOCH) { throw new IllegalStateException("Received a tombstone record to delete member " + memberId + " with invalid leave group epoch."); @@ -5387,12 +5406,18 @@ public void replay( ) { String groupId = key.groupId(); + ShareGroup shareGroup; + try { + shareGroup = getOrMaybeCreatePersistedShareGroup(groupId, value != null); + } catch (GroupIdNotFoundException ex) { + log.debug("ShareGroupMetadata tombstone without group - {}", groupId, ex); + return; + } + if (value != null) { - ShareGroup shareGroup = getOrMaybeCreatePersistedShareGroup(groupId, true); shareGroup.setGroupEpoch(value.epoch()); shareGroup.setMetadataHash(value.metadataHash()); } else { - ShareGroup shareGroup = getOrMaybeCreatePersistedShareGroup(groupId, false); if (!shareGroup.members().isEmpty()) { throw new IllegalStateException("Received a tombstone record to delete group " + groupId + " but the group still has " + shareGroup.members().size() + " members."); @@ -5584,7 +5609,14 @@ public void replay( ) { String groupId = key.groupId(); String memberId = key.memberId(); - ShareGroup group = getOrMaybeCreatePersistedShareGroup(groupId, false); + + ShareGroup group; + try { + group = getOrMaybeCreatePersistedShareGroup(groupId, value != null); + } catch (GroupIdNotFoundException ex) { + log.debug("ShareGroupTargetAssignmentMember tombstone without group - {}", groupId, ex); + return; + } if (value != null) { group.updateTargetAssignment(memberId, Assignment.fromRecord(value)); @@ -5606,7 +5638,14 @@ public void replay( ShareGroupTargetAssignmentMetadataValue value ) { String groupId = key.groupId(); - ShareGroup group = getOrMaybeCreatePersistedShareGroup(groupId, false); + + ShareGroup group; + try { + group = getOrMaybeCreatePersistedShareGroup(groupId, value != null); + } catch (GroupIdNotFoundException ex) { + log.debug("ShareGroupTargetAssignmentMetadata tombstone without group - {}", groupId, ex); + return; + } if (value != null) { group.setTargetAssignmentEpoch(value.assignmentEpoch()); @@ -5633,20 +5672,31 @@ public void replay( String groupId = key.groupId(); String memberId = key.memberId(); - ShareGroup group = getOrMaybeCreatePersistedShareGroup(groupId, false); - ShareGroupMember oldMember = group.getOrMaybeCreateMember(memberId, false); + ShareGroup group; + ShareGroupMember oldMember; + + try { + group = getOrMaybeCreatePersistedShareGroup(groupId, value != null); + oldMember = group.getOrMaybeCreateMember(memberId, value != null); + } catch (GroupIdNotFoundException ex) { + log.debug("ShareGroupCurrentMemberAssignment tombstone without group - {}", groupId, ex); + return; + } catch (UnknownMemberIdException ex) { + log.debug("ShareGroupCurrentMemberAssignment tombstone for groupId - {} without member - {}", groupId, memberId, ex); + return; + } if (value != null) { ShareGroupMember newMember = new ShareGroupMember.Builder(oldMember) - .updateWith(value) - .build(); + .updateWith(value) + .build(); group.updateMember(newMember); } else { ShareGroupMember newMember = new ShareGroupMember.Builder(oldMember) - .setMemberEpoch(LEAVE_GROUP_MEMBER_EPOCH) - .setPreviousMemberEpoch(LEAVE_GROUP_MEMBER_EPOCH) - .setAssignedPartitions(Map.of()) - .build(); + .setMemberEpoch(LEAVE_GROUP_MEMBER_EPOCH) + .setPreviousMemberEpoch(LEAVE_GROUP_MEMBER_EPOCH) + .setAssignedPartitions(Map.of()) + .build(); group.updateMember(newMember); } } @@ -5664,12 +5714,16 @@ public void replay( ) { String groupId = key.groupId(); - getOrMaybeCreatePersistedShareGroup(groupId, false); - // Update timeline structures with info about initialized/deleted topics. + try { + getOrMaybeCreatePersistedShareGroup(groupId, value != null); + } catch (GroupIdNotFoundException ex) { + // Ignore tombstone if group not found. + log.debug("ShareGroupStatePartitionMetadata tombstone for non-existent share group {}", groupId, ex); + } + if (value == null) { - // Tombstone! - shareGroupStatePartitionMetadata.remove(groupId); + shareGroupStatePartitionMetadata.remove(groupId); // Should not throw any exceptions. } else { long timestamp = time.milliseconds(); ShareGroupStatePartitionMetadataInfo info = new ShareGroupStatePartitionMetadataInfo( @@ -7968,19 +8022,21 @@ public Optional shareGroupBuildPartitionDeleteR // a retry for the same is possible. Since this is part of an admin operation // retrying delete should not pose issues related to // performance. Also, the share coordinator is idempotent on delete partitions. - Map deletingTopics = shareGroupStatePartitionMetadata.get(shareGroupId).deletingTopics().stream() - .map(tid -> { - TopicImage image = metadataImage.topics().getTopic(tid); - return Map.entry(tid, new InitMapValue(image.name(), image.partitions().keySet(), -1)); - }) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + Set deletingCurrent = shareGroupStatePartitionMetadata.get(shareGroupId).deletingTopics(); + if (metadataImage != null && !metadataImage.equals(MetadataImage.EMPTY)) { + Map deletingTopics = deletingCurrent.stream() + .map(tid -> metadataImage.topics().getTopic(tid)) + .filter(Objects::nonNull) + .map(image -> Map.entry(image.id(), new InitMapValue(image.name(), image.partitions().keySet(), -1))) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - if (!deletingTopics.isEmpty()) { - log.info("Existing deleting entries found in share group {} - {}", shareGroupId, deletingTopics); - deleteCandidates = combineInitMaps(deleteCandidates, deletingTopics); + if (!deletingTopics.isEmpty()) { + log.info("Existing deleting entries found in share group {} - {}", shareGroupId, deletingTopics); + deleteCandidates = combineInitMaps(deleteCandidates, deletingTopics); + } } - if (deleteCandidates.isEmpty()) { + if (deleteCandidates.isEmpty() && deletingCurrent.isEmpty()) { return Optional.empty(); } @@ -8004,6 +8060,10 @@ public Optional shareGroupBuildPartitionDeleteR attachTopicName(deleteCandidates.keySet()) )); + if (topicDataList.isEmpty()) { + return Optional.empty(); + } + return Optional.of(new DeleteShareGroupStateParameters.Builder() .setGroupTopicPartitionData(new GroupTopicPartitionData.Builder() .setGroupId(shareGroupId) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Utils.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Utils.java index d614123d2a72b..02b0ed28e6ec2 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Utils.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Utils.java @@ -280,6 +280,22 @@ static void throwIfNotEmptyCollection( } } + /** + * Throws an InvalidRequestException if the value is not null and non-empty. + * + * @param value The value. + * @param error The error message. + * @throws InvalidRequestException + */ + static void throwIfNotNullOrEmpty( + Collection value, + String error + ) throws InvalidRequestException { + if (value != null && !value.isEmpty()) { + throw new InvalidRequestException(error); + } + } + /** * Throws an InvalidRequestException if the value is non-null. * diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetrics.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetrics.java index dd21570b654f0..9d05727d2e974 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetrics.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetrics.java @@ -297,14 +297,12 @@ public GroupCoordinatorMetrics(MetricsRegistry registry, Metrics metrics) { Sensor shareGroupRebalanceSensor = metrics.sensor(SHARE_GROUP_REBALANCES_SENSOR_NAME); shareGroupRebalanceSensor.add(new Meter( - metrics.metricName("rebalance-rate", + metrics.metricName("share-group-rebalance-rate", METRICS_GROUP, - "The rate of share group rebalances", - SHARE_GROUP_PROTOCOL_TAG, Group.GroupType.SHARE.toString()), - metrics.metricName("rebalance-count", + "The rate of share group rebalances"), + metrics.metricName("share-group-rebalance-count", METRICS_GROUP, - "The total number of share group rebalances", - SHARE_GROUP_PROTOCOL_TAG, Group.GroupType.SHARE.toString()))); + "The total number of share group rebalances"))); Sensor streamsGroupRebalanceSensor = metrics.sensor(STREAMS_GROUP_REBALANCES_SENSOR_NAME); streamsGroupRebalanceSensor.add(new Meter( diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java index 7c5c7a5b8daad..84bd38ae60d72 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java @@ -198,6 +198,7 @@ public void testConfigs() { configs.put(GroupCoordinatorConfig.CONSUMER_GROUP_MAX_SESSION_TIMEOUT_MS_CONFIG, 666); configs.put(GroupCoordinatorConfig.CONSUMER_GROUP_MIN_HEARTBEAT_INTERVAL_MS_CONFIG, 111); configs.put(GroupCoordinatorConfig.CONSUMER_GROUP_MAX_HEARTBEAT_INTERVAL_MS_CONFIG, 222); + configs.put(GroupCoordinatorConfig.CONSUMER_GROUP_REGEX_REFRESH_INTERVAL_MS_CONFIG, 15 * 60 * 1000); GroupCoordinatorConfig config = createConfig(configs); @@ -226,6 +227,7 @@ public void testConfigs() { assertEquals(666, config.consumerGroupMaxSessionTimeoutMs()); assertEquals(111, config.consumerGroupMinHeartbeatIntervalMs()); assertEquals(222, config.consumerGroupMaxHeartbeatIntervalMs()); + assertEquals(15 * 60 * 1000, config.consumerGroupRegexRefreshIntervalMs()); } @Test diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java index 026af24f14109..01c87696053a2 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java @@ -574,6 +574,91 @@ public void testStreamsGroupHeartbeatWithException( future.get(5, TimeUnit.SECONDS) ); } + @Test + public void testStreamsGroupHeartbeatFailsForUnsupportedFeatures() throws Exception { + + GroupCoordinatorService service = new GroupCoordinatorServiceBuilder() + .setConfig(createConfig()) + .setRuntime(mockRuntime()) + .build(true); + + AuthorizableRequestContext context = mock(AuthorizableRequestContext.class); + when(context.requestVersion()).thenReturn((int) ApiKeys.STREAMS_GROUP_HEARTBEAT.latestVersion()); + + assertEquals( + new StreamsGroupHeartbeatResult( + new StreamsGroupHeartbeatResponseData() + .setErrorCode(Errors.INVALID_REQUEST.code()) + .setErrorMessage("Static membership is not yet supported."), + Map.of() + ), + service.streamsGroupHeartbeat( + context, + new StreamsGroupHeartbeatRequestData() + .setInstanceId(Uuid.randomUuid().toString()) + ).get(5, TimeUnit.SECONDS) + ); + + assertEquals( + new StreamsGroupHeartbeatResult( + new StreamsGroupHeartbeatResponseData() + .setErrorCode(Errors.INVALID_REQUEST.code()) + .setErrorMessage("TaskOffsets are not supported yet."), + Map.of() + ), + service.streamsGroupHeartbeat( + context, + new StreamsGroupHeartbeatRequestData() + .setTaskOffsets(List.of(new StreamsGroupHeartbeatRequestData.TaskOffset())) + ).get(5, TimeUnit.SECONDS) + ); + + assertEquals( + new StreamsGroupHeartbeatResult( + new StreamsGroupHeartbeatResponseData() + .setErrorCode(Errors.INVALID_REQUEST.code()) + .setErrorMessage("TaskEndOffsets are not supported yet."), + Map.of() + ), + service.streamsGroupHeartbeat( + context, + new StreamsGroupHeartbeatRequestData() + .setTaskEndOffsets(List.of(new StreamsGroupHeartbeatRequestData.TaskOffset())) + ).get(5, TimeUnit.SECONDS) + ); + + assertEquals( + new StreamsGroupHeartbeatResult( + new StreamsGroupHeartbeatResponseData() + .setErrorCode(Errors.INVALID_REQUEST.code()) + .setErrorMessage("WarmupTasks are not supported yet."), + Map.of() + ), + service.streamsGroupHeartbeat( + context, + new StreamsGroupHeartbeatRequestData() + .setWarmupTasks(List.of(new StreamsGroupHeartbeatRequestData.TaskIds())) + ).get(5, TimeUnit.SECONDS) + ); + + assertEquals( + new StreamsGroupHeartbeatResult( + new StreamsGroupHeartbeatResponseData() + .setErrorCode(Errors.INVALID_REQUEST.code()) + .setErrorMessage("Regular expressions for source topics are not supported yet."), + Map.of() + ), + service.streamsGroupHeartbeat( + context, + new StreamsGroupHeartbeatRequestData() + .setTopology(new StreamsGroupHeartbeatRequestData.Topology() + .setSubtopologies(List.of(new StreamsGroupHeartbeatRequestData.Subtopology() + .setSourceTopicRegex(List.of("foo.*")) + )) + ) + ).get(5, TimeUnit.SECONDS) + ); + } @SuppressWarnings("MethodLength") @Test @@ -584,7 +669,7 @@ public void testStreamsHeartbeatRequestValidation() throws ExecutionException, I .build(true); AuthorizableRequestContext context = mock(AuthorizableRequestContext.class); - when(context.requestVersion()).thenReturn((int) ApiKeys.SHARE_GROUP_HEARTBEAT.latestVersion()); + when(context.requestVersion()).thenReturn((int) ApiKeys.STREAMS_GROUP_HEARTBEAT.latestVersion()); String memberId = Uuid.randomUuid().toString(); @@ -743,24 +828,6 @@ public void testStreamsHeartbeatRequestValidation() throws ExecutionException, I ).get(5, TimeUnit.SECONDS) ); - // InstanceId must be non-empty if provided in all requests. - assertEquals( - new StreamsGroupHeartbeatResult( - new StreamsGroupHeartbeatResponseData() - .setErrorCode(Errors.INVALID_REQUEST.code()) - .setErrorMessage("InstanceId can't be empty."), - Map.of() - ), - service.streamsGroupHeartbeat( - context, - new StreamsGroupHeartbeatRequestData() - .setGroupId("foo") - .setMemberId(memberId) - .setMemberEpoch(1) - .setInstanceId("") - ).get(5, TimeUnit.SECONDS) - ); - // RackId must be non-empty if provided in all requests. assertEquals( new StreamsGroupHeartbeatResult( @@ -1558,7 +1625,7 @@ public void testDescribeGroupsInvalidGroupId() throws Exception { .setGroupId(""); List expectedDescribedGroups = Arrays.asList( new DescribeGroupsResponseData.DescribedGroup() - .setGroupId(null) + .setGroupId("") .setErrorCode(Errors.INVALID_GROUP_ID.code()), describedGroup ); @@ -1953,11 +2020,11 @@ public void testConsumerGroupDescribeInvalidGroupId() throws ExecutionException, service.startup(() -> partitionCount); ConsumerGroupDescribeResponseData.DescribedGroup describedGroup = new ConsumerGroupDescribeResponseData.DescribedGroup() - .setGroupId(null) + .setGroupId("") .setErrorCode(Errors.INVALID_GROUP_ID.code()); List expectedDescribedGroups = Arrays.asList( new ConsumerGroupDescribeResponseData.DescribedGroup() - .setGroupId(null) + .setGroupId("") .setErrorCode(Errors.INVALID_GROUP_ID.code()), describedGroup ); @@ -2091,11 +2158,11 @@ public void testStreamsGroupDescribeInvalidGroupId() throws ExecutionException, service.startup(() -> partitionCount); StreamsGroupDescribeResponseData.DescribedGroup describedGroup = new StreamsGroupDescribeResponseData.DescribedGroup() - .setGroupId(null) + .setGroupId("") .setErrorCode(Errors.INVALID_GROUP_ID.code()); List expectedDescribedGroups = Arrays.asList( new StreamsGroupDescribeResponseData.DescribedGroup() - .setGroupId(null) + .setGroupId("") .setErrorCode(Errors.INVALID_GROUP_ID.code()), describedGroup ); @@ -3435,11 +3502,11 @@ public void testShareGroupDescribeInvalidGroupId() throws ExecutionException, In service.startup(() -> partitionCount); ShareGroupDescribeResponseData.DescribedGroup describedGroup = new ShareGroupDescribeResponseData.DescribedGroup() - .setGroupId(null) + .setGroupId("") .setErrorCode(Errors.INVALID_GROUP_ID.code()); List expectedDescribedGroups = Arrays.asList( new ShareGroupDescribeResponseData.DescribedGroup() - .setGroupId(null) + .setGroupId("") .setErrorCode(Errors.INVALID_GROUP_ID.code()), describedGroup ); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java index dcc2a4ca5f3ae..bc5afd7704fcf 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java @@ -95,10 +95,18 @@ import org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataKey; import org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataValue; import org.apache.kafka.coordinator.group.generated.GroupMetadataValue; +import org.apache.kafka.coordinator.group.generated.ShareGroupCurrentMemberAssignmentKey; +import org.apache.kafka.coordinator.group.generated.ShareGroupCurrentMemberAssignmentValue; +import org.apache.kafka.coordinator.group.generated.ShareGroupMemberMetadataKey; +import org.apache.kafka.coordinator.group.generated.ShareGroupMemberMetadataValue; import org.apache.kafka.coordinator.group.generated.ShareGroupMetadataKey; import org.apache.kafka.coordinator.group.generated.ShareGroupMetadataValue; import org.apache.kafka.coordinator.group.generated.ShareGroupStatePartitionMetadataKey; import org.apache.kafka.coordinator.group.generated.ShareGroupStatePartitionMetadataValue; +import org.apache.kafka.coordinator.group.generated.ShareGroupTargetAssignmentMemberKey; +import org.apache.kafka.coordinator.group.generated.ShareGroupTargetAssignmentMemberValue; +import org.apache.kafka.coordinator.group.generated.ShareGroupTargetAssignmentMetadataKey; +import org.apache.kafka.coordinator.group.generated.ShareGroupTargetAssignmentMetadataValue; import org.apache.kafka.coordinator.group.generated.StreamsGroupMemberMetadataValue.Endpoint; import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue; import org.apache.kafka.coordinator.group.modern.Assignment; @@ -203,6 +211,8 @@ import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.STABLE; import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.CLASSIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME; import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.CONSUMER_GROUP_REBALANCES_SENSOR_NAME; +import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.SHARE_GROUP_REBALANCES_SENSOR_NAME; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotEquals; @@ -214,6 +224,7 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -19699,6 +19710,7 @@ public void testShareGroupHeartbeatOnConsumerGroup() { .setMemberId(Uuid.randomUuid().toString()) .setMemberEpoch(1) .setSubscribedTopicNames(List.of("foo", "bar")))); + verify(context.metrics, times(0)).record(SHARE_GROUP_REBALANCES_SENSOR_NAME); } @Test @@ -19785,6 +19797,7 @@ public void testShareGroupHeartbeatOnStreamsGroup() { .setGroupId(groupId) .setMemberEpoch(0) .setSubscribedTopicNames(List.of("foo", "bar")))); + verify(context.metrics, times(0)).record(SHARE_GROUP_REBALANCES_SENSOR_NAME); } @Test @@ -20804,7 +20817,7 @@ public void testConsumerGroupMemberJoinsWithRegexWithTopicAuthorizationFailure() .withMetadataHash(computeGroupHash(Map.of(fooTopicName, fooTopicHash)))) .build(); - // sleep for more than REGEX_BATCH_REFRESH_INTERVAL_MS + // sleep for more than REGEX_BATCH_REFRESH_MIN_INTERVAL_MS context.time.sleep(10001L); Map acls = new HashMap<>(); @@ -20887,7 +20900,7 @@ public void testConsumerGroupMemberJoinsWithRegexWithTopicAuthorizationFailure() context.processTasks() ); - // sleep for more than REGEX_BATCH_REFRESH_INTERVAL_MS + // sleep for more than REGEX_BATCH_REFRESH_MIN_INTERVAL_MS context.time.sleep(10001L); // Access to the bar topic is granted. @@ -20972,6 +20985,233 @@ memberId2, new MemberAssignmentImpl(mkAssignment( ); } + @Test + public void testConsumerGroupMemberJoinsRefreshTopicAuthorization() { + String groupId = "fooup"; + String memberId1 = Uuid.randomUuid().toString(); + String memberId2 = Uuid.randomUuid().toString(); + + Uuid fooTopicId = Uuid.randomUuid(); + Uuid barTopicId = Uuid.randomUuid(); + String fooTopicName = "foo"; + String barTopicName = "bar"; + + MetadataImage metadataImage = new MetadataImageBuilder() + .addTopic(fooTopicId, fooTopicName, 6) + .addTopic(barTopicId, barTopicName, 3) + .build(12345L); + long fooTopicHash = computeTopicHash(fooTopicName, metadataImage); + long barTopicHash = computeTopicHash(barTopicName, metadataImage); + + MockPartitionAssignor assignor = new MockPartitionAssignor("range"); + Authorizer authorizer = mock(Authorizer.class); + Plugin authorizerPlugin = Plugin.wrapInstance(authorizer, null, "authorizer.class.name"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor)) + .withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_REGEX_REFRESH_INTERVAL_MS_CONFIG, 60000) + .withMetadataImage(metadataImage) + .withAuthorizerPlugin(authorizerPlugin) + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) + .withMember(new ConsumerGroupMember.Builder(memberId1) + .setState(MemberState.STABLE) + .setMemberEpoch(10) + .setPreviousMemberEpoch(10) + .setClientId(DEFAULT_CLIENT_ID) + .setClientHost(DEFAULT_CLIENT_ADDRESS.toString()) + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(List.of("foo")) + .setServerAssignorName("range") + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2))) + .build()) + .withMember(new ConsumerGroupMember.Builder(memberId2) + .setState(MemberState.STABLE) + .setMemberEpoch(10) + .setPreviousMemberEpoch(10) + .setClientId(DEFAULT_CLIENT_ID) + .setClientHost(DEFAULT_CLIENT_ADDRESS.toString()) + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicRegex("foo*") + .setServerAssignorName("range") + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 3, 4, 5))) + .build()) + .withAssignment(memberId1, mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2))) + .withAssignment(memberId2, mkAssignment( + mkTopicAssignment(fooTopicId, 3, 4, 5))) + .withResolvedRegularExpression("foo*", new ResolvedRegularExpression( + Set.of(fooTopicName), 0L, 0L)) + .withAssignmentEpoch(10) + .withMetadataHash(computeGroupHash(Map.of(fooTopicName, fooTopicHash)))) + .build(); + + // sleep for more than REGEX_BATCH_REFRESH_MIN_INTERVAL_MS + context.time.sleep(10001L); + + Map acls = new HashMap<>(); + acls.put(fooTopicName, AuthorizationResult.ALLOWED); + acls.put(barTopicName, AuthorizationResult.DENIED); + when(authorizer.authorize(any(), any())).thenAnswer(invocation -> { + List actions = invocation.getArgument(1); + return actions.stream() + .map(action -> acls.getOrDefault(action.resourcePattern().name(), AuthorizationResult.DENIED)) + .collect(Collectors.toList()); + }); + + // Member 2 heartbeats with a different regular expression. + CoordinatorResult result1 = context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId2) + .setMemberEpoch(10) + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicRegex("foo*|bar*") + .setServerAssignor("range") + .setTopicPartitions(List.of()), + ApiKeys.CONSUMER_GROUP_HEARTBEAT.latestVersion() + ); + + assertResponseEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId2) + .setMemberEpoch(10) + .setHeartbeatIntervalMs(5000) + .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() + .setTopicPartitions(List.of( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(fooTopicId) + .setPartitions(List.of(3, 4, 5))))), + result1.response() + ); + + ConsumerGroupMember expectedMember2 = new ConsumerGroupMember.Builder(memberId2) + .setState(MemberState.STABLE) + .setMemberEpoch(10) + .setPreviousMemberEpoch(10) + .setClientId(DEFAULT_CLIENT_ID) + .setClientHost(DEFAULT_CLIENT_ADDRESS.toString()) + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicRegex("foo*|bar*") + .setServerAssignorName("range") + .build(); + + assertRecordsEquals( + List.of( + GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember2), + GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionTombstone(groupId, "foo*") + ), + result1.records() + ); + + // Execute pending tasks. + assertEquals( + List.of( + new MockCoordinatorExecutor.ExecutorResult<>( + groupId + "-regex", + new CoordinatorResult<>(List.of( + // The resolution of the new regex is persisted. + GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionRecord( + groupId, + "foo*|bar*", + new ResolvedRegularExpression( + Set.of("foo"), + 12345L, + context.time.milliseconds() + ) + ), + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, computeGroupHash(Map.of( + fooTopicName, fooTopicHash + ))) + )) + ) + ), + context.processTasks() + ); + + // sleep for more than REGEX_REFRESH_INTERVAL_MS + context.time.sleep(60001L); + + // Access to the bar topic is granted. + acls.put(barTopicName, AuthorizationResult.ALLOWED); + assignor.prepareGroupAssignment(new GroupAssignment(Map.of( + memberId1, new MemberAssignmentImpl(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2) + )), + memberId2, new MemberAssignmentImpl(mkAssignment( + mkTopicAssignment(fooTopicId, 3, 4, 5) + )) + ))); + + // Member 2 heartbeats again with the same regex. + CoordinatorResult result2 = context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId2) + .setMemberEpoch(10) + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicRegex("foo*|bar*") + .setServerAssignor("range") + .setTopicPartitions(List.of()), + ApiKeys.CONSUMER_GROUP_HEARTBEAT.latestVersion() + ); + + expectedMember2 = new ConsumerGroupMember.Builder(memberId2) + .setState(MemberState.STABLE) + .setMemberEpoch(11) + .setPreviousMemberEpoch(10) + .setClientId(DEFAULT_CLIENT_ID) + .setClientHost(DEFAULT_CLIENT_ADDRESS.toString()) + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicRegex("foo*|bar*") + .setServerAssignorName("range") + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 3, 4, 5))) + .build(); + + assertResponseEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId2) + .setMemberEpoch(11) + .setHeartbeatIntervalMs(5000) + .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() + .setTopicPartitions(List.of( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(fooTopicId) + .setPartitions(List.of(3, 4, 5))))), + result2.response() + ); + + assertRecordsEquals( + List.of( + GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochRecord(groupId, 11), + GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedMember2) + ), + result2.records() + ); + + // A regex refresh is triggered and the bar topic is included. + assertRecordsEquals( + List.of( + // The resolution of the new regex is persisted. + GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionRecord( + groupId, + "foo*|bar*", + new ResolvedRegularExpression( + Set.of("foo", "bar"), + 12345L, + context.time.milliseconds() + ) + ), + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 12, computeGroupHash(Map.of( + fooTopicName, fooTopicHash, + barTopicName, barTopicHash + ))) + ), + context.processTasks().get(0).result.records() + ); + } + @Test public void testResolvedRegularExpressionsRemovedWhenMembersLeaveOrFenced() { String groupId = "fooup"; @@ -22079,6 +22319,7 @@ public void testShareGroupHeartbeatInitializeOnPartitionUpdate() { ); assertEquals(Map.of(t1Uuid, Set.of(0, 1), t2Uuid, Set.of(0, 1)), context.groupMetadataManager.initializedShareGroupPartitions(groupId)); + verify(context.metrics, times(2)).record(SHARE_GROUP_REBALANCES_SENSOR_NAME); } @Test @@ -22188,6 +22429,7 @@ public void testShareGroupHeartbeatPersisterRequestWithInitializing() { 2, true ); + verify(context.metrics, times(2)).record(SHARE_GROUP_REBALANCES_SENSOR_NAME); } @Test @@ -22663,6 +22905,124 @@ t2Id, new InitMapValue(t2Name, Set.of(0), 1) assertEquals(Map.of(t1Id, new InitMapValue(t1Name, Set.of(0), 1)), GroupMetadataManager.combineInitMaps(m1, m2)); } + private static Stream shareGroupRecords() { + String groupId = "groupId"; + String memberId = Uuid.randomUuid().toString(); + + return Stream.of( + // Tombstones + CoordinatorRecord.tombstone( + new ShareGroupMemberMetadataKey() + .setGroupId(groupId) + .setMemberId(memberId) + ), + CoordinatorRecord.tombstone( + new ShareGroupMetadataKey() + .setGroupId(groupId) + ), + CoordinatorRecord.tombstone( + new ShareGroupTargetAssignmentMemberKey() + .setGroupId(groupId) + .setMemberId(memberId) + ), + CoordinatorRecord.tombstone( + new ShareGroupTargetAssignmentMetadataKey() + .setGroupId(groupId) + ), + CoordinatorRecord.tombstone( + new ShareGroupCurrentMemberAssignmentKey() + .setGroupId(groupId) + .setMemberId(memberId) + ), + CoordinatorRecord.tombstone( + new ShareGroupStatePartitionMetadataKey() + .setGroupId(groupId) + ), + // Data + CoordinatorRecord.record( + new ShareGroupMemberMetadataKey() + .setGroupId(groupId) + .setMemberId(memberId), + new ApiMessageAndVersion( + new ShareGroupMemberMetadataValue() + .setSubscribedTopicNames(List.of("tp1")), + (short) 10 + ) + ), + CoordinatorRecord.record( + new ShareGroupMetadataKey() + .setGroupId(groupId), + new ApiMessageAndVersion( + new ShareGroupMetadataValue() + .setEpoch(1) + .setMetadataHash(2L), + (short) 11 + ) + ), + CoordinatorRecord.record( + new ShareGroupTargetAssignmentMetadataKey() + .setGroupId(groupId), + new ApiMessageAndVersion( + new ShareGroupTargetAssignmentMetadataValue() + .setAssignmentEpoch(5), + (short) 12 + ) + ), + CoordinatorRecord.record( + new ShareGroupTargetAssignmentMemberKey() + .setGroupId(groupId) + .setMemberId(memberId), + new ApiMessageAndVersion(new ShareGroupTargetAssignmentMemberValue() + .setTopicPartitions(List.of( + new ShareGroupTargetAssignmentMemberValue.TopicPartition() + .setTopicId(Uuid.randomUuid()) + .setPartitions(List.of(0, 1, 2)) + )), + (short) 13 + ) + ), + CoordinatorRecord.record( + new ShareGroupCurrentMemberAssignmentKey() + .setGroupId(groupId) + .setMemberId(memberId), + new ApiMessageAndVersion(new ShareGroupCurrentMemberAssignmentValue() + .setAssignedPartitions(List.of( + new ShareGroupCurrentMemberAssignmentValue.TopicPartitions() + .setTopicId(Uuid.randomUuid()) + .setPartitions(List.of(0, 1, 2)) + ) + ) + .setMemberEpoch(5) + .setPreviousMemberEpoch(4) + .setState((byte) 0), + (short) 14 + ) + ), + CoordinatorRecord.record( + new ShareGroupStatePartitionMetadataKey() + .setGroupId(groupId), + new ApiMessageAndVersion(new ShareGroupStatePartitionMetadataValue() + .setInitializingTopics(List.of()) + .setInitializedTopics(List.of()) + .setDeletingTopics(List.of()), + (short) 15 + ) + ) + ); + } + + @ParameterizedTest + @MethodSource("shareGroupRecords") + public void testShareGroupRecordsNoExceptionOnReplay(CoordinatorRecord record) { + MockPartitionAssignor assignor = new MockPartitionAssignor("simple"); + assignor.prepareGroupAssignment(new GroupAssignment(Map.of())); + GroupMetadataManagerTestContext context = spy(new GroupMetadataManagerTestContext.Builder() + .withShareGroupAssignor(assignor) + .build()); + + assertDoesNotThrow(() -> context.replay(record)); + } + private static void checkJoinGroupResponse( JoinGroupResponseData expectedResponse, JoinGroupResponseData actualResponse, diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsTest.java index 3aa0a86172512..92a7cda873880 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsTest.java @@ -106,14 +106,8 @@ public void testMetricNames() { "group-count", GroupCoordinatorMetrics.METRICS_GROUP, Map.of("protocol", Group.GroupType.SHARE.toString())), - metrics.metricName( - "rebalance-rate", - GroupCoordinatorMetrics.METRICS_GROUP, - Map.of("protocol", Group.GroupType.SHARE.toString())), - metrics.metricName( - "rebalance-count", - GroupCoordinatorMetrics.METRICS_GROUP, - Map.of("protocol", Group.GroupType.SHARE.toString())), + metrics.metricName("share-group-rebalance-rate", GroupCoordinatorMetrics.METRICS_GROUP), + metrics.metricName("share-group-rebalance-count", GroupCoordinatorMetrics.METRICS_GROUP), metrics.metricName( "share-group-count", GroupCoordinatorMetrics.METRICS_GROUP, @@ -306,16 +300,14 @@ public void testGlobalSensors() { shard.record(SHARE_GROUP_REBALANCES_SENSOR_NAME, 50); assertMetricValue(metrics, metrics.metricName( - "rebalance-rate", + "share-group-rebalance-rate", GroupCoordinatorMetrics.METRICS_GROUP, - "The rate of share group rebalances", - "protocol", "share" + "The rate of share group rebalances" ), 5.0 / 3.0); assertMetricValue(metrics, metrics.metricName( - "rebalance-count", + "share-group-rebalance-count", GroupCoordinatorMetrics.METRICS_GROUP, - "The total number of share group rebalances", - "protocol", "share" + "The total number of share group rebalances" ), 50); shard.record(STREAMS_GROUP_REBALANCES_SENSOR_NAME, 50); diff --git a/licenses/mock-oauth2-server-MIT b/licenses/mock-oauth2-server-MIT new file mode 100644 index 0000000000000..ef1b1129b1035 --- /dev/null +++ b/licenses/mock-oauth2-server-MIT @@ -0,0 +1,21 @@ +# The MIT License + +Copyright 2025 NAV (Arbeids- og velferdsdirektoratet) - The Norwegian Labour and Welfare Administration + +Permission is hereby granted, free of charge, to any person obtaining +a copy of this software and associated documentation files (the "Software"), +to deal in the Software without restriction, including without limitation +the rights to use, copy, modify, merge, publish, distribute, sublicense, +and/or sell copies of the Software, and to permit persons to whom the +Software is furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included +in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. +IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, +DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR +OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE +USE OR OTHER DEALINGS IN THE SOFTWARE. \ No newline at end of file diff --git a/metadata/src/main/java/org/apache/kafka/controller/AclControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/AclControlManager.java index a2c8a717a603a..1a500e1a77e64 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/AclControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/AclControlManager.java @@ -184,6 +184,10 @@ ControllerResult> deleteAcls(List filter validateFilter(filter); AclDeleteResult result = deleteAclsForFilter(filter, records); results.add(result); + } catch (BoundedListTooLongException e) { + // we do not return partial results here because the fact that only a portion of the deletions + // succeeded can be easily missed due to response size. instead fail the entire response + throw new InvalidRequestException(e.getMessage(), e); } catch (Throwable e) { results.add(new AclDeleteResult(ApiError.fromThrowable(e).exception())); } @@ -199,13 +203,14 @@ AclDeleteResult deleteAclsForFilter(AclBindingFilter filter, StandardAcl acl = entry.getValue(); AclBinding binding = acl.toBinding(); if (filter.matches(binding)) { - deleted.add(new AclBindingDeleteResult(binding)); - records.add(new ApiMessageAndVersion( - new RemoveAccessControlEntryRecord().setId(id), (short) 0)); - if (records.size() > MAX_RECORDS_PER_USER_OP) { + // check size limitation first before adding additional records + if (records.size() >= MAX_RECORDS_PER_USER_OP) { throw new BoundedListTooLongException("Cannot remove more than " + MAX_RECORDS_PER_USER_OP + " acls in a single delete operation."); } + deleted.add(new AclBindingDeleteResult(binding)); + records.add(new ApiMessageAndVersion( + new RemoveAccessControlEntryRecord().setId(id), (short) 0)); } } return new AclDeleteResult(deleted); diff --git a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java index b367fda8114d5..d64695bd39b52 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java @@ -613,7 +613,7 @@ int getStaticallyConfiguredMinInsyncReplicas() { /** * Generate any configuration records that are needed to make it safe to enable ELR. - * Specifically, we need to remove all cluster-level configurations for min.insync.replicas, + * Specifically, we need to remove all broker-level configurations for min.insync.replicas, * and create a cluster-level configuration for min.insync.replicas. It is always safe to call * this function if ELR is already enabled; it will simply do nothing if the necessary * configurations already exist. diff --git a/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java b/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java index ea196dfc741b8..a1d35293bd1b8 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java +++ b/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java @@ -311,9 +311,10 @@ private boolean canElectLastKnownLeader() { topicId, partitionId, Arrays.toString(partition.lastKnownElr)); return false; } - if (isAcceptableLeader.test(partition.lastKnownElr[0])) { + if (!isAcceptableLeader.test(partition.lastKnownElr[0])) { log.trace("Try to elect last known leader for {}-{} but last known leader is not alive. last known leader={}", topicId, partitionId, partition.lastKnownElr[0]); + return false; } return true; } diff --git a/metadata/src/main/java/org/apache/kafka/image/TopicsDelta.java b/metadata/src/main/java/org/apache/kafka/image/TopicsDelta.java index b6deed5281ab7..e25bf71f05a8b 100644 --- a/metadata/src/main/java/org/apache/kafka/image/TopicsDelta.java +++ b/metadata/src/main/java/org/apache/kafka/image/TopicsDelta.java @@ -95,26 +95,43 @@ public void replay(PartitionChangeRecord record) { topicDelta.replay(record); } + private void maybeReplayClearElrRecord(Uuid topicId, ClearElrRecord record) { + // Only apply the record if the topic is not deleted. + if (!deletedTopicIds.contains(topicId)) { + TopicDelta topicDelta = getOrCreateTopicDelta(topicId); + topicDelta.replay(record); + } + } + + // When replaying the ClearElrRecord, we need to first find the latest topic ID associated with the topic(s) because + // multiple topic IDs for the same topic in a TopicsDelta is possible in the event of topic deletion and recreation. + // Second, we should not add the topicDelta if the given topic ID has been deleted. So that we don't leak the + // deleted topic ID. public void replay(ClearElrRecord record) { if (!record.topicName().isEmpty()) { - Uuid topicId; - if (image.getTopic(record.topicName()) != null) { - topicId = image.getTopic(record.topicName()).id(); - } else { + Uuid topicId = null; + // CreatedTopics contains the latest topic IDs. It should be checked first in case the topic is deleted and + // created in the same batch. + if (createdTopics.containsKey(record.topicName())) { topicId = createdTopics.get(record.topicName()); + } else if (image.getTopic(record.topicName()) != null) { + topicId = image.getTopic(record.topicName()).id(); } + if (topicId == null) { throw new RuntimeException("Unable to clear elr for topic with name " + record.topicName() + ": no such topic found."); } - TopicDelta topicDelta = getOrCreateTopicDelta(topicId); - topicDelta.replay(record); + + maybeReplayClearElrRecord(topicId, record); } else { // Update all the existing topics image.topicsById().forEach((topicId, image) -> { - TopicDelta topicDelta = getOrCreateTopicDelta(topicId); - topicDelta.replay(record); + maybeReplayClearElrRecord(topicId, record); }); + createdTopicIds().forEach((topicId -> { + maybeReplayClearElrRecord(topicId, record); + })); } } diff --git a/metadata/src/test/java/org/apache/kafka/controller/AclControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/AclControlManagerTest.java index e68310afc8ac6..d0a743ce63fd0 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/AclControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/AclControlManagerTest.java @@ -48,6 +48,7 @@ import org.apache.kafka.server.authorizer.AuthorizationResult; import org.apache.kafka.server.authorizer.AuthorizerServerInfo; import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.server.mutable.BoundedListTooLongException; import org.apache.kafka.timeline.SnapshotRegistry; import org.junit.jupiter.api.Test; @@ -69,6 +70,7 @@ import static org.apache.kafka.common.resource.PatternType.LITERAL; import static org.apache.kafka.common.resource.PatternType.MATCH; import static org.apache.kafka.common.resource.ResourceType.TOPIC; +import static org.apache.kafka.controller.QuorumController.MAX_RECORDS_PER_USER_OP; import static org.apache.kafka.metadata.authorizer.StandardAclWithIdTest.TEST_ACLS; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -380,4 +382,60 @@ public void testDeleteDedupe() { assertEquals(id, ((RemoveAccessControlEntryRecord) deleteAclResultsBothFilters.records().get(0).message()).id()); assertEquals(2, deleteAclResultsBothFilters.response().size()); } + + @Test + public void testDeleteExceedsMaxRecords() { + AclControlManager manager = new AclControlManager.Builder().build(); + MockClusterMetadataAuthorizer authorizer = new MockClusterMetadataAuthorizer(); + authorizer.loadSnapshot(manager.idToAcl()); + + List firstCreate = new ArrayList<>(); + List secondCreate = new ArrayList<>(); + + // create MAX_RECORDS_PER_USER_OP + 2 ACLs + for (int i = 0; i < MAX_RECORDS_PER_USER_OP + 2; i++) { + StandardAclWithId acl = new StandardAclWithId(Uuid.randomUuid(), + new StandardAcl( + ResourceType.TOPIC, + "mytopic_" + i, + PatternType.LITERAL, + "User:alice", + "127.0.0.1", + AclOperation.READ, + AclPermissionType.ALLOW)); + + // split acl creations between two create requests + if (i % 2 == 0) { + firstCreate.add(acl.toBinding()); + } else { + secondCreate.add(acl.toBinding()); + } + } + ControllerResult> firstCreateResult = manager.createAcls(firstCreate); + assertEquals((MAX_RECORDS_PER_USER_OP / 2) + 1, firstCreateResult.response().size()); + for (AclCreateResult result : firstCreateResult.response()) { + assertTrue(result.exception().isEmpty()); + } + + ControllerResult> secondCreateResult = manager.createAcls(secondCreate); + assertEquals((MAX_RECORDS_PER_USER_OP / 2) + 1, secondCreateResult.response().size()); + for (AclCreateResult result : secondCreateResult.response()) { + assertTrue(result.exception().isEmpty()); + } + + RecordTestUtils.replayAll(manager, firstCreateResult.records()); + RecordTestUtils.replayAll(manager, secondCreateResult.records()); + assertFalse(manager.idToAcl().isEmpty()); + + ArrayList filters = new ArrayList<>(); + for (int i = 0; i < MAX_RECORDS_PER_USER_OP + 2; i++) { + filters.add(new AclBindingFilter( + new ResourcePatternFilter(ResourceType.TOPIC, "mytopic_" + i, PatternType.LITERAL), + AccessControlEntryFilter.ANY)); + } + + Exception exception = assertThrows(InvalidRequestException.class, () -> manager.deleteAcls(filters)); + assertEquals(BoundedListTooLongException.class, exception.getCause().getClass()); + assertEquals("Cannot remove more than " + MAX_RECORDS_PER_USER_OP + " acls in a single delete operation.", exception.getCause().getMessage()); + } } diff --git a/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java index eac143209dd6b..c67f553d843d2 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java @@ -395,13 +395,25 @@ public void testCannotDowngradeBeforeMinimumKraftVersion() { MetadataVersion.MINIMUM_VERSION.featureLevel(), MetadataVersion.latestTesting().featureLevel())). build(); manager.replay(new FeatureLevelRecord().setName(MetadataVersion.FEATURE_NAME).setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel())); - assertEquals(ControllerResult.of(List.of(), new ApiError(Errors.INVALID_UPDATE_VERSION, - "Invalid update version 6 for feature metadata.version. Local controller 0 only supports versions 7-28")), - manager.updateFeatures( - Map.of(MetadataVersion.FEATURE_NAME, MetadataVersionTestUtils.IBP_3_3_IV2_FEATURE_LEVEL), - Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE), - true, - 0)); + assertEquals( + ControllerResult.of( + List.of(), + new ApiError( + Errors.INVALID_UPDATE_VERSION, + String.format( + "Invalid update version 6 for feature metadata.version. Local controller 0 only supports versions %s-%s", + MetadataVersion.MINIMUM_VERSION.featureLevel(), + MetadataVersion.latestTesting().featureLevel() + ) + ) + ), + manager.updateFeatures( + Map.of(MetadataVersion.FEATURE_NAME, MetadataVersionTestUtils.IBP_3_3_IV2_FEATURE_LEVEL), + Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE), + true, + 0 + ) + ); } @Test diff --git a/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java b/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java index bd7de9bbd1530..f836be9e93f58 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java @@ -34,6 +34,7 @@ import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.EnumSource; import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.ValueSource; @@ -1233,4 +1234,39 @@ public void testEligibleLeaderReplicas_ElectLastKnownLeaderShouldFail() { // No change to the partition. assertEquals(Optional.empty(), builder.build()); } + + @ParameterizedTest + @EnumSource(value = Election.class) + public void testEligibleLeaderReplicas_NotEligibleLastKnownLeader(Election type) { + short version = 2; + PartitionRegistration partition = new PartitionRegistration.Builder() + .setReplicas(new int[] {1, 2, 3, 4}) + .setDirectories(new Uuid[]{ + Uuid.fromString("zANDdMukTEqefOvHpmniMg"), + Uuid.fromString("Ui2Eq8rbRiuW7m7uiPTRyg"), + Uuid.fromString("MhgJOZrrTsKNcGM0XKK4aA"), + Uuid.fromString("Y25PaCAmRfyGIKxAThhBAw") + }) + .setIsr(new int[] {}) + .setElr(new int[] {}) + .setLastKnownElr(new int[] {1}) + .setLeader(-1) + .setLeaderRecoveryState(LeaderRecoveryState.RECOVERED) + .setLeaderEpoch(100) + .setPartitionEpoch(200) + .build(); + Uuid topicId = Uuid.fromString("FbrrdcfiR-KC2CPSTHaJrg"); + + PartitionChangeBuilder builder = new PartitionChangeBuilder(partition, topicId, 0, r -> false, + metadataVersionForPartitionChangeRecordVersion(version), 3) + .setElection(type) + .setEligibleLeaderReplicasEnabled(true) + .setDefaultDirProvider(DEFAULT_DIR_PROVIDER) + .setUseLastKnownLeaderInBalancedRecovery(true); + + builder.setTargetIsr(List.of()); + + // No change to the partition. + assertEquals(Optional.empty(), builder.build()); + } } diff --git a/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java b/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java index 1fb44875c4598..e56661562326b 100644 --- a/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java @@ -428,6 +428,229 @@ public void testClearElrRecords() { assertEquals(0, image.getTopic(barId).partitions().get(0).lastKnownElr.length); } + @Test + public void testClearElrRecordOnNonExistingTopic() { + TopicsImage image = TopicsImage.EMPTY; + + List topicRecords = new ArrayList<>(); + topicRecords.addAll(List.of( + new ApiMessageAndVersion( + new ClearElrRecord().setTopicName("foo"), + CLEAR_ELR_RECORD.highestSupportedVersion() + )) + ); + TopicsDelta delta = new TopicsDelta(image); + assertThrows(RuntimeException.class, () -> RecordTestUtils.replayAll(delta, topicRecords)); + } + + @Test + public void testClearElrRecords_All_ForDeletedTopics() { + Uuid fooId = Uuid.fromString("0hHJ3X5ZQ-CFfQ5xgpj90w"); + Uuid fooId2 = Uuid.randomUuid(); + Uuid barId = Uuid.fromString("f62ptyETTjet8SL5ZeREiw"); + Uuid barId2 = Uuid.randomUuid(); + + List topics = new ArrayList<>(); + topics.add( + newTopicImage( + "foo", + fooId, + newPartition(new int[] {0, 1, 2, 3}) + ) + ); + TopicsImage image = new TopicsImage(newTopicsByIdMap(topics), + newTopicsByNameMap(topics)); + + List topicRecords = new ArrayList<>(); + topicRecords.add( + new ApiMessageAndVersion( + new PartitionRecord().setTopicId(fooId). + setPartitionId(0). + setLeader(0). + setIsr(List.of(1, 2, 3)), + PARTITION_RECORD.highestSupportedVersion() + ) + ); + + TopicsDelta delta = new TopicsDelta(image); + RecordTestUtils.replayAll(delta, topicRecords); + image = delta.apply(); + + topicRecords = new ArrayList<>(); + /* Test the following: + 1. Topic foo is deleted and created in the same delta, the clear elr applies on the new topic + 2. Topic bar is created, deleted, then created in the same delta, the clear elr applies on the new topic + */ + topicRecords.addAll(List.of( + new ApiMessageAndVersion( + new RemoveTopicRecord().setTopicId(fooId), + REMOVE_TOPIC_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new TopicRecord().setTopicId(fooId2). + setName("foo"), + TOPIC_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new PartitionRecord().setTopicId(fooId2).setPartitionId(0). + setIsr(List.of(0, 1)). + setEligibleLeaderReplicas(List.of(2)). + setLastKnownElr(List.of(3)), + PARTITION_CHANGE_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new TopicRecord().setTopicId(barId). + setName("bar"), + TOPIC_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new PartitionRecord().setTopicId(barId). + setPartitionId(0). + setLeader(0). + setIsr(List.of(1, 2, 3)), + PARTITION_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new RemoveTopicRecord().setTopicId(barId), + REMOVE_TOPIC_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new TopicRecord().setTopicId(barId2). + setName("bar"), + TOPIC_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new PartitionRecord().setTopicId(barId2). + setPartitionId(0). + setLeader(0). + setIsr(List.of(1)). + setEligibleLeaderReplicas(List.of(2)). + setLastKnownElr(List.of(3)), + PARTITION_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new ClearElrRecord(), + CLEAR_ELR_RECORD.highestSupportedVersion() + )) + ); + delta = new TopicsDelta(image); + RecordTestUtils.replayAll(delta, topicRecords); + image = delta.apply(); + assertEquals(2, image.topicsById().size()); + assertEquals(2, image.topicsByName().size()); + + assertEquals(0, image.getTopic(fooId2).partitions().get(0).elr.length); + assertEquals(0, image.getTopic(fooId2).partitions().get(0).lastKnownElr.length); + assertEquals(0, image.getTopic(barId2).partitions().get(0).elr.length); + assertEquals(0, image.getTopic(barId2).partitions().get(0).lastKnownElr.length); + } + + @Test + public void testClearElrRecords_Single_ForDeletedTopics() { + Uuid fooId = Uuid.fromString("0hHJ3X5ZQ-CFfQ5xgpj90w"); + Uuid fooId2 = Uuid.randomUuid(); + Uuid barId = Uuid.fromString("f62ptyETTjet8SL5ZeREiw"); + Uuid barId2 = Uuid.randomUuid(); + + List topics = new ArrayList<>(); + topics.add( + newTopicImage( + "foo", + fooId, + newPartition(new int[] {0, 1, 2, 3}) + ) + ); + TopicsImage image = new TopicsImage(newTopicsByIdMap(topics), + newTopicsByNameMap(topics)); + + List topicRecords = new ArrayList<>(); + topicRecords.add( + new ApiMessageAndVersion( + new PartitionRecord().setTopicId(fooId). + setPartitionId(0). + setLeader(0). + setIsr(List.of(1, 2, 3)), + PARTITION_RECORD.highestSupportedVersion() + ) + ); + + TopicsDelta delta = new TopicsDelta(image); + RecordTestUtils.replayAll(delta, topicRecords); + image = delta.apply(); + + topicRecords = new ArrayList<>(); + /* Test the following: + 1. Topic foo is deleted and created in the same delta, the clear elr applies on the new topic + 2. Topic bar is created, deleted, then created in the same delta, the clear elr applies on the new topic + */ + topicRecords.addAll(List.of( + new ApiMessageAndVersion( + new RemoveTopicRecord().setTopicId(fooId), + REMOVE_TOPIC_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new TopicRecord().setTopicId(fooId2). + setName("foo"), + TOPIC_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new PartitionRecord().setTopicId(fooId2).setPartitionId(0). + setIsr(List.of(0, 1)). + setEligibleLeaderReplicas(List.of(2)). + setLastKnownElr(List.of(3)), + PARTITION_CHANGE_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new TopicRecord().setTopicId(barId). + setName("bar"), + TOPIC_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new PartitionRecord().setTopicId(barId). + setPartitionId(0). + setLeader(0). + setIsr(List.of(1, 2, 3)), + PARTITION_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new RemoveTopicRecord().setTopicId(barId), + REMOVE_TOPIC_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new TopicRecord().setTopicId(barId2). + setName("bar"), + TOPIC_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new PartitionRecord().setTopicId(barId2). + setPartitionId(0). + setLeader(0). + setIsr(List.of(1)). + setEligibleLeaderReplicas(List.of(2)). + setLastKnownElr(List.of(3)), + PARTITION_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new ClearElrRecord().setTopicName("foo"), + CLEAR_ELR_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new ClearElrRecord().setTopicName("bar"), + CLEAR_ELR_RECORD.highestSupportedVersion() + )) + ); + delta = new TopicsDelta(image); + RecordTestUtils.replayAll(delta, topicRecords); + image = delta.apply(); + assertEquals(2, image.topicsById().size()); + assertEquals(2, image.topicsByName().size()); + + assertEquals(0, image.getTopic(fooId2).partitions().get(0).elr.length); + assertEquals(0, image.getTopic(fooId2).partitions().get(0).lastKnownElr.length); + assertEquals(0, image.getTopic(barId2).partitions().get(0).elr.length); + assertEquals(0, image.getTopic(barId2).partitions().get(0).lastKnownElr.length); + } + @Test public void testClearElrRecordForNonExistTopic() { TopicsImage image = new TopicsImage(newTopicsByIdMap(List.of()), diff --git a/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java b/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java index 2eeeab2259a21..5ddcd2d88890d 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java @@ -378,6 +378,9 @@ public void testFeatureFlag(short version) throws Exception { setName(MetadataVersion.FEATURE_NAME). setFeatureLevel(MetadataVersion.latestProduction().featureLevel()), (short) 0)); + expected.add(new ApiMessageAndVersion(new FeatureLevelRecord(). + setName(EligibleLeaderReplicasVersion.FEATURE_NAME). + setFeatureLevel(EligibleLeaderReplicasVersion.ELRV_1.featureLevel()), (short) 0)); expected.add(new ApiMessageAndVersion(new FeatureLevelRecord(). setName(GroupVersion.FEATURE_NAME). setFeatureLevel(GroupVersion.GV_1.featureLevel()), (short) 0)); diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 760d6b8c15997..970ee2d69101b 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -25,7 +25,6 @@ import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.errors.ClusterAuthorizationException; import org.apache.kafka.common.errors.CorruptRecordException; -import org.apache.kafka.common.errors.NotLeaderOrFollowerException; import org.apache.kafka.common.feature.SupportedVersionRange; import org.apache.kafka.common.memory.MemoryPool; import org.apache.kafka.common.message.AddRaftVoterRequestData; @@ -386,6 +385,11 @@ private void onUpdateLeaderHighWatermark( // records still held in memory directly to the listener appendPurgatory.maybeComplete(highWatermark.offset(), currentTimeMs); + // After updating the high-watermark, complete all of the deferred + // fetch requests. This is always correct because all fetch request + // deferred have a HWM less or equal to the previous leader's HWM. + fetchPurgatory.completeAll(currentTimeMs); + // It is also possible that the high watermark is being updated // for the first time following the leader election, so we need // to give lagging listeners an opportunity to catch up as well @@ -741,7 +745,10 @@ private void transitionToUnattached(int epoch, OptionalInt leaderId) { private void transitionToResigned(List preferredSuccessors) { fetchPurgatory.completeAllExceptionally( - Errors.NOT_LEADER_OR_FOLLOWER.exception("Not handling request since this node is resigning")); + Errors.NOT_LEADER_OR_FOLLOWER.exception( + "Not handling request since this node is resigning" + ) + ); quorum.transitionToResigned(preferredSuccessors); resetConnections(); } @@ -753,12 +760,18 @@ private void onBecomeFollower(long currentTimeMs) { // After becoming a follower, we need to complete all pending fetches so that // they can be re-sent to the leader without waiting for their expirations - fetchPurgatory.completeAllExceptionally(new NotLeaderOrFollowerException( - "Cannot process the fetch request because the node is no longer the leader.")); + fetchPurgatory.completeAllExceptionally( + Errors.NOT_LEADER_OR_FOLLOWER.exception( + "Cannot process the fetch request because the node is no longer the leader" + ) + ); // Clearing the append purgatory should complete all futures exceptionally since this node is no longer the leader - appendPurgatory.completeAllExceptionally(new NotLeaderOrFollowerException( - "Failed to receive sufficient acknowledgments for this append before leader change.")); + appendPurgatory.completeAllExceptionally( + Errors.NOT_LEADER_OR_FOLLOWER.exception( + "Failed to receive sufficient acknowledgments for this append before leader change" + ) + ); } private void transitionToFollower( @@ -1514,19 +1527,22 @@ private CompletableFuture handleFetchRequest( || FetchResponse.recordsSize(partitionResponse) > 0 || request.maxWaitMs() == 0 || isPartitionDiverged(partitionResponse) - || isPartitionSnapshotted(partitionResponse)) { + || isPartitionSnapshotted(partitionResponse) + || isHighWatermarkUpdated(partitionResponse, fetchPartition)) { // Reply immediately if any of the following is true // 1. The response contains an error // 2. There are records in the response // 3. The fetching replica doesn't want to wait for the partition to contain new data // 4. The fetching replica needs to truncate because the log diverged // 5. The fetching replica needs to fetch a snapshot + // 6. The fetching replica should update its high-watermark return completedFuture(response); } CompletableFuture future = fetchPurgatory.await( fetchPartition.fetchOffset(), - request.maxWaitMs()); + request.maxWaitMs() + ); return future.handle((completionTimeMs, exception) -> { if (exception != null) { @@ -1556,26 +1572,25 @@ private CompletableFuture handleFetchRequest( Optional.empty() ); } - } - - // FIXME: `completionTimeMs`, which can be null - logger.trace( - "Completing delayed fetch from {} starting at offset {} at {}", - replicaKey, - fetchPartition.fetchOffset(), - completionTimeMs - ); + } else { + logger.trace( + "Completing delayed fetch from {} starting at offset {} at {}", + replicaKey, + fetchPartition.fetchOffset(), + completionTimeMs + ); - // It is safe to call tryCompleteFetchRequest because only the polling thread completes this - // future successfully. This is true because only the polling thread appends record batches to - // the log from maybeAppendBatches. - return tryCompleteFetchRequest( - requestMetadata.listenerName(), - requestMetadata.apiVersion(), - replicaKey, - fetchPartition, - time.milliseconds() - ); + // It is safe to call tryCompleteFetchRequest because only the polling thread completes + // this future successfully. The future is completed successfully either because of an + // append (maybeAppendBatches) or because the HWM was updated (onUpdateLeaderHighWatermark) + return tryCompleteFetchRequest( + requestMetadata.listenerName(), + requestMetadata.apiVersion(), + replicaKey, + fetchPartition, + completionTimeMs + ); + } }); } @@ -1633,18 +1648,29 @@ private FetchResponseData tryCompleteFetchRequest( } } - private static boolean isPartitionDiverged(FetchResponseData.PartitionData partitionResponseData) { + private static boolean isPartitionDiverged( + FetchResponseData.PartitionData partitionResponseData + ) { FetchResponseData.EpochEndOffset divergingEpoch = partitionResponseData.divergingEpoch(); return divergingEpoch.epoch() != -1 || divergingEpoch.endOffset() != -1; } - private static boolean isPartitionSnapshotted(FetchResponseData.PartitionData partitionResponseData) { + private static boolean isPartitionSnapshotted( + FetchResponseData.PartitionData partitionResponseData + ) { FetchResponseData.SnapshotId snapshotId = partitionResponseData.snapshotId(); return snapshotId.epoch() != -1 || snapshotId.endOffset() != -1; } + private static boolean isHighWatermarkUpdated( + FetchResponseData.PartitionData partitionResponseData, + FetchRequestData.FetchPartition partitionRequestData + ) { + return partitionRequestData.highWatermark() < partitionResponseData.highWatermark(); + } + private static OptionalInt optionalLeaderId(int leaderIdOrNil) { if (leaderIdOrNil < 0) return OptionalInt.empty(); @@ -2882,6 +2908,7 @@ private FetchRequestData buildFetchRequest() { .setLastFetchedEpoch(log.lastFetchedEpoch()) .setFetchOffset(log.endOffset().offset()) .setReplicaDirectoryId(quorum.localDirectoryId()) + .setHighWatermark(quorum.highWatermark().map(LogOffsetMetadata::offset).orElse(-1L)) ); return request diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientClusterAuthTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientClusterAuthTest.java index 62c8e11769817..5fa6a0781a8b5 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientClusterAuthTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientClusterAuthTest.java @@ -53,7 +53,7 @@ void testClusterAuthorizationFailedInFetch(boolean withKip853Rpc) throws Excepti context.pollUntilRequest(); - RaftRequest.Outbound request = context.assertSentFetchRequest(epoch, 0, 0); + RaftRequest.Outbound request = context.assertSentFetchRequest(epoch, 0, 0, context.client.highWatermark()); FetchResponseData response = new FetchResponseData() .setErrorCode(Errors.CLUSTER_AUTHORIZATION_FAILED.code()); context.deliverResponse( diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientFetchTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientFetchTest.java index ade509d8051f4..cfbc03070bb3e 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientFetchTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientFetchTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.raft; import org.apache.kafka.common.compress.Compression; +import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.ArbitraryMemoryRecords; import org.apache.kafka.common.record.InvalidMemoryRecordsProvider; @@ -33,7 +34,10 @@ import org.junit.jupiter.params.provider.ArgumentsSource; import java.nio.ByteBuffer; +import java.util.List; import java.util.Optional; +import java.util.OptionalInt; +import java.util.OptionalLong; import java.util.stream.IntStream; import java.util.stream.Stream; @@ -72,7 +76,7 @@ private static void testFetchResponseWithInvalidRecord(MemoryRecords records, in context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, OptionalLong.empty()); long oldLogEndOffset = context.log.endOffset().offset(); @@ -107,7 +111,7 @@ void testReplicationOfHigherPartitionLeaderEpoch() throws Exception { context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, OptionalLong.empty()); long oldLogEndOffset = context.log.endOffset().offset(); int numberOfRecords = 10; @@ -149,4 +153,327 @@ void testReplicationOfHigherPartitionLeaderEpoch() throws Exception { // Check that only the first batch was appended because the second batch has a greater epoch assertEquals(oldLogEndOffset + numberOfRecords, context.log.endOffset().offset()); } + + @Test + void testHighWatermarkSentInFetchRequest() throws Exception { + int epoch = 2; + int localId = KafkaRaftClientTest.randomReplicaId(); + ReplicaKey local = KafkaRaftClientTest.replicaKey(localId, true); + ReplicaKey electedLeader = KafkaRaftClientTest.replicaKey(localId + 1, true); + + RaftClientTestContext context = new RaftClientTestContext.Builder( + local.id(), + local.directoryId().get() + ) + .appendToLog(epoch, List.of("a", "b", "c")) + .appendToLog(epoch, List.of("d", "e", "f")) + .withStartingVoters( + VoterSetTest.voterSet(Stream.of(local, electedLeader)), KRaftVersion.KRAFT_VERSION_1 + ) + .withElectedLeader(epoch, electedLeader.id()) + .withRaftProtocol(RaftClientTestContext.RaftProtocol.KIP_1166_PROTOCOL) + .build(); + + var localLogEndOffset = context.log.endOffset().offset(); + + context.pollUntilRequest(); + RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); + context.assertFetchRequestData( + fetchRequest, + epoch, + localLogEndOffset, + epoch, + OptionalLong.empty() + ); + + // Set the HWM to the LEO + context.deliverResponse( + fetchRequest.correlationId(), + fetchRequest.destination(), + context.fetchResponse( + epoch, + electedLeader.id(), + MemoryRecords.EMPTY, + localLogEndOffset, + Errors.NONE + ) + ); + + context.pollUntilRequest(); + fetchRequest = context.assertSentFetchRequest(); + context.assertFetchRequestData( + fetchRequest, + epoch, + localLogEndOffset, + epoch, + OptionalLong.of(localLogEndOffset) + ); + } + + @Test + void testDefaultHwmDeferred() throws Exception { + var epoch = 2; + var local = KafkaRaftClientTest.replicaKey( + KafkaRaftClientTest.randomReplicaId(), + true + ); + var voter = KafkaRaftClientTest.replicaKey(local.id() + 1, true); + var remote = KafkaRaftClientTest.replicaKey(local.id() + 2, true); + + RaftClientTestContext context = new RaftClientTestContext.Builder( + local.id(), + local.directoryId().get() + ) + .appendToLog(epoch, List.of("a", "b", "c")) + .appendToLog(epoch, List.of("d", "e", "f")) + .withStartingVoters( + VoterSetTest.voterSet(Stream.of(local, voter)), KRaftVersion.KRAFT_VERSION_1 + ) + .withUnknownLeader(epoch) + .withRaftProtocol(RaftClientTestContext.RaftProtocol.KIP_1166_PROTOCOL) + .build(); + + context.unattachedToLeader(); + epoch = context.currentEpoch(); + + context.advanceLocalLeaderHighWatermarkToLogEndOffset(); + + var localLogEndOffset = context.log.endOffset().offset(); + var lastFetchedEpoch = context.log.lastFetchedEpoch(); + context.deliverRequest( + context.fetchRequest( + epoch, + remote, + localLogEndOffset, + lastFetchedEpoch, + Integer.MAX_VALUE + ) + ); + + // Check that the fetch response was deferred + for (var i = 0; i < 10; ++i) { + context.client.poll(); + assertEquals(List.of(), context.drainSentResponses(ApiKeys.FETCH)); + } + } + + @Test + void testUnknownHwmDeferredWhenLeaderDoesNotKnowHwm() throws Exception { + var epoch = 2; + var local = KafkaRaftClientTest.replicaKey( + KafkaRaftClientTest.randomReplicaId(), + true + ); + var voter = KafkaRaftClientTest.replicaKey(local.id() + 1, true); + var remote = KafkaRaftClientTest.replicaKey(local.id() + 2, true); + + RaftClientTestContext context = new RaftClientTestContext.Builder( + local.id(), + local.directoryId().get() + ) + .appendToLog(epoch, List.of("a", "b", "c")) + .appendToLog(epoch, List.of("d", "e", "f")) + .withStartingVoters( + VoterSetTest.voterSet(Stream.of(local, voter)), KRaftVersion.KRAFT_VERSION_1 + ) + .withUnknownLeader(epoch) + .withRaftProtocol(RaftClientTestContext.RaftProtocol.KIP_1166_PROTOCOL) + .build(); + + context.unattachedToLeader(); + epoch = context.currentEpoch(); + + var localLogEndOffset = context.log.endOffset().offset(); + var lastFetchedEpoch = context.log.lastFetchedEpoch(); + context.deliverRequest( + context.fetchRequest( + epoch, + remote, + localLogEndOffset, + lastFetchedEpoch, + OptionalLong.empty(), + Integer.MAX_VALUE + ) + ); + + // Check that the fetch response was deferred + for (var i = 0; i < 10; ++i) { + context.client.poll(); + assertEquals(List.of(), context.drainSentResponses(ApiKeys.FETCH)); + } + } + + @Test + void testOutdatedHwmCompletedWhenLeaderKnowsHwm() throws Exception { + var epoch = 2; + var local = KafkaRaftClientTest.replicaKey( + KafkaRaftClientTest.randomReplicaId(), + true + ); + var voter = KafkaRaftClientTest.replicaKey(local.id() + 1, true); + var remote = KafkaRaftClientTest.replicaKey(local.id() + 2, true); + + RaftClientTestContext context = new RaftClientTestContext.Builder( + local.id(), + local.directoryId().get() + ) + .appendToLog(epoch, List.of("a", "b", "c")) + .appendToLog(epoch, List.of("d", "e", "f")) + .withStartingVoters( + VoterSetTest.voterSet(Stream.of(local, voter)), KRaftVersion.KRAFT_VERSION_1 + ) + .withUnknownLeader(epoch) + .withRaftProtocol(RaftClientTestContext.RaftProtocol.KIP_1166_PROTOCOL) + .build(); + + context.unattachedToLeader(); + epoch = context.currentEpoch(); + + context.advanceLocalLeaderHighWatermarkToLogEndOffset(); + + var localLogEndOffset = context.log.endOffset().offset(); + var lastFetchedEpoch = context.log.lastFetchedEpoch(); + + // FETCH response completed when remote replica doesn't know HWM + context.deliverRequest( + context.fetchRequest( + epoch, + remote, + localLogEndOffset, + lastFetchedEpoch, + OptionalLong.empty(), + Integer.MAX_VALUE + ) + ); + context.pollUntilResponse(); + context.assertSentFetchPartitionResponse(localLogEndOffset, epoch); + + // FETCH response completed when remote replica has outdated HWM + context.deliverRequest( + context.fetchRequest( + epoch, + remote, + localLogEndOffset, + lastFetchedEpoch, + OptionalLong.of(localLogEndOffset - 1), + Integer.MAX_VALUE + ) + ); + context.pollUntilResponse(); + context.assertSentFetchPartitionResponse(localLogEndOffset, epoch); + } + + @Test + void testUnchangedHighWatermarkDeferred() throws Exception { + var epoch = 2; + var local = KafkaRaftClientTest.replicaKey( + KafkaRaftClientTest.randomReplicaId(), + true + ); + var voter = KafkaRaftClientTest.replicaKey(local.id() + 1, true); + var remote = KafkaRaftClientTest.replicaKey(local.id() + 2, true); + + RaftClientTestContext context = new RaftClientTestContext.Builder( + local.id(), + local.directoryId().get() + ) + .appendToLog(epoch, List.of("a", "b", "c")) + .appendToLog(epoch, List.of("d", "e", "f")) + .withStartingVoters( + VoterSetTest.voterSet(Stream.of(local, voter)), KRaftVersion.KRAFT_VERSION_1 + ) + .withUnknownLeader(epoch) + .withRaftProtocol(RaftClientTestContext.RaftProtocol.KIP_1166_PROTOCOL) + .build(); + + context.unattachedToLeader(); + epoch = context.currentEpoch(); + + context.advanceLocalLeaderHighWatermarkToLogEndOffset(); + + var localLogEndOffset = context.log.endOffset().offset(); + var lastFetchedEpoch = context.log.lastFetchedEpoch(); + context.deliverRequest( + context.fetchRequest( + epoch, + remote, + localLogEndOffset, + lastFetchedEpoch, + OptionalLong.of(localLogEndOffset), + Integer.MAX_VALUE + ) + ); + + // Check that the fetch response was deferred + for (var i = 0; i < 10; ++i) { + context.client.poll(); + assertEquals(List.of(), context.drainSentResponses(ApiKeys.FETCH)); + } + } + + @Test + void testUpdatedHighWatermarkCompleted() throws Exception { + var epoch = 2; + var local = KafkaRaftClientTest.replicaKey( + KafkaRaftClientTest.randomReplicaId(), + true + ); + var voter = KafkaRaftClientTest.replicaKey(local.id() + 1, true); + var remote = KafkaRaftClientTest.replicaKey(local.id() + 2, true); + + RaftClientTestContext context = new RaftClientTestContext.Builder( + local.id(), + local.directoryId().get() + ) + .appendToLog(epoch, List.of("a", "b", "c")) + .appendToLog(epoch, List.of("d", "e", "f")) + .withStartingVoters( + VoterSetTest.voterSet(Stream.of(local, voter)), KRaftVersion.KRAFT_VERSION_1 + ) + .withUnknownLeader(epoch) + .withRaftProtocol(RaftClientTestContext.RaftProtocol.KIP_1166_PROTOCOL) + .build(); + + context.unattachedToLeader(); + epoch = context.currentEpoch(); + + // Establish a HWM (3) but don't set it to the LEO + context.deliverRequest(context.fetchRequest(epoch, voter, 3L, 2, 0)); + context.pollUntilResponse(); + context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(local.id())); + + var localLogEndOffset = context.log.endOffset().offset(); + var lastFetchedEpoch = context.log.lastFetchedEpoch(); + context.deliverRequest( + context.fetchRequest( + epoch, + remote, + localLogEndOffset, + lastFetchedEpoch, + OptionalLong.of(localLogEndOffset), + Integer.MAX_VALUE + ) + ); + + // Check that the fetch response was deferred + for (var i = 0; i < 10; ++i) { + context.client.poll(); + assertEquals(List.of(), context.drainSentResponses(ApiKeys.FETCH)); + } + + // Update the HWM and complete the deferred FETCH response + context.deliverRequest( + context.fetchRequest(epoch, voter, localLogEndOffset, lastFetchedEpoch, 0) + ); + context.pollUntilResponse(); + + // Check that two fetch requests were completed + var fetchResponses = context.drainSentResponses(ApiKeys.FETCH); + for (var fetchResponse : fetchResponses) { + var partitionResponse = context.assertFetchResponseData(fetchResponse); + assertEquals(Errors.NONE, Errors.forCode(partitionResponse.errorCode())); + assertEquals(epoch, partitionResponse.currentLeader().leaderEpoch()); + assertEquals(localLogEndOffset, partitionResponse.highWatermark()); + } + } } diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java index 4efe3e3600ae3..c3ecb2c20c6c2 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java @@ -65,7 +65,7 @@ public void testHandlePreVoteRequestAsFollower( if (hasFetchedFromLeader) { context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest.correlationId(), @@ -350,7 +350,7 @@ public void testHandlePreVoteRequestAsFollowerObserver( if (hasFetchedFromLeader) { context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest.correlationId(), @@ -654,7 +654,7 @@ public void testFollowerGrantsPreVoteIfHasNotFetchedYet(KRaftVersion kraftVersio // After fetching successfully from the leader once, follower will no longer grant PreVotes context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest.correlationId(), diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java index b000fbbcd5953..3189386830d6e 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java @@ -229,7 +229,7 @@ public void testFollowerDoesNotRequestLeaderBootstrapSnapshot() throws Exception // check that follower will send fetch request to leader context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); // check if leader response were to contain bootstrap snapshot id, follower would not send fetch snapshot request context.deliverResponse( @@ -239,7 +239,7 @@ public void testFollowerDoesNotRequestLeaderBootstrapSnapshot() throws Exception ); context.pollUntilRequest(); fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); } @Test @@ -259,7 +259,7 @@ public void testFollowerReadsKRaftBootstrapRecords() throws Exception { // check that follower will send fetch request to leader context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); // check that before receiving bootstrap records from leader, follower is not in the voter set assertFalse(context.client.quorum().isVoter(follower)); @@ -2142,7 +2142,7 @@ void testFollowerSendsUpdateVoter() throws Exception { context.time.sleep(context.fetchTimeoutMs - 1); context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest.correlationId(), @@ -2179,7 +2179,7 @@ void testFollowerSendsUpdateVoter() throws Exception { // after sending an update voter the next request should be a fetch context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); } @ParameterizedTest @@ -2215,7 +2215,7 @@ void testFollowerSendsUpdateVoterWithKraftVersion0(Errors updateVoterError) thro context.time.sleep(context.fetchTimeoutMs - 1); context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest.correlationId(), @@ -2255,7 +2255,7 @@ void testFollowerSendsUpdateVoterWithKraftVersion0(Errors updateVoterError) thro context.time.sleep(context.fetchTimeoutMs - 1); context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest.correlationId(), @@ -2306,7 +2306,7 @@ void testFollowerSendsUpdateVoterAfterElectionWithKraftVersion0(Errors updateVot context.time.sleep(context.fetchTimeoutMs - 1); context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest.correlationId(), @@ -2342,7 +2342,7 @@ void testFollowerSendsUpdateVoterAfterElectionWithKraftVersion0(Errors updateVot context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); // Election a new leader causes the replica to resend update voter request int newEpoch = epoch + 1; @@ -2354,7 +2354,7 @@ void testFollowerSendsUpdateVoterAfterElectionWithKraftVersion0(Errors updateVot context.time.sleep(context.fetchTimeoutMs - 1); context.pollUntilRequest(); fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, newEpoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, newEpoch, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest.correlationId(), @@ -2390,7 +2390,7 @@ void testFollowerSendsUpdateVoterAfterElectionWithKraftVersion0(Errors updateVot context.pollUntilRequest(); fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, newEpoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, newEpoch, 0L, 0, context.client.highWatermark()); } @Test @@ -2640,7 +2640,7 @@ void testFollowerSendsUpdateVoterWhenDifferent() throws Exception { context.time.sleep(context.fetchTimeoutMs - 1); context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest.correlationId(), @@ -2661,7 +2661,7 @@ void testFollowerSendsUpdateVoterWhenDifferent() throws Exception { context.time.sleep(context.fetchTimeoutMs - 1); context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); // after more than 3 fetch timeouts the update voter period timer should have expired. // check that the update voter period timer doesn't remain at zero (0) and cause the message queue to get @@ -2701,7 +2701,7 @@ void testFollowerSendsUpdateVoterIfPendingFetchDuringTimeout() throws Exception context.time.sleep(context.fetchTimeoutMs - 1); context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest.correlationId(), @@ -2721,7 +2721,7 @@ void testFollowerSendsUpdateVoterIfPendingFetchDuringTimeout() throws Exception // expect one last FETCH request context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); // don't send a response but increase the time context.time.sleep(context.requestTimeoutMs() - 1); @@ -2781,7 +2781,7 @@ void testUpdateVoterResponseCausesEpochChange() throws Exception { context.time.sleep(context.fetchTimeoutMs - 1); context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest.correlationId(), @@ -2818,7 +2818,7 @@ void testUpdateVoterResponseCausesEpochChange() throws Exception { // check that there is a fetch to the new leader context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch + 1, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch + 1, 0L, 0, context.client.highWatermark()); assertEquals(voter2.id(), fetchRequest.destination().id()); } @@ -2837,7 +2837,7 @@ void testObserverDiscoversLeaderWithUnknownVoters() throws Exception { context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); assertEquals(-2, fetchRequest.destination().id()); } diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java index bc4be592a2082..fd696458b80aa 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java @@ -155,7 +155,13 @@ public void testFollowerListenerNotified(boolean entireLog) throws Exception { long localLogEndOffset = context.log.endOffset().offset(); context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, localLogEndOffset, snapshotId.epoch()); + context.assertFetchRequestData( + fetchRequest, + epoch, + localLogEndOffset, + snapshotId.epoch(), + context.client.highWatermark() + ); context.deliverResponse( fetchRequest.correlationId(), fetchRequest.destination(), @@ -163,7 +169,12 @@ public void testFollowerListenerNotified(boolean entireLog) throws Exception { ); context.pollUntilRequest(); - context.assertSentFetchRequest(epoch, localLogEndOffset, snapshotId.epoch()); + context.assertSentFetchRequest( + epoch, + localLogEndOffset, + snapshotId.epoch(), + context.client.highWatermark() + ); // Check that listener was notified of the new snapshot try (SnapshotReader snapshot = context.listener.drainHandledSnapshot().get()) { @@ -197,7 +208,13 @@ public void testSecondListenerNotified(boolean entireLog) throws Exception { long localLogEndOffset = context.log.endOffset().offset(); context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, localLogEndOffset, snapshotId.epoch()); + context.assertFetchRequestData( + fetchRequest, + epoch, + localLogEndOffset, + snapshotId.epoch(), + context.client.highWatermark() + ); context.deliverResponse( fetchRequest.correlationId(), fetchRequest.destination(), @@ -205,7 +222,12 @@ public void testSecondListenerNotified(boolean entireLog) throws Exception { ); context.pollUntilRequest(); - context.assertSentFetchRequest(epoch, localLogEndOffset, snapshotId.epoch()); + context.assertSentFetchRequest( + epoch, + localLogEndOffset, + snapshotId.epoch(), + context.client.highWatermark() + ); RaftClientTestContext.MockListener secondListener = new RaftClientTestContext.MockListener(OptionalInt.of(localId)); context.client.register(secondListener); @@ -1145,7 +1167,7 @@ public void testFetchResponseWithInvalidSnapshotId(boolean withKip853Rpc) throws context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest.correlationId(), @@ -1162,7 +1184,7 @@ public void testFetchResponseWithInvalidSnapshotId(boolean withKip853Rpc) throws context.pollUntilRequest(); fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest.correlationId(), @@ -1179,7 +1201,7 @@ public void testFetchResponseWithInvalidSnapshotId(boolean withKip853Rpc) throws context.pollUntilRequest(); fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); // Fetch timer is not reset; sleeping for remainder should transition to prospective context.time.sleep(context.fetchTimeoutMs - slept); @@ -1206,7 +1228,7 @@ public void testFetchResponseWithSnapshotId(boolean withKip853Rpc) throws Except context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest.correlationId(), @@ -1249,7 +1271,13 @@ public void testFetchResponseWithSnapshotId(boolean withKip853Rpc) throws Except context.pollUntilRequest(); fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, snapshotId.offset(), snapshotId.epoch()); + context.assertFetchRequestData( + fetchRequest, + epoch, + snapshotId.offset(), + snapshotId.epoch(), + context.client.highWatermark() + ); // Check that the snapshot was written to the log RawSnapshotReader snapshot = context.log.readSnapshot(snapshotId).get(); @@ -1279,7 +1307,7 @@ public void testFetchSnapshotResponsePartialData(boolean withKip853Rpc) throws E context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest.correlationId(), @@ -1354,7 +1382,13 @@ public void testFetchSnapshotResponsePartialData(boolean withKip853Rpc) throws E context.pollUntilRequest(); fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, snapshotId.offset(), snapshotId.epoch()); + context.assertFetchRequestData( + fetchRequest, + epoch, + snapshotId.offset(), + snapshotId.epoch(), + context.client.highWatermark() + ); // Check that the snapshot was written to the log RawSnapshotReader snapshot = context.log.readSnapshot(snapshotId).get(); @@ -1384,7 +1418,7 @@ public void testFetchSnapshotResponseMissingSnapshot(boolean withKip853Rpc) thro context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest.correlationId(), @@ -1426,7 +1460,7 @@ public void testFetchSnapshotResponseMissingSnapshot(boolean withKip853Rpc) thro context.pollUntilRequest(); fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); } @ParameterizedTest @@ -1446,7 +1480,7 @@ public void testFetchSnapshotResponseFromNewerEpochNotLeader(boolean withKip853R context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest.correlationId(), @@ -1488,7 +1522,7 @@ public void testFetchSnapshotResponseFromNewerEpochNotLeader(boolean withKip853R context.pollUntilRequest(); fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch + 1, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch + 1, 0L, 0, context.client.highWatermark()); } @ParameterizedTest @@ -1507,7 +1541,7 @@ public void testFetchSnapshotResponseFromNewerEpochLeader(boolean withKip853Rpc) context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest.correlationId(), @@ -1549,7 +1583,7 @@ public void testFetchSnapshotResponseFromNewerEpochLeader(boolean withKip853Rpc) context.pollUntilRequest(); fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch + 1, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch + 1, 0L, 0, context.client.highWatermark()); } @ParameterizedTest @@ -1568,7 +1602,7 @@ public void testFetchSnapshotResponseFromOlderEpoch(boolean withKip853Rpc) throw context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest.correlationId(), @@ -1639,7 +1673,7 @@ public void testFetchSnapshotResponseWithInvalidId(boolean withKip853Rpc) throws context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest.correlationId(), @@ -1687,7 +1721,7 @@ public void testFetchSnapshotResponseWithInvalidId(boolean withKip853Rpc) throws // Follower should send a fetch request fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest.correlationId(), @@ -1736,7 +1770,7 @@ public void testFetchSnapshotResponseWithInvalidId(boolean withKip853Rpc) throws // Follower should send a fetch request fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); } @ParameterizedTest @@ -1755,7 +1789,7 @@ public void testFetchSnapshotResponseToNotFollower(boolean withKip853Rpc) throws context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest.correlationId(), @@ -2014,7 +2048,7 @@ public void testCreateSnapshotAsFollowerWithInvalidSnapshotId(boolean withKip853 context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); assertTrue(voters.contains(fetchRequest.destination().id())); - context.assertFetchRequestData(fetchRequest, epoch, 1L, 1); + context.assertFetchRequestData(fetchRequest, epoch, 1L, 1, context.client.highWatermark()); // The response does not advance the high watermark List records1 = List.of("b", "c"); @@ -2042,7 +2076,7 @@ public void testCreateSnapshotAsFollowerWithInvalidSnapshotId(boolean withKip853 context.pollUntilRequest(); fetchRequest = context.assertSentFetchRequest(); assertTrue(voters.contains(fetchRequest.destination().id())); - context.assertFetchRequestData(fetchRequest, epoch, 3L, 3); + context.assertFetchRequestData(fetchRequest, epoch, 3L, 3, context.client.highWatermark()); List records2 = List.of("d", "e", "f"); int batch2Epoch = 4; diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index 95cc3c3c4fbff..eddb99608fb96 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -818,7 +818,7 @@ public void testCannotResignIfObserver(boolean withKip853Rpc) throws Exception { RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); assertTrue(voters.contains(fetchRequest.destination().id())); - context.assertFetchRequestData(fetchRequest, 0, 0L, 0); + context.assertFetchRequestData(fetchRequest, 0, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest.correlationId(), @@ -863,7 +863,7 @@ public void testInitializeAsUnattachedAndBecomeLeader(boolean withKip853Rpc) thr context.assertUnknownLeaderAndNoVotedCandidate(0); context.pollUntilRequest(); - RaftRequest.Outbound request = context.assertSentFetchRequest(0, 0L, 0); + RaftRequest.Outbound request = context.assertSentFetchRequest(0, 0L, 0, OptionalLong.empty()); assertTrue(context.client.quorum().isUnattached()); assertTrue(context.client.quorum().isVoter()); @@ -1430,7 +1430,7 @@ public void testHandleEndQuorumRequestWithLowerPriorityToBecomeLeader(boolean wi context.time.sleep(1); context.client.poll(); - context.assertSentFetchRequest(leaderEpoch, 0, 0); + context.assertSentFetchRequest(leaderEpoch, 0, 0, OptionalLong.empty()); context.time.sleep(context.electionBackoffMaxMs); context.client.poll(); @@ -1885,7 +1885,7 @@ public void testInitializeAsFollowerEmptyLog(boolean withKip853Rpc) throws Excep context.pollUntilRequest(); - context.assertSentFetchRequest(epoch, 0L, 0); + context.assertSentFetchRequest(epoch, 0L, 0, OptionalLong.empty()); } @ParameterizedTest @@ -1906,7 +1906,7 @@ public void testInitializeAsFollowerNonEmptyLog(boolean withKip853Rpc) throws Ex context.assertElectedLeader(epoch, otherNodeId); context.pollUntilRequest(); - context.assertSentFetchRequest(epoch, 1L, lastEpoch); + context.assertSentFetchRequest(epoch, 1L, lastEpoch, OptionalLong.empty()); } @ParameterizedTest @@ -1926,7 +1926,7 @@ public void testVoterBecomeProspectiveAfterFetchTimeout(boolean withKip853Rpc) t context.assertElectedLeader(epoch, otherNodeId); context.pollUntilRequest(); - context.assertSentFetchRequest(epoch, 1L, lastEpoch); + context.assertSentFetchRequest(epoch, 1L, lastEpoch, OptionalLong.empty()); context.time.sleep(context.fetchTimeoutMs); context.client.poll(); @@ -1952,7 +1952,7 @@ public void testFollowerAsObserverDoesNotBecomeProspectiveAfterFetchTimeout(bool context.assertElectedLeader(epoch, otherNodeId); context.pollUntilRequest(); - context.assertSentFetchRequest(epoch, 1L, lastEpoch); + context.assertSentFetchRequest(epoch, 1L, lastEpoch, OptionalLong.empty()); context.time.sleep(context.fetchTimeoutMs); context.pollUntilRequest(); @@ -1979,13 +1979,13 @@ public void testUnattachedAsObserverDoesNotBecomeProspectiveAfterElectionTimeout .build(); context.pollUntilRequest(); - context.assertSentFetchRequest(epoch, 0L, 0); + context.assertSentFetchRequest(epoch, 0L, 0, OptionalLong.empty()); assertTrue(context.client.quorum().isUnattached()); context.time.sleep(context.electionTimeoutMs() * 2); context.pollUntilRequest(); assertTrue(context.client.quorum().isUnattached()); - context.assertSentFetchRequest(epoch, 0L, 0); + context.assertSentFetchRequest(epoch, 0L, 0, OptionalLong.empty()); // confirm no vote request was sent assertEquals(0, context.channel.drainSendQueue().size()); @@ -1998,7 +1998,7 @@ public void testUnattachedAsObserverDoesNotBecomeProspectiveAfterElectionTimeout context.pollUntilRequest(); // observer cannot transition to prospective though assertTrue(context.client.quorum().isUnattached()); - context.assertSentFetchRequest(epoch + 1, 0L, 0); + context.assertSentFetchRequest(epoch + 1, 0L, 0, OptionalLong.empty()); assertEquals(0, context.channel.drainSendQueue().size()); } @@ -2017,7 +2017,7 @@ public void testUnattachedAsVoterCanBecomeFollowerAfterFindingLeader(boolean wit .build(); context.pollUntilRequest(); - RaftRequest.Outbound request = context.assertSentFetchRequest(epoch, 0L, 0); + RaftRequest.Outbound request = context.assertSentFetchRequest(epoch, 0L, 0, OptionalLong.empty()); assertTrue(context.client.quorum().isUnattached()); assertTrue(context.client.quorum().isVoter()); @@ -2049,7 +2049,7 @@ public void testInitializeObserverNoPreviousState(boolean withKip853Rpc) throws context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); assertTrue(voters.contains(fetchRequest.destination().id())); - context.assertFetchRequestData(fetchRequest, 0, 0L, 0); + context.assertFetchRequestData(fetchRequest, 0, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest.correlationId(), @@ -2081,7 +2081,7 @@ public void testObserverQuorumDiscoveryFailure(boolean withKip853Rpc) throws Exc context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); assertTrue(context.bootstrapIds.contains(fetchRequest.destination().id())); - context.assertFetchRequestData(fetchRequest, 0, 0L, 0); + context.assertFetchRequestData(fetchRequest, 0, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest.correlationId(), @@ -2095,7 +2095,7 @@ public void testObserverQuorumDiscoveryFailure(boolean withKip853Rpc) throws Exc fetchRequest = context.assertSentFetchRequest(); assertTrue(context.bootstrapIds.contains(fetchRequest.destination().id())); - context.assertFetchRequestData(fetchRequest, 0, 0L, 0); + context.assertFetchRequestData(fetchRequest, 0, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest.correlationId(), @@ -2128,7 +2128,7 @@ public void testObserverSendDiscoveryFetchAfterFetchTimeout(boolean withKip853Rp context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); assertTrue(context.bootstrapIds.contains(fetchRequest.destination().id())); - context.assertFetchRequestData(fetchRequest, 0, 0L, 0); + context.assertFetchRequestData(fetchRequest, 0, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest.correlationId(), @@ -2145,7 +2145,7 @@ public void testObserverSendDiscoveryFetchAfterFetchTimeout(boolean withKip853Rp fetchRequest = context.assertSentFetchRequest(); assertNotEquals(leaderId, fetchRequest.destination().id()); assertTrue(context.bootstrapIds.contains(fetchRequest.destination().id())); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); } @ParameterizedTest @@ -2173,7 +2173,7 @@ public void testObserverHandleRetryFetchToBootstrapServer(boolean withKip853Rpc) RaftRequest.Outbound discoveryFetchRequest = context.assertSentFetchRequest(); assertFalse(voters.contains(discoveryFetchRequest.destination().id())); assertTrue(context.bootstrapIds.contains(discoveryFetchRequest.destination().id())); - context.assertFetchRequestData(discoveryFetchRequest, 0, 0L, 0); + context.assertFetchRequestData(discoveryFetchRequest, 0, 0L, 0, context.client.highWatermark()); // Send a response with the leader and epoch context.deliverResponse( @@ -2189,7 +2189,7 @@ public void testObserverHandleRetryFetchToBootstrapServer(boolean withKip853Rpc) context.pollUntilRequest(); RaftRequest.Outbound toLeaderFetchRequest = context.assertSentFetchRequest(); assertEquals(leaderId, toLeaderFetchRequest.destination().id()); - context.assertFetchRequestData(toLeaderFetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(toLeaderFetchRequest, epoch, 0L, 0, context.client.highWatermark()); context.time.sleep(context.requestTimeoutMs()); @@ -2198,7 +2198,7 @@ public void testObserverHandleRetryFetchToBootstrapServer(boolean withKip853Rpc) RaftRequest.Outbound retryToBootstrapServerFetchRequest = context.assertSentFetchRequest(); assertFalse(voters.contains(retryToBootstrapServerFetchRequest.destination().id())); assertTrue(context.bootstrapIds.contains(retryToBootstrapServerFetchRequest.destination().id())); - context.assertFetchRequestData(retryToBootstrapServerFetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(retryToBootstrapServerFetchRequest, epoch, 0L, 0, context.client.highWatermark()); // Deliver the delayed responses from the leader Records records = context.buildBatch(0L, 3, List.of("a", "b")); @@ -2247,7 +2247,7 @@ public void testObserverHandleRetryFetchToLeader(boolean withKip853Rpc) throws E RaftRequest.Outbound discoveryFetchRequest = context.assertSentFetchRequest(); assertFalse(voters.contains(discoveryFetchRequest.destination().id())); assertTrue(context.bootstrapIds.contains(discoveryFetchRequest.destination().id())); - context.assertFetchRequestData(discoveryFetchRequest, 0, 0L, 0); + context.assertFetchRequestData(discoveryFetchRequest, 0, 0L, 0, context.client.highWatermark()); // Send a response with the leader and epoch context.deliverResponse( @@ -2263,7 +2263,7 @@ public void testObserverHandleRetryFetchToLeader(boolean withKip853Rpc) throws E context.pollUntilRequest(); RaftRequest.Outbound toLeaderFetchRequest = context.assertSentFetchRequest(); assertEquals(leaderId, toLeaderFetchRequest.destination().id()); - context.assertFetchRequestData(toLeaderFetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(toLeaderFetchRequest, epoch, 0L, 0, context.client.highWatermark()); context.time.sleep(context.requestTimeoutMs()); @@ -2272,7 +2272,7 @@ public void testObserverHandleRetryFetchToLeader(boolean withKip853Rpc) throws E RaftRequest.Outbound retryToBootstrapServerFetchRequest = context.assertSentFetchRequest(); assertFalse(voters.contains(retryToBootstrapServerFetchRequest.destination().id())); assertTrue(context.bootstrapIds.contains(retryToBootstrapServerFetchRequest.destination().id())); - context.assertFetchRequestData(retryToBootstrapServerFetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(retryToBootstrapServerFetchRequest, epoch, 0L, 0, context.client.highWatermark()); // At this point toLeaderFetchRequest has timed out but retryToBootstrapServerFetchRequest // is still waiting for a response. @@ -2378,17 +2378,23 @@ public void testFetchRequestClusterIdValidation(boolean withKip853Rpc) throws Ex context.assertSentFetchPartitionResponse(Errors.INVALID_REQUEST, epoch, OptionalInt.of(localId)); // null cluster id is accepted - context.deliverRequest(context.fetchRequest(epoch, null, otherNodeKey, -5L, 0, 0)); + context.deliverRequest( + context.fetchRequest(epoch, null, otherNodeKey, -5L, 0, OptionalLong.of(Long.MAX_VALUE), 0) + ); context.pollUntilResponse(); context.assertSentFetchPartitionResponse(Errors.INVALID_REQUEST, epoch, OptionalInt.of(localId)); // empty cluster id is rejected - context.deliverRequest(context.fetchRequest(epoch, "", otherNodeKey, -5L, 0, 0)); + context.deliverRequest( + context.fetchRequest(epoch, "", otherNodeKey, -5L, 0, OptionalLong.of(Long.MAX_VALUE), 0) + ); context.pollUntilResponse(); context.assertSentFetchPartitionResponse(Errors.INCONSISTENT_CLUSTER_ID); // invalid cluster id is rejected - context.deliverRequest(context.fetchRequest(epoch, "invalid-uuid", otherNodeKey, -5L, 0, 0)); + context.deliverRequest( + context.fetchRequest(epoch, "invalid-uuid", otherNodeKey, -5L, 0, OptionalLong.of(Long.MAX_VALUE), 0) + ); context.pollUntilResponse(); context.assertSentFetchPartitionResponse(Errors.INCONSISTENT_CLUSTER_ID); } @@ -2778,7 +2784,7 @@ public void testFetchResponseIgnoredAfterBecomingProspective(boolean withKip853R // Wait until we have a Fetch inflight to the leader context.pollUntilRequest(); - RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(epoch, 0L, 0); + RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(epoch, 0L, 0, OptionalLong.empty()); // Now await the fetch timeout and become prospective context.time.sleep(context.fetchTimeoutMs); @@ -2818,7 +2824,7 @@ public void testFetchResponseIgnoredAfterBecomingFollowerOfDifferentLeader( // Wait until we have a Fetch inflight to the leader context.pollUntilRequest(); - RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(epoch, 0L, 0); + RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(epoch, 0L, 0, OptionalLong.empty()); // Now receive a BeginEpoch from `voter3` context.deliverRequest(context.beginEpochRequest(epoch + 1, voter3)); @@ -2915,7 +2921,7 @@ public void testFollowerLeaderRediscoveryAfterBrokerNotAvailableError(boolean wi context.pollUntilRequest(); RaftRequest.Outbound fetchRequest1 = context.assertSentFetchRequest(); assertEquals(leaderId, fetchRequest1.destination().id()); - context.assertFetchRequestData(fetchRequest1, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest1, epoch, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest1.correlationId(), @@ -2929,7 +2935,7 @@ public void testFollowerLeaderRediscoveryAfterBrokerNotAvailableError(boolean wi RaftRequest.Outbound fetchRequest2 = context.assertSentFetchRequest(); assertNotEquals(leaderId, fetchRequest2.destination().id()); assertTrue(context.bootstrapIds.contains(fetchRequest2.destination().id())); - context.assertFetchRequestData(fetchRequest2, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest2, epoch, 0L, 0, context.client.highWatermark()); } @ParameterizedTest @@ -2954,7 +2960,7 @@ public void testFollowerLeaderRediscoveryAfterRequestTimeout(boolean withKip853R context.pollUntilRequest(); RaftRequest.Outbound fetchRequest1 = context.assertSentFetchRequest(); assertEquals(leaderId, fetchRequest1.destination().id()); - context.assertFetchRequestData(fetchRequest1, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest1, epoch, 0L, 0, context.client.highWatermark()); context.time.sleep(context.requestTimeoutMs()); context.pollUntilRequest(); @@ -2964,7 +2970,7 @@ public void testFollowerLeaderRediscoveryAfterRequestTimeout(boolean withKip853R RaftRequest.Outbound fetchRequest2 = context.assertSentFetchRequest(); assertNotEquals(leaderId, fetchRequest2.destination().id()); assertTrue(context.bootstrapIds.contains(fetchRequest2.destination().id())); - context.assertFetchRequestData(fetchRequest2, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest2, epoch, 0L, 0, context.client.highWatermark()); } @ParameterizedTest @@ -2985,12 +2991,12 @@ public void testObserverLeaderRediscoveryAfterBrokerNotAvailableError(boolean wi .withKip853Rpc(withKip853Rpc) .build(); - context.discoverLeaderAsObserver(leaderId, epoch); + context.discoverLeaderAsObserver(leaderId, epoch, context.client.highWatermark()); context.pollUntilRequest(); RaftRequest.Outbound fetchRequest1 = context.assertSentFetchRequest(); assertEquals(leaderId, fetchRequest1.destination().id()); - context.assertFetchRequestData(fetchRequest1, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest1, epoch, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest1.correlationId(), @@ -3004,7 +3010,7 @@ public void testObserverLeaderRediscoveryAfterBrokerNotAvailableError(boolean wi RaftRequest.Outbound fetchRequest2 = context.assertSentFetchRequest(); assertNotEquals(leaderId, fetchRequest2.destination().id()); assertTrue(context.bootstrapIds.contains(fetchRequest2.destination().id())); - context.assertFetchRequestData(fetchRequest2, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest2, epoch, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest2.correlationId(), @@ -3034,12 +3040,12 @@ public void testObserverLeaderRediscoveryAfterRequestTimeout(boolean withKip853R .withKip853Rpc(withKip853Rpc) .build(); - context.discoverLeaderAsObserver(leaderId, epoch); + context.discoverLeaderAsObserver(leaderId, epoch, context.client.highWatermark()); context.pollUntilRequest(); RaftRequest.Outbound fetchRequest1 = context.assertSentFetchRequest(); assertEquals(leaderId, fetchRequest1.destination().id()); - context.assertFetchRequestData(fetchRequest1, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest1, epoch, 0L, 0, context.client.highWatermark()); context.time.sleep(context.requestTimeoutMs()); context.pollUntilRequest(); @@ -3049,7 +3055,7 @@ public void testObserverLeaderRediscoveryAfterRequestTimeout(boolean withKip853R RaftRequest.Outbound fetchRequest2 = context.assertSentFetchRequest(); assertNotEquals(leaderId, fetchRequest2.destination().id()); assertTrue(context.bootstrapIds.contains(fetchRequest2.destination().id())); - context.assertFetchRequestData(fetchRequest2, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest2, epoch, 0L, 0, context.client.highWatermark()); context.deliverResponse( fetchRequest2.correlationId(), @@ -3727,7 +3733,7 @@ public void testFollowerReplication(boolean withKip853Rpc) throws Exception { context.pollUntilRequest(); - RaftRequest.Outbound fetchQuorumRequest = context.assertSentFetchRequest(epoch, 0L, 0); + RaftRequest.Outbound fetchQuorumRequest = context.assertSentFetchRequest(epoch, 0L, 0, OptionalLong.empty()); Records records = context.buildBatch(0L, 3, List.of("a", "b")); FetchResponseData response = context.fetchResponse(epoch, otherNodeId, records, 0L, Errors.NONE); context.deliverResponse( @@ -3758,7 +3764,7 @@ public void testObserverReplication(boolean withKip853Rpc, boolean alwaysFlush) context.pollUntilRequest(); - RaftRequest.Outbound fetchQuorumRequest = context.assertSentFetchRequest(epoch, 0L, 0); + RaftRequest.Outbound fetchQuorumRequest = context.assertSentFetchRequest(epoch, 0L, 0, OptionalLong.empty()); Records records = context.buildBatch(0L, 3, List.of("a", "b")); FetchResponseData response = context.fetchResponse(epoch, otherNodeId, records, 0L, Errors.NONE); context.deliverResponse( @@ -3789,7 +3795,7 @@ public void testEmptyRecordSetInFetchResponse(boolean withKip853Rpc) throws Exce // Receive an empty fetch response context.pollUntilRequest(); - RaftRequest.Outbound fetchQuorumRequest = context.assertSentFetchRequest(epoch, 0L, 0); + RaftRequest.Outbound fetchQuorumRequest = context.assertSentFetchRequest(epoch, 0L, 0, OptionalLong.empty()); FetchResponseData fetchResponse = context.fetchResponse( epoch, otherNodeId, @@ -3809,7 +3815,7 @@ public void testEmptyRecordSetInFetchResponse(boolean withKip853Rpc) throws Exce // Receive some records in the next poll, but do not advance high watermark context.pollUntilRequest(); Records records = context.buildBatch(0L, epoch, List.of("a", "b")); - fetchQuorumRequest = context.assertSentFetchRequest(epoch, 0L, 0); + fetchQuorumRequest = context.assertSentFetchRequest(epoch, 0L, 0, OptionalLong.of(0)); fetchResponse = context.fetchResponse(epoch, otherNodeId, records, 0L, Errors.NONE); context.deliverResponse( fetchQuorumRequest.correlationId(), @@ -3822,7 +3828,7 @@ public void testEmptyRecordSetInFetchResponse(boolean withKip853Rpc) throws Exce // The next fetch response is empty, but should still advance the high watermark context.pollUntilRequest(); - fetchQuorumRequest = context.assertSentFetchRequest(epoch, 2L, epoch); + fetchQuorumRequest = context.assertSentFetchRequest(epoch, 2L, epoch, OptionalLong.of(0)); fetchResponse = context.fetchResponse( epoch, otherNodeId, @@ -3974,10 +3980,20 @@ public void testFollowerLogReconciliation(boolean withKip853Rpc) throws Exceptio context.pollUntilRequest(); - RaftRequest.Outbound request = context.assertSentFetchRequest(epoch, 3L, lastEpoch); + RaftRequest.Outbound request = context.assertSentFetchRequest( + epoch, + 3L, + lastEpoch, + OptionalLong.empty() + ); - FetchResponseData response = context.divergingFetchResponse(epoch, otherNodeId, 2L, - lastEpoch, 1L); + FetchResponseData response = context.divergingFetchResponse( + epoch, + otherNodeId, + 2L, + lastEpoch, + 1L + ); context.deliverResponse(request.correlationId(), request.destination(), response); // Poll again to complete truncation @@ -3987,7 +4003,7 @@ public void testFollowerLogReconciliation(boolean withKip853Rpc) throws Exceptio // Now we should be fetching context.client.poll(); - context.assertSentFetchRequest(epoch, 2L, lastEpoch); + context.assertSentFetchRequest(epoch, 2L, lastEpoch, context.client.highWatermark()); } @ParameterizedTest @@ -4255,7 +4271,7 @@ public void testHandleCommitCallbackFiresAfterFollowerHighWatermarkAdvances(bool context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); assertTrue(voters.contains(fetchRequest.destination().id())); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); // The response does not advance the high watermark List records1 = List.of("a", "b", "c"); @@ -4276,7 +4292,7 @@ public void testHandleCommitCallbackFiresAfterFollowerHighWatermarkAdvances(bool context.pollUntilRequest(); fetchRequest = context.assertSentFetchRequest(); assertTrue(voters.contains(fetchRequest.destination().id())); - context.assertFetchRequestData(fetchRequest, epoch, 3L, 3); + context.assertFetchRequestData(fetchRequest, epoch, 3L, 3, context.client.highWatermark()); // The high watermark advances to include the first batch we fetched List records2 = List.of("d", "e", "f"); @@ -4514,7 +4530,7 @@ public void testObserverFetchWithNoLocalId(boolean withKip853Rpc) throws Excepti context.pollUntilRequest(); RaftRequest.Outbound fetchRequest1 = context.assertSentFetchRequest(); assertTrue(context.bootstrapIds.contains(fetchRequest1.destination().id())); - context.assertFetchRequestData(fetchRequest1, 0, 0L, 0); + context.assertFetchRequestData(fetchRequest1, 0, 0L, 0, context.client.highWatermark()); int leaderEpoch = 5; @@ -4531,7 +4547,7 @@ public void testObserverFetchWithNoLocalId(boolean withKip853Rpc) throws Excepti context.pollUntilRequest(); RaftRequest.Outbound fetchRequest2 = context.assertSentFetchRequest(); assertEquals(leaderId, fetchRequest2.destination().id()); - context.assertFetchRequestData(fetchRequest2, leaderEpoch, 0L, 0); + context.assertFetchRequestData(fetchRequest2, leaderEpoch, 0L, 0, context.client.highWatermark()); List records = List.of("a", "b", "c"); MemoryRecords batch1 = context.buildBatch(0L, 3, records); diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index 9f0a5084326c8..b84c603ac9fb8 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -957,7 +957,7 @@ List assertSentBeginQuorumEpochRequest(int epoch, Set drainSentResponses( + List drainSentResponses( ApiKeys apiKey ) { List res = new ArrayList<>(); @@ -1114,23 +1114,20 @@ RaftRequest.Outbound assertSentFetchRequest() { RaftRequest.Outbound assertSentFetchRequest( int epoch, long fetchOffset, - int lastFetchedEpoch + int lastFetchedEpoch, + OptionalLong highWatermark ) { List sentMessages = channel.drainSendQueue(); assertEquals(1, sentMessages.size()); RaftRequest.Outbound raftRequest = sentMessages.get(0); - assertFetchRequestData(raftRequest, epoch, fetchOffset, lastFetchedEpoch); + assertFetchRequestData(raftRequest, epoch, fetchOffset, lastFetchedEpoch, highWatermark); return raftRequest; } - FetchResponseData.PartitionData assertSentFetchPartitionResponse() { - List sentMessages = drainSentResponses(ApiKeys.FETCH); - assertEquals( - 1, sentMessages.size(), "Found unexpected sent messages " + sentMessages); - RaftResponse.Outbound raftMessage = sentMessages.get(0); - assertEquals(ApiKeys.FETCH.id, raftMessage.data().apiKey()); - FetchResponseData response = (FetchResponseData) raftMessage.data(); + FetchResponseData.PartitionData assertFetchResponseData(RaftResponse.Outbound message) { + assertEquals(ApiKeys.FETCH.id, message.data().apiKey()); + FetchResponseData response = (FetchResponseData) message.data(); assertEquals(Errors.NONE, Errors.forCode(response.errorCode())); assertEquals(1, response.responses().size()); @@ -1152,17 +1149,30 @@ FetchResponseData.PartitionData assertSentFetchPartitionResponse() { return partitionResponse; } + FetchResponseData.PartitionData assertSentFetchPartitionResponse() { + List sentMessages = drainSentResponses(ApiKeys.FETCH); + assertEquals( + 1, + sentMessages.size(), + "Found unexpected sent messages " + sentMessages + ); + + return assertFetchResponseData(sentMessages.get(0)); + } + void assertSentFetchPartitionResponse(Errors topLevelError) { List sentMessages = drainSentResponses(ApiKeys.FETCH); assertEquals( - 1, sentMessages.size(), "Found unexpected sent messages " + sentMessages); + 1, + sentMessages.size(), + "Found unexpected sent messages " + sentMessages + ); RaftResponse.Outbound raftMessage = sentMessages.get(0); assertEquals(ApiKeys.FETCH.id, raftMessage.data().apiKey()); FetchResponseData response = (FetchResponseData) raftMessage.data(); assertEquals(topLevelError, Errors.forCode(response.errorCode())); } - MemoryRecords assertSentFetchPartitionResponse( Errors error, int epoch, @@ -1375,7 +1385,8 @@ List collectEndQuorumRequests( void discoverLeaderAsObserver( int leaderId, - int epoch + int epoch, + OptionalLong highWatermark ) throws Exception { pollUntilRequest(); RaftRequest.Outbound fetchRequest = assertSentFetchRequest(); @@ -1384,7 +1395,7 @@ void discoverLeaderAsObserver( startingVoters.voterIds().contains(destinationId) || bootstrapIds.contains(destinationId), String.format("id %d is not in sets %s or %s", destinationId, startingVoters, bootstrapIds) ); - assertFetchRequestData(fetchRequest, 0, 0L, 0); + assertFetchRequestData(fetchRequest, 0, 0L, 0, highWatermark); deliverResponse( fetchRequest.correlationId(), @@ -1672,7 +1683,8 @@ void assertFetchRequestData( RaftRequest.Outbound message, int epoch, long fetchOffset, - int lastFetchedEpoch + int lastFetchedEpoch, + OptionalLong highWatermark ) { assertInstanceOf( FetchRequestData.class, @@ -1691,6 +1703,7 @@ void assertFetchRequestData( assertEquals(fetchOffset, fetchPartition.fetchOffset()); assertEquals(lastFetchedEpoch, fetchPartition.lastFetchedEpoch()); assertEquals(localId.orElse(-1), request.replicaState().replicaId()); + assertEquals(highWatermark.orElse(-1), fetchPartition.highWatermark()); // Assert that voters have flushed up to the fetch offset if ((localId.isPresent() && startingVoters.voterIds().contains(localId.getAsInt())) || @@ -1716,6 +1729,24 @@ FetchRequestData fetchRequest( long fetchOffset, int lastFetchedEpoch, int maxWaitTimeMs + ) { + return fetchRequest( + epoch, + replicaKey, + fetchOffset, + lastFetchedEpoch, + OptionalLong.of(Long.MAX_VALUE), + maxWaitTimeMs + ); + } + + FetchRequestData fetchRequest( + int epoch, + ReplicaKey replicaKey, + long fetchOffset, + int lastFetchedEpoch, + OptionalLong highWatermark, + int maxWaitTimeMs ) { return fetchRequest( epoch, @@ -1723,6 +1754,7 @@ FetchRequestData fetchRequest( replicaKey, fetchOffset, lastFetchedEpoch, + highWatermark, maxWaitTimeMs ); } @@ -1733,6 +1765,7 @@ FetchRequestData fetchRequest( ReplicaKey replicaKey, long fetchOffset, int lastFetchedEpoch, + OptionalLong highWatermark, int maxWaitTimeMs ) { FetchRequestData request = RaftUtil.singletonFetchRequest( @@ -1742,7 +1775,8 @@ FetchRequestData fetchRequest( fetchPartition .setCurrentLeaderEpoch(epoch) .setLastFetchedEpoch(lastFetchedEpoch) - .setFetchOffset(fetchOffset); + .setFetchOffset(fetchOffset) + .setHighWatermark(highWatermark.orElse(-1)); if (raftProtocol.isReconfigSupported()) { fetchPartition .setReplicaDirectoryId(replicaKey.directoryId().orElse(ReplicaKey.NO_DIRECTORY_ID)); @@ -1932,7 +1966,9 @@ UpdateRaftVoterResponseData updateVoterResponse( } private short fetchRpcVersion() { - if (raftProtocol.isReconfigSupported()) { + if (raftProtocol.isHwmInFetchSupported()) { + return 18; + } else if (raftProtocol.isReconfigSupported()) { return 17; } else { return 16; @@ -2236,7 +2272,9 @@ enum RaftProtocol { // dynamic quorum reconfiguration support KIP_853_PROTOCOL, // preVote support - KIP_996_PROTOCOL; + KIP_996_PROTOCOL, + // HWM in FETCH request support + KIP_1166_PROTOCOL; boolean isKRaftSupported() { return isAtLeast(KIP_595_PROTOCOL); @@ -2250,6 +2288,10 @@ boolean isPreVoteSupported() { return isAtLeast(KIP_996_PROTOCOL); } + boolean isHwmInFetchSupported() { + return isAtLeast(KIP_1166_PROTOCOL); + } + private boolean isAtLeast(RaftProtocol otherRpc) { return this.compareTo(otherRpc) >= 0; } diff --git a/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java index 9e9ed6f8447e6..dd7c5937bdc9a 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java @@ -105,17 +105,20 @@ public enum MetadataVersion { // Enables async remote LIST_OFFSETS support (KIP-1075) IBP_4_0_IV3(25, "4.0", "IV3", false), + // Enables ELR by default for new clusters (KIP-966). + // Share groups are preview in 4.1 (KIP-932). + // Streams groups are early access in 4.1 (KIP-1071). + IBP_4_1_IV0(26, "4.1", "IV0", false), + + // Send FETCH version 18 in the replica fetcher (KIP-1166) + IBP_4_1_IV1(27, "4.1", "IV1", false), + // // NOTE: MetadataVersions after this point are unstable and may be changed. // If users attempt to use an unstable MetadataVersion, they will get an error. // Please move this comment when updating the LATEST_PRODUCTION constant. // - // Enables ELR by default for new clusters (KIP-966). - // Share groups are preview in 4.1 (KIP-932). - // Streams groups are early access in 4.1 (KIP-1071). - IBP_4_1_IV0(26, "4.1", "IV0", false), - // Insert any additional IBP_4_1_IVx versions above this comment, and bump the feature level of // IBP_4_2_IVx accordingly. When 4.2 development begins, IBP_4_2_IV0 will cease to be // a placeholder. @@ -126,7 +129,7 @@ public enum MetadataVersion { // *** SHARE GROUPS BECOME PRODUCTION-READY IN THE FUTURE. ITS DEFINITION ALLOWS A SHARE *** // *** GROUPS FEATURE TO BE DEFINED IN 4.1 BUT TURNED OFF BY DEFAULT, ABLE TO BE TURNED ON *** // *** DYNAMICALLY TO TRY OUT THE PREVIEW CAPABILITY. *** - IBP_4_2_IV0(27, "4.2", "IV0", false), + IBP_4_2_IV0(28, "4.2", "IV0", false), // Enables "streams" groups by default for new clusters (KIP-1071). // @@ -134,7 +137,7 @@ public enum MetadataVersion { // *** STREAMS GROUPS BECOME PRODUCTION-READY IN THE FUTURE. ITS DEFINITION ALLOWS A STREAMS *** // *** GROUPS FEATURE TO BE DEFINED IN 4.1 BUT TURNED OFF BY DEFAULT, ABLE TO BE TURNED ON *** // *** DYNAMICALLY TO TRY OUT THE EARLY ACCESS CAPABILITY. *** - IBP_4_2_IV1(28, "4.2", "IV1", false); + IBP_4_2_IV1(29, "4.2", "IV1", false); // NOTES when adding a new version: // Update the default version in @ClusterTest annotation to point to the latest version @@ -154,7 +157,7 @@ public enum MetadataVersion { * Think carefully before you update this value. ONCE A METADATA VERSION IS PRODUCTION, * IT CANNOT BE CHANGED. */ - public static final MetadataVersion LATEST_PRODUCTION = IBP_4_0_IV3; + public static final MetadataVersion LATEST_PRODUCTION = IBP_4_1_IV1; // If you change the value above please also update // LATEST_STABLE_METADATA_VERSION version in tests/kafkatest/version.py @@ -264,13 +267,15 @@ public short partitionRecordVersion() { } public short fetchRequestVersion() { - if (this.isAtLeast(IBP_3_9_IV0)) { + if (isAtLeast(IBP_4_1_IV1)) { + return 18; + } else if (isAtLeast(IBP_3_9_IV0)) { return 17; - } else if (this.isAtLeast(IBP_3_7_IV4)) { + } else if (isAtLeast(IBP_3_7_IV4)) { return 16; - } else if (this.isAtLeast(IBP_3_5_IV1)) { + } else if (isAtLeast(IBP_3_5_IV1)) { return 15; - } else if (this.isAtLeast(IBP_3_5_IV0)) { + } else if (isAtLeast(IBP_3_5_IV0)) { return 14; } else { return 13; diff --git a/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java b/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java index 0414f7cd1cc9d..cdc66b8b52111 100644 --- a/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java +++ b/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java @@ -78,12 +78,16 @@ public void testFromVersionString() { assertEquals(IBP_3_9_IV0, MetadataVersion.fromVersionString("3.9-IV0")); // 4.0-IV3 is the latest production version in the 4.0 line + assertEquals(IBP_4_0_IV3, MetadataVersion.fromVersionString("4.0")); assertEquals(IBP_4_0_IV0, MetadataVersion.fromVersionString("4.0-IV0")); assertEquals(IBP_4_0_IV1, MetadataVersion.fromVersionString("4.0-IV1")); assertEquals(IBP_4_0_IV2, MetadataVersion.fromVersionString("4.0-IV2")); assertEquals(IBP_4_0_IV3, MetadataVersion.fromVersionString("4.0-IV3")); + // 4.1-IV1 is the latest production version in the 4.1 line + assertEquals(IBP_4_1_IV1, MetadataVersion.fromVersionString("4.1")); assertEquals(IBP_4_1_IV0, MetadataVersion.fromVersionString("4.1-IV0")); + assertEquals(IBP_4_1_IV1, MetadataVersion.fromVersionString("4.1-IV1")); } @Test diff --git a/server/src/main/java/org/apache/kafka/server/ClientMetricsManager.java b/server/src/main/java/org/apache/kafka/server/ClientMetricsManager.java index b143466574c3d..d7343098b4e2a 100644 --- a/server/src/main/java/org/apache/kafka/server/ClientMetricsManager.java +++ b/server/src/main/java/org/apache/kafka/server/ClientMetricsManager.java @@ -391,7 +391,7 @@ private void validateGetRequest(GetTelemetrySubscriptionsRequest request, ClientMetricsInstance clientInstance, long timestamp) { if (!clientInstance.maybeUpdateGetRequestTimestamp(timestamp) && (clientInstance.lastKnownError() != Errors.UNKNOWN_SUBSCRIPTION_ID - || clientInstance.lastKnownError() != Errors.UNSUPPORTED_COMPRESSION_TYPE)) { + && clientInstance.lastKnownError() != Errors.UNSUPPORTED_COMPRESSION_TYPE)) { clientMetricsStats.recordThrottleCount(clientInstance.clientInstanceId()); String msg = String.format("Request from the client [%s] arrived before the next push interval time", request.data().clientInstanceId()); diff --git a/server/src/main/java/org/apache/kafka/server/share/fetch/DelayedShareFetchPartitionKey.java b/server/src/main/java/org/apache/kafka/server/share/fetch/DelayedShareFetchPartitionKey.java index 584613cde17e4..c1e40975c2bce 100644 --- a/server/src/main/java/org/apache/kafka/server/share/fetch/DelayedShareFetchPartitionKey.java +++ b/server/src/main/java/org/apache/kafka/server/share/fetch/DelayedShareFetchPartitionKey.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.server.share.fetch; +import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.Uuid; import java.util.Objects; @@ -27,6 +28,10 @@ public class DelayedShareFetchPartitionKey implements DelayedShareFetchKey { private final Uuid topicId; private final int partition; + public DelayedShareFetchPartitionKey(TopicIdPartition topicIdPartition) { + this(topicIdPartition.topicId(), topicIdPartition.partition()); + } + public DelayedShareFetchPartitionKey(Uuid topicId, int partition) { this.topicId = topicId; this.partition = partition; diff --git a/server/src/test/java/org/apache/kafka/server/ClientMetricsManagerTest.java b/server/src/test/java/org/apache/kafka/server/ClientMetricsManagerTest.java index 3b0bd181b9182..c5b13eefbe45b 100644 --- a/server/src/test/java/org/apache/kafka/server/ClientMetricsManagerTest.java +++ b/server/src/test/java/org/apache/kafka/server/ClientMetricsManagerTest.java @@ -826,7 +826,6 @@ public void testPushTelemetrySubscriptionIdInvalid() throws Exception { assertEquals((double) 0, getMetric(ClientMetricsManager.ClientMetricsStats.PLUGIN_ERROR + "-count").metricValue()); assertEquals(Double.NaN, getMetric(ClientMetricsManager.ClientMetricsStats.PLUGIN_EXPORT_TIME + "-avg").metricValue()); assertEquals(Double.NaN, getMetric(ClientMetricsManager.ClientMetricsStats.PLUGIN_EXPORT_TIME + "-max").metricValue()); - } @Test @@ -1124,6 +1123,115 @@ public void testPushTelemetryPluginException() throws Exception { } } + @Test + public void testGetTelemetrySubscriptionAfterPushTelemetryUnknownSubscriptionSucceeds() throws Exception { + clientMetricsManager.updateSubscription("sub-1", ClientMetricsTestUtils.defaultTestProperties()); + assertEquals(1, clientMetricsManager.subscriptions().size()); + + GetTelemetrySubscriptionsRequest subscriptionsRequest = new GetTelemetrySubscriptionsRequest.Builder( + new GetTelemetrySubscriptionsRequestData(), true).build(); + + GetTelemetrySubscriptionsResponse subscriptionsResponse = clientMetricsManager.processGetTelemetrySubscriptionRequest( + subscriptionsRequest, ClientMetricsTestUtils.requestContext()); + + Properties properties = new Properties(); + properties.put("interval.ms", "100"); + clientMetricsManager.updateSubscription("sub-2", properties); + assertEquals(2, clientMetricsManager.subscriptions().size()); + + PushTelemetryRequest request = new Builder( + new PushTelemetryRequestData() + .setClientInstanceId(subscriptionsResponse.data().clientInstanceId()) + .setSubscriptionId(subscriptionsResponse.data().subscriptionId()) + .setCompressionType(CompressionType.NONE.id) + .setMetrics(ByteBuffer.wrap("test-data".getBytes(StandardCharsets.UTF_8))), true).build(); + + PushTelemetryResponse response = clientMetricsManager.processPushTelemetryRequest( + request, ClientMetricsTestUtils.requestContext()); + + assertEquals(Errors.UNKNOWN_SUBSCRIPTION_ID, response.error()); + ClientMetricsInstance instance = clientMetricsManager.clientInstance(subscriptionsResponse.data().clientInstanceId()); + assertNotNull(instance); + assertEquals(Errors.UNKNOWN_SUBSCRIPTION_ID, instance.lastKnownError()); + + subscriptionsRequest = new GetTelemetrySubscriptionsRequest.Builder( + new GetTelemetrySubscriptionsRequestData().setClientInstanceId(subscriptionsResponse.data().clientInstanceId()), true).build(); + subscriptionsResponse = clientMetricsManager.processGetTelemetrySubscriptionRequest( + subscriptionsRequest, ClientMetricsTestUtils.requestContext()); + assertEquals(Errors.NONE, subscriptionsResponse.error()); + } + + @Test + public void testGetTelemetrySubscriptionAfterPushTelemetryUnknownCompressionSucceeds() throws Exception { + clientMetricsManager.updateSubscription("sub-1", ClientMetricsTestUtils.defaultTestProperties()); + assertEquals(1, clientMetricsManager.subscriptions().size()); + + GetTelemetrySubscriptionsRequest subscriptionsRequest = new GetTelemetrySubscriptionsRequest.Builder( + new GetTelemetrySubscriptionsRequestData(), true).build(); + + GetTelemetrySubscriptionsResponse subscriptionsResponse = clientMetricsManager.processGetTelemetrySubscriptionRequest( + subscriptionsRequest, ClientMetricsTestUtils.requestContext()); + + PushTelemetryRequest request = new Builder( + new PushTelemetryRequestData() + .setClientInstanceId(subscriptionsResponse.data().clientInstanceId()) + .setSubscriptionId(subscriptionsResponse.data().subscriptionId()) + .setCompressionType((byte) 10) // // Invalid compression type + .setMetrics(ByteBuffer.wrap("test-data".getBytes(StandardCharsets.UTF_8))), true).build(); + + PushTelemetryResponse response = clientMetricsManager.processPushTelemetryRequest( + request, ClientMetricsTestUtils.requestContext()); + + assertEquals(Errors.UNSUPPORTED_COMPRESSION_TYPE, response.error()); + ClientMetricsInstance instance = clientMetricsManager.clientInstance(subscriptionsResponse.data().clientInstanceId()); + assertNotNull(instance); + assertEquals(Errors.UNSUPPORTED_COMPRESSION_TYPE, instance.lastKnownError()); + + subscriptionsRequest = new GetTelemetrySubscriptionsRequest.Builder( + new GetTelemetrySubscriptionsRequestData().setClientInstanceId(subscriptionsResponse.data().clientInstanceId()), true).build(); + subscriptionsResponse = clientMetricsManager.processGetTelemetrySubscriptionRequest( + subscriptionsRequest, ClientMetricsTestUtils.requestContext()); + assertEquals(Errors.NONE, subscriptionsResponse.error()); + } + + @Test + public void testGetTelemetrySubscriptionAfterPushTelemetryBytesExceptionFails() throws Exception { + try ( + Metrics kafkaMetrics = new Metrics(); + ClientMetricsManager clientMetricsManager = new ClientMetricsManager(clientMetricsReceiverPlugin, 1, time, kafkaMetrics) + ) { + GetTelemetrySubscriptionsRequest subscriptionsRequest = new GetTelemetrySubscriptionsRequest.Builder( + new GetTelemetrySubscriptionsRequestData(), true).build(); + + GetTelemetrySubscriptionsResponse subscriptionsResponse = clientMetricsManager.processGetTelemetrySubscriptionRequest( + subscriptionsRequest, ClientMetricsTestUtils.requestContext()); + + byte[] metrics = "ab".getBytes(StandardCharsets.UTF_8); + assertEquals(2, metrics.length); + + PushTelemetryRequest request = new PushTelemetryRequest.Builder( + new PushTelemetryRequestData() + .setClientInstanceId(subscriptionsResponse.data().clientInstanceId()) + .setSubscriptionId(subscriptionsResponse.data().subscriptionId()) + .setMetrics(ByteBuffer.wrap(metrics)), true).build(); + + // Set the max bytes 1 to force the error. + PushTelemetryResponse response = clientMetricsManager.processPushTelemetryRequest( + request, ClientMetricsTestUtils.requestContext()); + + assertEquals(Errors.TELEMETRY_TOO_LARGE, response.error()); + ClientMetricsInstance instance = clientMetricsManager.clientInstance(subscriptionsResponse.data().clientInstanceId()); + assertNotNull(instance); + assertEquals(Errors.TELEMETRY_TOO_LARGE, instance.lastKnownError()); + + subscriptionsRequest = new GetTelemetrySubscriptionsRequest.Builder( + new GetTelemetrySubscriptionsRequestData().setClientInstanceId(subscriptionsResponse.data().clientInstanceId()), true).build(); + subscriptionsResponse = clientMetricsManager.processGetTelemetrySubscriptionRequest( + subscriptionsRequest, ClientMetricsTestUtils.requestContext()); + assertEquals(Errors.THROTTLING_QUOTA_EXCEEDED, subscriptionsResponse.error()); + } + } + @Test public void testCacheEviction() throws Exception { Properties properties = new Properties(); diff --git a/server/src/test/java/org/apache/kafka/server/LogManagerIntegrationTest.java b/server/src/test/java/org/apache/kafka/server/LogManagerIntegrationTest.java index 6a5e49825d741..15a57bbe22da0 100644 --- a/server/src/test/java/org/apache/kafka/server/LogManagerIntegrationTest.java +++ b/server/src/test/java/org/apache/kafka/server/LogManagerIntegrationTest.java @@ -34,9 +34,11 @@ import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.Type; +import org.apache.kafka.storage.internals.checkpoint.CleanShutdownFileHandler; import org.apache.kafka.storage.internals.checkpoint.PartitionMetadataFile; import org.apache.kafka.test.TestUtils; +import java.io.File; import java.io.IOException; import java.time.Duration; import java.util.ArrayList; @@ -57,6 +59,70 @@ public LogManagerIntegrationTest(ClusterInstance cluster) { this.cluster = cluster; } + @ClusterTest(types = {Type.KRAFT}) + public void testIOExceptionOnLogSegmentCloseResultsInRecovery() throws IOException, InterruptedException, ExecutionException { + try (Admin admin = cluster.admin()) { + admin.createTopics(List.of(new NewTopic("foo", 1, (short) 1))).all().get(); + } + cluster.waitForTopic("foo", 1); + + // Produce some data into the topic + Map producerConfigs = Map.of( + ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers(), + ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName(), + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName() + ); + + try (Producer producer = new KafkaProducer<>(producerConfigs)) { + producer.send(new ProducerRecord<>("foo", 0, null, "bar")).get(); + producer.flush(); + } + + var broker = cluster.brokers().get(0); + + File timeIndexFile = broker.logManager() + .getLog(new TopicPartition("foo", 0), false).get() + .activeSegment() + .timeIndexFile(); + + // Set read only so that we throw an IOException on shutdown + assertTrue(timeIndexFile.exists()); + assertTrue(timeIndexFile.setReadOnly()); + + broker.shutdown(); + + assertEquals(1, broker.config().logDirs().size()); + String logDir = broker.config().logDirs().get(0); + CleanShutdownFileHandler cleanShutdownFileHandler = new CleanShutdownFileHandler(logDir); + assertFalse(cleanShutdownFileHandler.exists(), "Did not expect the clean shutdown file to exist"); + + // Ensure we have a corrupt index on broker shutdown + long maxIndexSize = broker.config().logIndexSizeMaxBytes(); + long expectedIndexSize = 12 * (maxIndexSize / 12); + assertEquals(expectedIndexSize, timeIndexFile.length()); + + // Allow write permissions before startup + assertTrue(timeIndexFile.setWritable(true)); + + broker.startup(); + // make sure there is no error during load logs + assertTrue(cluster.firstFatalException().isEmpty()); + try (Admin admin = cluster.admin()) { + TestUtils.waitForCondition(() -> { + List partitionInfos = admin.describeTopics(List.of("foo")) + .topicNameValues().get("foo").get().partitions(); + return partitionInfos.get(0).leader().id() == 0; + }, "Partition does not have a leader assigned"); + } + + // Ensure that sanity check does not fail + broker.logManager() + .getLog(new TopicPartition("foo", 0), false).get() + .activeSegment() + .timeIndex() + .sanityCheck(); + } + @ClusterTest(types = {Type.KRAFT, Type.CO_KRAFT}, brokers = 3) public void testRestartBrokerNoErrorIfMissingPartitionMetadata() throws IOException, ExecutionException, InterruptedException { diff --git a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorOffsetsManager.java b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorOffsetsManager.java index 69070f65e936f..0b3e5a5ff0871 100644 --- a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorOffsetsManager.java +++ b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorOffsetsManager.java @@ -60,13 +60,20 @@ public ShareCoordinatorOffsetsManager(SnapshotRegistry snapshotRegistry) { * * @param key - represents {@link SharePartitionKey} whose offset needs updating * @param offset - represents the latest partition offset for provided key + * @param isDelete - true if the offset is for a tombstone record */ - public void updateState(SharePartitionKey key, long offset) { + public void updateState(SharePartitionKey key, long offset, boolean isDelete) { lastRedundantOffset.set(Math.min(lastRedundantOffset.get(), offset)); offsets.put(key, offset); Optional redundantOffset = findRedundantOffset(); redundantOffset.ifPresent(lastRedundantOffset::set); + + // If the share partition is deleted, we should not hold onto its offset in our calculations + // as there is nothing beyond deletion which is going to update its state. + if (isDelete) { + offsets.remove(key); + } } private Optional findRedundantOffset() { diff --git a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java index e26aac124ff7d..44315418c66f6 100644 --- a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java +++ b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java @@ -253,10 +253,12 @@ public int partitionFor(SharePartitionKey key) { @Override public Properties shareGroupStateTopicConfigs() { Properties properties = new Properties(); - properties.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE); // as defined in KIP-932 + // As defined in KIP-932. + properties.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE); properties.put(TopicConfig.COMPRESSION_TYPE_CONFIG, BrokerCompressionType.PRODUCER.name); properties.put(TopicConfig.SEGMENT_BYTES_CONFIG, config.shareCoordinatorStateTopicSegmentBytes()); properties.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, config.shareCoordinatorStateTopicMinIsr()); + properties.put(TopicConfig.RETENTION_MS_CONFIG, -1); return properties; } diff --git a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java index f5d9a3cfb3088..612b497291b5f 100644 --- a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java +++ b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java @@ -265,7 +265,7 @@ private void handleShareSnapshot(ShareSnapshotKey key, ShareSnapshotValue value, } } - offsetsManager.updateState(mapKey, offset); + offsetsManager.updateState(mapKey, offset, value == null); } private void handleShareUpdate(ShareUpdateKey key, ShareUpdateValue value) { diff --git a/share-coordinator/src/main/resources/common/message/ShareSnapshotValue.json b/share-coordinator/src/main/resources/common/message/ShareSnapshotValue.json index 6126cfd0a25af..46cd4f6c489da 100644 --- a/share-coordinator/src/main/resources/common/message/ShareSnapshotValue.json +++ b/share-coordinator/src/main/resources/common/message/ShareSnapshotValue.json @@ -20,7 +20,7 @@ "validVersions": "0", "flexibleVersions": "0+", "fields": [ - { "name": "SnapshotEpoch", "type": "uint16", "versions": "0+", + { "name": "SnapshotEpoch", "type": "int32", "versions": "0+", "about": "The snapshot epoch." }, { "name": "StateEpoch", "type": "int32", "versions": "0+", "about": "The state epoch for this share-partition." }, diff --git a/share-coordinator/src/main/resources/common/message/ShareUpdateValue.json b/share-coordinator/src/main/resources/common/message/ShareUpdateValue.json index 35e31462a9c02..5cd20c0d15923 100644 --- a/share-coordinator/src/main/resources/common/message/ShareUpdateValue.json +++ b/share-coordinator/src/main/resources/common/message/ShareUpdateValue.json @@ -20,7 +20,7 @@ "validVersions": "0", "flexibleVersions": "0+", "fields": [ - { "name": "SnapshotEpoch", "type": "uint16", "versions": "0+", + { "name": "SnapshotEpoch", "type": "int32", "versions": "0+", "about": "The snapshot epoch." }, { "name": "LeaderEpoch", "type": "int32", "versions": "0+", "about": "The leader epoch of the share-partition." }, diff --git a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorOffsetsManagerTest.java b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorOffsetsManagerTest.java index 262f166be192d..73d72bde9f01e 100644 --- a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorOffsetsManagerTest.java +++ b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorOffsetsManagerTest.java @@ -32,6 +32,7 @@ import java.util.stream.Stream; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; public class ShareCoordinatorOffsetsManagerTest { @@ -48,16 +49,19 @@ public void setUp() { @Test public void testUpdateStateAddsToInternalState() { - manager.updateState(KEY1, 0L); + manager.updateState(KEY1, 0L, false); assertEquals(Optional.empty(), manager.lastRedundantOffset()); - manager.updateState(KEY1, 10L); + manager.updateState(KEY1, 10L, false); assertEquals(Optional.of(10L), manager.lastRedundantOffset()); // [0-9] offsets are redundant. - manager.updateState(KEY2, 15L); + manager.updateState(KEY2, 15L, false); assertEquals(Optional.of(10L), manager.lastRedundantOffset()); // No update to last redundant after adding 15L so, still 10L. - assertEquals(10L, manager.curState().get(KEY1)); + manager.updateState(KEY1, 25L, true); + assertEquals(Optional.of(15L), manager.lastRedundantOffset()); // KEY1 deleted, no longer part of calculation. + + assertNull(manager.curState().get(KEY1)); assertEquals(15L, manager.curState().get(KEY2)); } @@ -66,15 +70,21 @@ static class TestTuple { final SharePartitionKey key; final long offset; final Optional expectedOffset; + final boolean isDelete; - private TestTuple(SharePartitionKey key, long offset, Optional expectedOffset) { + private TestTuple(SharePartitionKey key, long offset, Optional expectedOffset, boolean isDelete) { this.key = key; this.offset = offset; this.expectedOffset = expectedOffset; + this.isDelete = isDelete; } static TestTuple instance(SharePartitionKey key, long offset, Optional expectedOffset) { - return new TestTuple(key, offset, expectedOffset); + return new TestTuple(key, offset, expectedOffset, false); + } + + static TestTuple instance(SharePartitionKey key, long offset, Optional expectedOffset, boolean isDelete) { + return new TestTuple(key, offset, expectedOffset, isDelete); } } @@ -96,19 +106,35 @@ static TestTuple instance(SharePartitionKey key, long offset, Optional exp static Stream generateNoRedundantStateCases() { return Stream.of( new ShareOffsetTestHolder( - "no redundant state single key", + "no redundant state single key.", List.of( ShareOffsetTestHolder.TestTuple.instance(KEY1, 10L, Optional.of(10L)) ) ), new ShareOffsetTestHolder( - "no redundant state multiple keys", + "no redundant state single key with delete.", + List.of( + ShareOffsetTestHolder.TestTuple.instance(KEY1, 10L, Optional.of(10L), true) + ) + ), + + new ShareOffsetTestHolder( + "no redundant state multiple keys.", List.of( ShareOffsetTestHolder.TestTuple.instance(KEY1, 10L, Optional.of(10L)), ShareOffsetTestHolder.TestTuple.instance(KEY4, 11L, Optional.of(10L)), ShareOffsetTestHolder.TestTuple.instance(KEY2, 13L, Optional.of(10L)) ) + ), + + new ShareOffsetTestHolder( + "no redundant state multiple keys with delete.", + List.of( + ShareOffsetTestHolder.TestTuple.instance(KEY1, 10L, Optional.of(10L), true), + ShareOffsetTestHolder.TestTuple.instance(KEY4, 11L, Optional.of(11L), true), + ShareOffsetTestHolder.TestTuple.instance(KEY2, 13L, Optional.of(13L), true) + ) ) ); } @@ -116,7 +142,7 @@ static Stream generateNoRedundantStateCases() { static Stream generateRedundantStateCases() { return Stream.of( new ShareOffsetTestHolder( - "redundant state single key", + "redundant state single key.", List.of( ShareOffsetTestHolder.TestTuple.instance(KEY1, 10L, Optional.of(10L)), ShareOffsetTestHolder.TestTuple.instance(KEY1, 11L, Optional.of(11L)), @@ -125,7 +151,7 @@ static Stream generateRedundantStateCases() { ), new ShareOffsetTestHolder( - "redundant state multiple keys", + "redundant state multiple keys.", // KEY1: 10 17 // KEY2: 11 16 // KEY3: 15 @@ -136,6 +162,20 @@ static Stream generateRedundantStateCases() { ShareOffsetTestHolder.TestTuple.instance(KEY2, 16L, Optional.of(10L)), // KEY2 11 redundant but should not be returned ShareOffsetTestHolder.TestTuple.instance(KEY1, 17L, Optional.of(15L)) ) + ), + + new ShareOffsetTestHolder( + "redundant state multiple keys with delete.", + // KEY1: 10 17 + // KEY2: 11 16 + // KEY3: 15 + List.of( + ShareOffsetTestHolder.TestTuple.instance(KEY1, 10L, Optional.of(10L)), + ShareOffsetTestHolder.TestTuple.instance(KEY2, 11L, Optional.of(10L)), + ShareOffsetTestHolder.TestTuple.instance(KEY3, 15L, Optional.of(10L), true), + ShareOffsetTestHolder.TestTuple.instance(KEY2, 16L, Optional.of(10L)), // KEY2 11 redundant but should not be returned + ShareOffsetTestHolder.TestTuple.instance(KEY1, 17L, Optional.of(16L)) // Because we have removed KEY3 from calculation + ) ) ); @@ -144,7 +184,7 @@ static Stream generateRedundantStateCases() { static Stream generateComplexCases() { return Stream.of( new ShareOffsetTestHolder( - "redundant state reverse key order", + "redundant state reverse key order.", // Requests come in order KEY1, KEY2, KEY3, KEY3, KEY2, KEY1. List.of( ShareOffsetTestHolder.TestTuple.instance(KEY1, 10L, Optional.of(10L)), @@ -156,6 +196,18 @@ static Stream generateComplexCases() { ) ), + new ShareOffsetTestHolder( + "redundant state reverse key order with delete.", + List.of( + ShareOffsetTestHolder.TestTuple.instance(KEY1, 10L, Optional.of(10L)), + ShareOffsetTestHolder.TestTuple.instance(KEY2, 11L, Optional.of(10L)), + ShareOffsetTestHolder.TestTuple.instance(KEY3, 15L, Optional.of(10L)), + ShareOffsetTestHolder.TestTuple.instance(KEY3, 18L, Optional.of(10L), true), + ShareOffsetTestHolder.TestTuple.instance(KEY2, 20L, Optional.of(10L), true), + ShareOffsetTestHolder.TestTuple.instance(KEY1, 25L, Optional.of(25L)) // Because KEY2 and KEY3 are gone. + ) + ), + new ShareOffsetTestHolder( "redundant state infrequently written partition.", List.of( @@ -170,6 +222,20 @@ static Stream generateComplexCases() { ShareOffsetTestHolder.TestTuple.instance(KEY3, 28L, Optional.of(10L)), ShareOffsetTestHolder.TestTuple.instance(KEY1, 30L, Optional.of(27L)) ) + ), + + new ShareOffsetTestHolder( + "redundant state infrequently written partition with delete.", + List.of( + ShareOffsetTestHolder.TestTuple.instance(KEY1, 10L, Optional.of(10L)), + ShareOffsetTestHolder.TestTuple.instance(KEY2, 11L, Optional.of(10L)), + ShareOffsetTestHolder.TestTuple.instance(KEY3, 15L, Optional.of(10L)), + ShareOffsetTestHolder.TestTuple.instance(KEY2, 18L, Optional.of(10L)), + ShareOffsetTestHolder.TestTuple.instance(KEY3, 20L, Optional.of(10L), true), //KEY3 no longer party to calculation + ShareOffsetTestHolder.TestTuple.instance(KEY2, 22L, Optional.of(10L)), + ShareOffsetTestHolder.TestTuple.instance(KEY2, 27L, Optional.of(10L), true), //KEY2 no longer party to calculation + ShareOffsetTestHolder.TestTuple.instance(KEY1, 30L, Optional.of(30L)) + ) ) ); } @@ -179,7 +245,7 @@ static Stream generateComplexCases() { public void testUpdateStateNoRedundantState(ShareOffsetTestHolder holder) { if (holder.shouldRun) { holder.tuples.forEach(tuple -> { - manager.updateState(tuple.key, tuple.offset); + manager.updateState(tuple.key, tuple.offset, tuple.isDelete); assertEquals(tuple.expectedOffset, manager.lastRedundantOffset(), holder.testName); }); } @@ -190,7 +256,7 @@ public void testUpdateStateNoRedundantState(ShareOffsetTestHolder holder) { public void testUpdateStateRedundantState(ShareOffsetTestHolder holder) { if (holder.shouldRun) { holder.tuples.forEach(tuple -> { - manager.updateState(tuple.key, tuple.offset); + manager.updateState(tuple.key, tuple.offset, tuple.isDelete); assertEquals(tuple.expectedOffset, manager.lastRedundantOffset(), holder.testName); }); } @@ -201,9 +267,9 @@ public void testUpdateStateRedundantState(ShareOffsetTestHolder holder) { public void testUpdateStateComplexCases(ShareOffsetTestHolder holder) { if (holder.shouldRun) { holder.tuples.forEach(tuple -> { - manager.updateState(tuple.key, tuple.offset); + manager.updateState(tuple.key, tuple.offset, tuple.isDelete); assertEquals(tuple.expectedOffset, manager.lastRedundantOffset(), holder.testName); }); } } -} +} \ No newline at end of file diff --git a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java index 542d9cd8d0d00..57ad1da67a009 100644 --- a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java +++ b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java @@ -2158,10 +2158,12 @@ public void testShareStateTopicConfigs() { TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.COMPRESSION_TYPE_CONFIG, TopicConfig.SEGMENT_BYTES_CONFIG, - TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG + TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, + TopicConfig.RETENTION_MS_CONFIG ); Properties actual = service.shareGroupStateTopicConfigs(); - propNames.forEach(actual::contains); + propNames.forEach(actual::remove); + assertTrue(actual.isEmpty()); service.shutdown(); } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/AbstractIndex.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/AbstractIndex.java index bf1bd802f5e0c..f2e7c9830bd9e 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/AbstractIndex.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/AbstractIndex.java @@ -263,7 +263,9 @@ public boolean deleteIfExists() throws IOException { public void trimToValidSize() throws IOException { lock.lock(); try { - resize(entrySize() * entries); + if (mmap != null) { + resize(entrySize() * entries); + } } finally { lock.unlock(); } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java index cead243408763..de5e0f50433ed 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java @@ -93,6 +93,9 @@ public class LogSegment implements Closeable { // NOTED: the offset is the last offset of batch having the max timestamp. private volatile TimestampOffset maxTimestampAndOffsetSoFar = TimestampOffset.UNKNOWN; + // Lock for maxTimestampAndOffsetSoFar to ensure that it will be initialized only once + private final Object maxTimestampAndOffsetLock = new Object(); + private long created; /* the number of bytes since we last added an entry in the offset index */ @@ -177,7 +180,7 @@ public void resizeIndexes(int size) throws IOException { public void sanityCheck(boolean timeIndexFileNewlyCreated) throws IOException { if (offsetIndexFile().exists()) { // Resize the time index file to 0 if it is newly created. - if (timeIndexFileNewlyCreated) + if (timeIndexFileNewlyCreated) timeIndex().resize(0); // Sanity checks for time index and offset index are skipped because // we will recover the segments above the recovery point in recoverLog() @@ -192,8 +195,17 @@ public void sanityCheck(boolean timeIndexFileNewlyCreated) throws IOException { * the time index). */ public TimestampOffset readMaxTimestampAndOffsetSoFar() throws IOException { - if (maxTimestampAndOffsetSoFar == TimestampOffset.UNKNOWN) - maxTimestampAndOffsetSoFar = timeIndex().lastEntry(); + if (maxTimestampAndOffsetSoFar == TimestampOffset.UNKNOWN) { + // As stated in LogSegment class javadoc, this class is not thread-safe so basically we assume that + // methods are called within UnifiedLog#lock. + // However, there's exceptional paths where this method can be called outside of the lock, + // so we need lock here to prevent multiple threads trying to modify maxTimestampAndOffsetSoFar + synchronized (maxTimestampAndOffsetLock) { + if (maxTimestampAndOffsetSoFar == TimestampOffset.UNKNOWN) { + maxTimestampAndOffsetSoFar = timeIndex().lastEntry(); + } + } + } return maxTimestampAndOffsetSoFar; } @@ -751,10 +763,7 @@ public Optional findOffsetByTimestamp(long times public void close() throws IOException { if (maxTimestampAndOffsetSoFar != TimestampOffset.UNKNOWN) Utils.swallow(LOGGER, Level.WARN, "maybeAppend", () -> timeIndex().maybeAppend(maxTimestampSoFar(), shallowOffsetOfMaxTimestampSoFar(), true)); - Utils.closeQuietly(lazyOffsetIndex, "offsetIndex", LOGGER); - Utils.closeQuietly(lazyTimeIndex, "timeIndex", LOGGER); - Utils.closeQuietly(log, "log", LOGGER); - Utils.closeQuietly(txnIndex, "txnIndex", LOGGER); + Utils.closeAll(lazyOffsetIndex, lazyTimeIndex, log, txnIndex); } /** diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegments.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegments.java index a79602d56d12e..4b89d7115a45d 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegments.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegments.java @@ -17,6 +17,7 @@ package org.apache.kafka.storage.internals.log; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Utils; import java.io.Closeable; import java.io.File; @@ -104,8 +105,7 @@ public void clear() { */ @Override public void close() throws IOException { - for (LogSegment s : values()) - s.close(); + Utils.closeAll(values().toArray(new LogSegment[0])); } /** diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/TransactionIndex.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/TransactionIndex.java index 9e28c253a5e9b..076dfa0627cd8 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/TransactionIndex.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/TransactionIndex.java @@ -109,7 +109,7 @@ public void reset() throws IOException { public void close() throws IOException { FileChannel channel = channelOrNull(); - if (channel != null) + if (channel != null && channel.isOpen()) channel.close(); maybeChannel = Optional.empty(); } diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/LogSegmentTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/LogSegmentTest.java index bdca354a757a8..76607803400c9 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/LogSegmentTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/LogSegmentTest.java @@ -41,6 +41,7 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.ValueSource; @@ -57,6 +58,10 @@ import java.util.Map; import java.util.Optional; import java.util.OptionalLong; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -65,7 +70,11 @@ import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.clearInvocations; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; public class LogSegmentTest { @@ -856,6 +865,42 @@ public void testNonMonotonicTimestampForMultipleBatchesInMemoryRecords() throws assertEquals(new TimestampOffset(2, 2), segment.timeIndex().entry(1)); } + @Test + @Timeout(30) + public void testConcurrentAccessToMaxTimestampSoFar() throws Exception { + int numThreads = 16; + ExecutorService executor = Executors.newFixedThreadPool(numThreads); + TimeIndex mockTimeIndex = mock(TimeIndex.class); + when(mockTimeIndex.lastEntry()).thenReturn(new TimestampOffset(RecordBatch.NO_TIMESTAMP, 0L)); + + try { + // to reproduce race, we iterate test for certain duration + long remainingDurationNanos = Duration.ofSeconds(1).toNanos(); + while (remainingDurationNanos > 0) { + long t0 = System.nanoTime(); + clearInvocations(mockTimeIndex); + try (LogSegment seg = spy(LogTestUtils.createSegment(0, logDir, 10, Time.SYSTEM))) { + when(seg.timeIndex()).thenReturn(mockTimeIndex); + List> futures = new ArrayList<>(); + for (int i = 0; i < numThreads; i++) { + futures.add(executor.submit(() -> assertDoesNotThrow(seg::maxTimestampSoFar))); + } + for (Future future : futures) { + future.get(); + } + // timeIndex.lastEntry should be called once if no race + verify(mockTimeIndex, times(1)).lastEntry(); + + long elapsedNanos = System.nanoTime() - t0; + remainingDurationNanos -= elapsedNanos; + } + } + } finally { + executor.shutdown(); + executor.awaitTermination(Long.MAX_VALUE, TimeUnit.SECONDS); + } + } + private ProducerStateManager newProducerStateManager() throws IOException { return new ProducerStateManager( topicPartition, diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/LogSegmentsTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/LogSegmentsTest.java index fcf02cc7b764d..4136a76995aec 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/LogSegmentsTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/LogSegmentsTest.java @@ -38,7 +38,10 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; public class LogSegmentsTest { @@ -47,7 +50,7 @@ public class LogSegmentsTest { /* create a segment with the given base offset */ private static LogSegment createSegment(Long offset) throws IOException { - return LogTestUtils.createSegment(offset, logDir, 10, Time.SYSTEM); + return spy(LogTestUtils.createSegment(offset, logDir, 10, Time.SYSTEM)); } @BeforeEach @@ -274,4 +277,22 @@ public void testUpdateDir() throws IOException { } } + @Test + public void testCloseClosesAllLogSegmentsOnExceptionWhileClosingOne() throws IOException { + LogSegment seg1 = createSegment(0L); + LogSegment seg2 = createSegment(100L); + LogSegment seg3 = createSegment(200L); + LogSegments segments = new LogSegments(topicPartition); + segments.add(seg1); + segments.add(seg2); + segments.add(seg3); + + doThrow(new IOException("Failure")).when(seg2).close(); + + assertThrows(IOException.class, segments::close, "Expected IOException to be thrown"); + verify(seg1).close(); + verify(seg2).close(); + verify(seg3).close(); + } + } diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/OffsetIndexTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/OffsetIndexTest.java index 918e9dd409c09..ad7fa5908529c 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/OffsetIndexTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/OffsetIndexTest.java @@ -225,7 +225,6 @@ public void forceUnmapTest() throws IOException { idx.forceUnmap(); // mmap should be null after unmap causing lookup to throw a NPE assertThrows(NullPointerException.class, () -> idx.lookup(1)); - assertThrows(NullPointerException.class, idx::close); } @Test diff --git a/streams/quickstart/java/pom.xml b/streams/quickstart/java/pom.xml index 9e6d406898c1a..b8e1b21c68446 100644 --- a/streams/quickstart/java/pom.xml +++ b/streams/quickstart/java/pom.xml @@ -26,7 +26,7 @@ org.apache.kafka streams-quickstart - 4.1.0-SNAPSHOT + 4.1.1-SNAPSHOT .. diff --git a/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml b/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml index 133d0ee951e5d..06080ee90e203 100644 --- a/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml +++ b/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml @@ -29,7 +29,7 @@ UTF-8 - 4.1.0-SNAPSHOT + 4.1.1-SNAPSHOT 2.0.16 diff --git a/streams/quickstart/pom.xml b/streams/quickstart/pom.xml index 021e9d0c0d6f9..1707352a08751 100644 --- a/streams/quickstart/pom.xml +++ b/streams/quickstart/pom.xml @@ -22,7 +22,7 @@ org.apache.kafka streams-quickstart pom - 4.1.0-SNAPSHOT + 4.1.1-SNAPSHOT Kafka Streams :: Quickstart diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index b3723ef447d32..3c6f4975cb8ce 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -185,7 +185,6 @@ public class KafkaStreams implements AutoCloseable { protected final TopologyMetadata topologyMetadata; private final QueryableStoreProvider queryableStoreProvider; private final DelegatingStandbyUpdateListener delegatingStandbyUpdateListener; - private final LogContext logContext; GlobalStreamThread globalStreamThread; protected StateDirectory stateDirectory = null; @@ -643,9 +642,6 @@ private void maybeSetRunning() { return; } - // all (alive) threads have received their assignment, close any remaining startup tasks, they're not needed - stateDirectory.closeStartupTasks(); - setState(State.RUNNING); } @@ -968,7 +964,7 @@ private KafkaStreams(final TopologyMetadata topologyMetadata, } else { clientId = userClientId; } - logContext = new LogContext(String.format("stream-client [%s] ", clientId)); + final LogContext logContext = new LogContext(String.format("stream-client [%s] ", clientId)); this.log = logContext.logger(getClass()); topologyMetadata.setLog(logContext); @@ -1422,9 +1418,6 @@ private static HostInfo parseHostInfo(final String endPoint) { */ public synchronized void start() throws IllegalStateException, StreamsException { if (setState(State.REBALANCING)) { - log.debug("Initializing STANDBY tasks for existing local state"); - stateDirectory.initializeStartupTasks(topologyMetadata, streamsMetrics, logContext); - log.debug("Starting Streams client"); if (globalStreamThread != null) { diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java b/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java index e56a4cbfb4e56..2dbf48d67a423 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java @@ -75,7 +75,7 @@ public class StreamsBuilder { public StreamsBuilder() { topology = new Topology(); internalTopologyBuilder = topology.internalTopologyBuilder; - internalStreamsBuilder = new InternalStreamsBuilder(internalTopologyBuilder); + internalStreamsBuilder = new InternalStreamsBuilder(internalTopologyBuilder, false); } /** @@ -87,7 +87,14 @@ public StreamsBuilder() { public StreamsBuilder(final TopologyConfig topologyConfigs) { topology = newTopology(topologyConfigs); internalTopologyBuilder = topology.internalTopologyBuilder; - internalStreamsBuilder = new InternalStreamsBuilder(internalTopologyBuilder); + internalStreamsBuilder = new InternalStreamsBuilder( + internalTopologyBuilder, + TopologyConfig.InternalConfig.getBoolean( + topologyConfigs.originals(), + TopologyConfig.InternalConfig.ENABLE_PROCESS_PROCESSVALUE_FIX, + false + ) + ); } protected Topology newTopology(final TopologyConfig topologyConfigs) { diff --git a/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java b/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java index da8c246b26d12..fd76f07686a27 100644 --- a/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java @@ -34,6 +34,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Map; import java.util.Optional; import java.util.Properties; import java.util.function.Supplier; @@ -86,6 +87,28 @@ */ @SuppressWarnings("deprecation") public final class TopologyConfig extends AbstractConfig { + + public static class InternalConfig { + // Cf https://issues.apache.org/jira/browse/KAFKA-19668 + public static final String ENABLE_PROCESS_PROCESSVALUE_FIX = "__enable.process.processValue.fix__"; + + public static boolean getBoolean(final Map configs, final String key, final boolean defaultValue) { + final Object value = configs.getOrDefault(key, defaultValue); + if (value instanceof Boolean) { + return (boolean) value; + } else if (value instanceof String) { + return Boolean.parseBoolean((String) value); + } else { + log.warn( + "Invalid value ({}) on internal configuration '{}'. Please specify a true/false value.", + value, + key + ); + return defaultValue; + } + } + } + private static final ConfigDef CONFIG; static { CONFIG = new ConfigDef() diff --git a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/OpenIterators.java b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/OpenIterators.java new file mode 100644 index 0000000000000..2ef50d54abfba --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/OpenIterators.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.internals.metrics; + +import org.apache.kafka.common.MetricName; +import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.apache.kafka.streams.state.internals.MeteredIterator; +import org.apache.kafka.streams.state.internals.metrics.StateStoreMetrics; + +import java.util.Comparator; +import java.util.Iterator; +import java.util.NavigableSet; +import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.LongAdder; + +public class OpenIterators { + private final TaskId taskId; + private final String metricsScope; + private final String name; + private final StreamsMetricsImpl streamsMetrics; + + private final LongAdder numOpenIterators = new LongAdder(); + private final NavigableSet openIterators = new ConcurrentSkipListSet<>(Comparator.comparingLong(MeteredIterator::startTimestamp)); + private final AtomicLong oldestStartTimestamp = new AtomicLong(); + + private MetricName metricName; + + public OpenIterators(final TaskId taskId, + final String metricsScope, + final String name, + final StreamsMetricsImpl streamsMetrics) { + this.taskId = taskId; + this.metricsScope = metricsScope; + this.name = name; + this.streamsMetrics = streamsMetrics; + } + + public void add(final MeteredIterator iterator) { + openIterators.add(iterator); + numOpenIterators.increment(); + updateOldestStartTimestamp(); + + if (numOpenIterators.intValue() == 1) { + metricName = StateStoreMetrics.addOldestOpenIteratorGauge(taskId.toString(), metricsScope, name, streamsMetrics, + (config, now) -> oldestStartTimestamp.get() + ); + } + } + + public void remove(final MeteredIterator iterator) { + if (numOpenIterators.intValue() == 1) { + streamsMetrics.removeMetric(metricName); + } + numOpenIterators.decrement(); + openIterators.remove(iterator); + updateOldestStartTimestamp(); + } + + public long sum() { + return numOpenIterators.sum(); + } + + private void updateOldestStartTimestamp() { + final Iterator openIteratorsIterator = openIterators.iterator(); + if (openIteratorsIterator.hasNext()) { + oldestStartTimestamp.set(openIteratorsIterator.next().startTimestamp()); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadMetricsDelegatingReporter.java b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadMetricsDelegatingReporter.java index 65b7990dfe0c3..3f45a257b7232 100644 --- a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadMetricsDelegatingReporter.java +++ b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadMetricsDelegatingReporter.java @@ -57,7 +57,13 @@ public void metricChange(final KafkaMetric metric) { consumer.registerMetricForSubscription(metric); } } - + /* + The KafkaMetric object is a singleton shared by all StreamThread instances. + So we need to make sure we only pass metrics for the current StreamThread that contains this + MetricsReporter instance, which will register metrics with the embedded KafkaConsumer to pass + through the telemetry pipeline. + Otherwise, Kafka Streams would register multiple metrics for all StreamThreads. + */ private boolean tagMatchStreamOrStateUpdaterThreadId(final KafkaMetric metric) { final Map tags = metric.metricName().tags(); final boolean shouldInclude = tags.containsKey(THREAD_ID_TAG) && (tags.get(THREAD_ID_TAG).equals(threadId) || diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java index 968276bd501b0..6460313a62bf9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java @@ -72,6 +72,7 @@ public class InternalStreamsBuilder implements InternalNameProvider { private static final String TABLE_SOURCE_SUFFIX = "-source"; final InternalTopologyBuilder internalTopologyBuilder; + private final boolean processProcessValueFixEnabled; private final AtomicInteger index = new AtomicInteger(0); private final AtomicInteger buildPriorityIndex = new AtomicInteger(0); @@ -91,8 +92,10 @@ public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { } }; - public InternalStreamsBuilder(final InternalTopologyBuilder internalTopologyBuilder) { + public InternalStreamsBuilder(final InternalTopologyBuilder internalTopologyBuilder, + final boolean processProcessValueFixEnabled) { this.internalTopologyBuilder = internalTopologyBuilder; + this.processProcessValueFixEnabled = processProcessValueFixEnabled; } public KStream stream(final Collection topics, @@ -709,4 +712,7 @@ public InternalTopologyBuilder internalTopologyBuilder() { return internalTopologyBuilder; } + public boolean processProcessValueFixEnabled() { + return processProcessValueFixEnabled; + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index 1927aed03faed..02342a9de4a75 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -1306,7 +1306,12 @@ public KStream process( final ProcessorToStateConnectorNode processNode = new ProcessorToStateConnectorNode<>( name, new ProcessorParameters<>(processorSupplier, name), - stateStoreNames); + stateStoreNames + ); + if (builder.processProcessValueFixEnabled()) { + processNode.setKeyChangingOperation(true); + processNode.setValueChangingOperation(true); + } builder.addGraphNode(graphNode, processNode); @@ -1350,7 +1355,11 @@ public KStream processValues( final ProcessorToStateConnectorNode processNode = new ProcessorToStateConnectorNode<>( name, new ProcessorParameters<>(processorSupplier, name), - stateStoreNames); + stateStoreNames + ); + if (builder.processProcessValueFixEnabled()) { + processNode.setValueChangingOperation(true); + } builder.addGraphNode(graphNode, processNode); // cannot inherit value serde diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java index a686692b40a87..567d9a2947f00 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java @@ -213,7 +213,7 @@ public void close() { private ValueAndTimestamp> mapValue(final K key, final ValueAndTimestamp valueAndTimestamp) { return ValueAndTimestamp.make( mapper.apply(key, getValueOrNull(valueAndTimestamp)), - valueAndTimestamp == null ? context.recordContext().timestamp() : valueAndTimestamp.timestamp() + valueAndTimestamp == null ? context.timestamp() : valueAndTimestamp.timestamp() ); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java index d65244bcc86dc..45ab411d62e65 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java @@ -105,6 +105,13 @@ void register(final StateStore store, /** * Get the state store given the store name. * + *

    The returned state store represent one shard of the overall state, which belongs to the current task. + * The returned shard of the state store may only be used by the current + * {@link org.apache.kafka.streams.kstream.Transformer}, {@link org.apache.kafka.streams.kstream.ValueTransformer}, + * or {@link org.apache.kafka.streams.kstream.ValueTransformerWithKey} instance. + * Sharing a shard across different transformers (ie, from different "sibling" tasks; same sub-topology but different + * partition) may lead to data corruption and/or data loss. + * * @param name The store name * @param The type or interface of the store to return * @return The state store instance diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/api/ProcessingContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/api/ProcessingContext.java index 78aa2c8489b0e..7f6874c8c1947 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/api/ProcessingContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/api/ProcessingContext.java @@ -107,6 +107,12 @@ public interface ProcessingContext { /** * Get the state store given the store name. * + *

    The returned state store represent one shard of the overall state, which belongs to the current task. + * The returned shard of the state store may only be used by the current {@link Processor} or + * {@link FixedKeyProcessor} instance. + * Sharing a shard across different processors (ie, from different "sibling" tasks; same sub-topology but different + * partition) may lead to data corruption and/or data loss. + * * @param name The store name * @param The type or interface of the store to return * @return The state store instance diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdater.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdater.java index 716d8c42ec5ff..a3a44f6f02d31 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdater.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdater.java @@ -36,6 +36,7 @@ import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.Task.State; import org.apache.kafka.streams.processor.internals.TaskAndAction.Action; +import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.slf4j.Logger; @@ -89,7 +90,7 @@ private class StateUpdaterThread extends Thread { private volatile KafkaFutureImpl clientInstanceIdFuture = new KafkaFutureImpl<>(); public StateUpdaterThread(final String name, - final Metrics metrics, + final StreamsMetricsImpl metrics, final ChangelogReader changelogReader) { super(name); this.changelogReader = changelogReader; @@ -745,7 +746,7 @@ private void recordMetrics(final long now, final long totalLatency, final long t private final Time time; private final Logger log; private final String name; - private final Metrics metrics; + private final StreamsMetricsImpl metrics; private final Consumer restoreConsumer; private final ChangelogReader changelogReader; private final TopologyMetadata topologyMetadata; @@ -766,7 +767,7 @@ private void recordMetrics(final long now, final long totalLatency, final long t private StateUpdaterThread stateUpdaterThread = null; public DefaultStateUpdater(final String name, - final Metrics metrics, + final StreamsMetricsImpl metrics, final StreamsConfig config, final Consumer restoreConsumer, final ChangelogReader changelogReader, @@ -1062,70 +1063,71 @@ private class StateUpdaterMetrics { private final Sensor standbyRestoreRatioSensor; private final Sensor checkpointRatioSensor; - private final Deque allSensorNames = new LinkedList<>(); + private final Deque allSensors = new LinkedList<>(); private final Deque allMetricNames = new LinkedList<>(); - private StateUpdaterMetrics(final Metrics metrics, final String threadId) { + private StateUpdaterMetrics(final StreamsMetricsImpl metrics, final String threadId) { final Map threadLevelTags = new LinkedHashMap<>(); threadLevelTags.put(THREAD_ID_TAG, threadId); + final Metrics metricsRegistry = metrics.metricsRegistry(); - MetricName metricName = metrics.metricName("active-restoring-tasks", + MetricName metricName = metricsRegistry.metricName("active-restoring-tasks", STATE_LEVEL_GROUP, "The number of active tasks currently undergoing restoration", threadLevelTags); - metrics.addMetric(metricName, (config, now) -> stateUpdaterThread != null ? + metricsRegistry.addMetric(metricName, (config, now) -> stateUpdaterThread != null ? stateUpdaterThread.numRestoringActiveTasks() : 0); allMetricNames.push(metricName); - metricName = metrics.metricName("standby-updating-tasks", + metricName = metricsRegistry.metricName("standby-updating-tasks", STATE_LEVEL_GROUP, "The number of standby tasks currently undergoing state update", threadLevelTags); - metrics.addMetric(metricName, (config, now) -> stateUpdaterThread != null ? + metricsRegistry.addMetric(metricName, (config, now) -> stateUpdaterThread != null ? stateUpdaterThread.numUpdatingStandbyTasks() : 0); allMetricNames.push(metricName); - metricName = metrics.metricName("active-paused-tasks", + metricName = metricsRegistry.metricName("active-paused-tasks", STATE_LEVEL_GROUP, "The number of active tasks paused restoring", threadLevelTags); - metrics.addMetric(metricName, (config, now) -> stateUpdaterThread != null ? + metricsRegistry.addMetric(metricName, (config, now) -> stateUpdaterThread != null ? stateUpdaterThread.numPausedActiveTasks() : 0); allMetricNames.push(metricName); - metricName = metrics.metricName("standby-paused-tasks", + metricName = metricsRegistry.metricName("standby-paused-tasks", STATE_LEVEL_GROUP, "The number of standby tasks paused state update", threadLevelTags); - metrics.addMetric(metricName, (config, now) -> stateUpdaterThread != null ? + metricsRegistry.addMetric(metricName, (config, now) -> stateUpdaterThread != null ? stateUpdaterThread.numPausedStandbyTasks() : 0); allMetricNames.push(metricName); - this.idleRatioSensor = metrics.sensor("idle-ratio", RecordingLevel.INFO); + this.idleRatioSensor = metrics.threadLevelSensor(threadId, "idle-ratio", RecordingLevel.INFO); this.idleRatioSensor.add(new MetricName("idle-ratio", STATE_LEVEL_GROUP, IDLE_RATIO_DESCRIPTION, threadLevelTags), new Avg()); - allSensorNames.add("idle-ratio"); + allSensors.add(this.idleRatioSensor); - this.activeRestoreRatioSensor = metrics.sensor("active-restore-ratio", RecordingLevel.INFO); + this.activeRestoreRatioSensor = metrics.threadLevelSensor(threadId, "active-restore-ratio", RecordingLevel.INFO); this.activeRestoreRatioSensor.add(new MetricName("active-restore-ratio", STATE_LEVEL_GROUP, RESTORE_RATIO_DESCRIPTION, threadLevelTags), new Avg()); - allSensorNames.add("active-restore-ratio"); + allSensors.add(this.activeRestoreRatioSensor); - this.standbyRestoreRatioSensor = metrics.sensor("standby-update-ratio", RecordingLevel.INFO); + this.standbyRestoreRatioSensor = metrics.threadLevelSensor(threadId, "standby-update-ratio", RecordingLevel.INFO); this.standbyRestoreRatioSensor.add(new MetricName("standby-update-ratio", STATE_LEVEL_GROUP, UPDATE_RATIO_DESCRIPTION, threadLevelTags), new Avg()); - allSensorNames.add("standby-update-ratio"); + allSensors.add(this.standbyRestoreRatioSensor); - this.checkpointRatioSensor = metrics.sensor("checkpoint-ratio", RecordingLevel.INFO); + this.checkpointRatioSensor = metrics.threadLevelSensor(threadId, "checkpoint-ratio", RecordingLevel.INFO); this.checkpointRatioSensor.add(new MetricName("checkpoint-ratio", STATE_LEVEL_GROUP, CHECKPOINT_RATIO_DESCRIPTION, threadLevelTags), new Avg()); - allSensorNames.add("checkpoint-ratio"); + allSensors.add(this.checkpointRatioSensor); - this.restoreSensor = metrics.sensor("restore-records", RecordingLevel.INFO); + this.restoreSensor = metrics.threadLevelSensor(threadId, "restore-records", RecordingLevel.INFO); this.restoreSensor.add(new MetricName("restore-records-rate", STATE_LEVEL_GROUP, RESTORE_RECORDS_RATE_DESCRIPTION, threadLevelTags), new Rate()); this.restoreSensor.add(new MetricName("restore-call-rate", STATE_LEVEL_GROUP, RESTORE_RATE_DESCRIPTION, threadLevelTags), new Rate(new WindowedCount())); - allSensorNames.add("restore-records"); + allSensors.add(this.restoreSensor); } void clear() { - while (!allSensorNames.isEmpty()) { - metrics.removeSensor(allSensorNames.pop()); + while (!allSensors.isEmpty()) { + metrics.removeSensor(allSensors.pop()); } while (!allMetricNames.isEmpty()) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java index 01b694863fd0d..828ae3a0a7968 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java @@ -84,7 +84,7 @@ public void forward(final Record record, final String childName) { @Override public void forward(final KIn key, final VIn value) { - forward(new Record<>(key, value, recordContext().timestamp(), headers())); + forward(new Record<>(key, value, timestamp(), headers())); } /** diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java index bfab9a770f6a7..1d8f18d326d69 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java @@ -120,8 +120,8 @@ public void update(final ConsumerRecord record) { final Record toProcess = new Record<>( deserialized.key(), deserialized.value(), - processorContext.recordContext().timestamp(), - processorContext.recordContext().headers() + processorContext.timestamp(), + processorContext.headers() ); ((SourceNode) sourceNodeAndDeserializer.sourceNode()).process(toProcess); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java index 93961daf97b79..b5e0515522ad0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -190,7 +190,7 @@ public void forward(final K key, final Record toForward = new Record<>( key, value, - recordContext.timestamp(), + timestamp(), headers() ); forward(toForward); @@ -204,7 +204,7 @@ public void forward(final K key, final Record toForward = new Record<>( key, value, - toInternal.hasTimestamp() ? toInternal.timestamp() : recordContext.timestamp(), + toInternal.hasTimestamp() ? toInternal.timestamp() : timestamp(), headers() ); forward(toForward, toInternal.child()); @@ -250,11 +250,11 @@ public void forward(final Record record, final String childName) { // old API processors wouldn't see the timestamps or headers of upstream // new API processors. But then again, from the perspective of those old-API // processors, even consulting the timestamp or headers when the record context - // is undefined is itself not well-defined. Plus, I don't think we need to worry + // is undefined is itself not well defined. Plus, I don't think we need to worry // too much about heterogeneous applications, in which the upstream processor is // implementing the new API and the downstream one is implementing the old API. // So, this seems like a fine compromise for now. - if (recordContext != null && (record.timestamp() != recordContext.timestamp() || record.headers() != recordContext.headers())) { + if (recordContext != null && (record.timestamp() != timestamp() || record.headers() != headers())) { recordContext = new ProcessorRecordContext( record.timestamp(), recordContext.offset(), diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java index 1dddc55ca3c26..62173e807fded 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java @@ -209,13 +209,13 @@ public void process(final Record record) { // (instead of `RuntimeException`) to work well with those languages final ErrorHandlerContext errorHandlerContext = new DefaultErrorHandlerContext( null, // only required to pass for DeserializationExceptionHandler - internalProcessorContext.recordContext().topic(), - internalProcessorContext.recordContext().partition(), - internalProcessorContext.recordContext().offset(), - internalProcessorContext.recordContext().headers(), + internalProcessorContext.topic(), + internalProcessorContext.partition(), + internalProcessorContext.offset(), + internalProcessorContext.headers(), internalProcessorContext.currentNode().name(), internalProcessorContext.taskId(), - internalProcessorContext.recordContext().timestamp(), + internalProcessorContext.timestamp(), internalProcessorContext.recordContext().sourceRawKey(), internalProcessorContext.recordContext().sourceRawValue() ); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java index 3506845d288af..7c5cc947c41b3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java @@ -222,39 +222,6 @@ public ProcessorStateManager(final TaskId taskId, log.debug("Created state store manager for task {}", taskId); } - /** - * Special constructor used by {@link StateDirectory} to partially initialize startup tasks for local state, before - * they're assigned to a thread. When the task is assigned to a thread, the initialization of this StateManager is - * completed in {@link #assignToStreamThread(LogContext, ChangelogRegister, Collection)}. - */ - static ProcessorStateManager createStartupTaskStateManager(final TaskId taskId, - final boolean eosEnabled, - final LogContext logContext, - final StateDirectory stateDirectory, - final Map storeToChangelogTopic, - final Set sourcePartitions, - final boolean stateUpdaterEnabled) { - return new ProcessorStateManager(taskId, TaskType.STANDBY, eosEnabled, logContext, stateDirectory, null, storeToChangelogTopic, sourcePartitions, stateUpdaterEnabled); - } - - /** - * Standby tasks initialized for local state on-startup are only partially initialized, because they are not yet - * assigned to a StreamThread. Once assigned to a StreamThread, we complete their initialization here using the - * assigned StreamThread's context. - */ - void assignToStreamThread(final LogContext logContext, - final ChangelogRegister changelogReader, - final Collection sourcePartitions) { - if (this.changelogReader != null) { - throw new IllegalStateException("Attempted to replace an existing changelogReader on a StateManager without closing it."); - } - this.sourcePartitions.clear(); - this.log = logContext.logger(ProcessorStateManager.class); - this.logPrefix = logContext.logPrefix(); - this.changelogReader = changelogReader; - this.sourcePartitions.addAll(sourcePartitions); - } - void registerStateStores(final List allStores, final InternalProcessorContext processorContext) { processorContext.uninitialize(); for (final StateStore store : allStores) { @@ -347,7 +314,7 @@ void initializeStoreOffsetsFromCheckpoint(final boolean storeDirIsEmpty) { } private void maybeRegisterStoreWithChangelogReader(final String storeName) { - if (isLoggingEnabled(storeName) && changelogReader != null) { + if (isLoggingEnabled(storeName)) { changelogReader.register(getStorePartition(storeName), this); } } @@ -616,7 +583,7 @@ public void flushCache() { public void close() throws ProcessorStateException { log.debug("Closing its state manager and all the registered state stores: {}", stores); - if (!stateUpdaterEnabled && changelogReader != null) { + if (!stateUpdaterEnabled) { changelogReader.unregister(getAllChangelogTopicPartitions()); } @@ -664,7 +631,7 @@ else if (exception instanceof StreamsException) void recycle() { log.debug("Recycling state for {} task {}.", taskType, taskId); - if (!stateUpdaterEnabled && changelogReader != null) { + if (!stateUpdaterEnabled) { final List allChangelogs = getAllChangelogTopicPartitions(); changelogReader.unregister(allChangelogs); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java index d32cf2523e084..8cb82a6bfa5b0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java @@ -85,9 +85,9 @@ public void process(final Record record) { final ProcessorRecordContext contextForExtraction = new ProcessorRecordContext( timestamp, - context.recordContext().offset(), - context.recordContext().partition(), - context.recordContext().topic(), + context.offset(), + context.partition(), + context.topic(), record.headers() ); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java index a95d20ddae0a1..070f732f4e0cd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java @@ -16,18 +16,12 @@ */ package org.apache.kafka.streams.processor.internals; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.ProcessorStateException; import org.apache.kafka.streams.errors.StreamsException; -import org.apache.kafka.streams.errors.TaskCorruptedException; -import org.apache.kafka.streams.internals.StreamsConfigUtils; import org.apache.kafka.streams.processor.TaskId; -import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; -import org.apache.kafka.streams.state.internals.ThreadCache; import com.fasterxml.jackson.annotation.JsonIgnoreProperties; import com.fasterxml.jackson.annotation.JsonProperty; @@ -49,18 +43,13 @@ import java.nio.file.attribute.PosixFilePermissions; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; -import java.util.HashSet; +import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.Objects; import java.util.Set; import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Predicate; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -106,14 +95,11 @@ public StateDirectoryProcessFile() { private final boolean hasPersistentStores; private final boolean hasNamedTopologies; - private final ConcurrentMap lockedTasksToOwner = new ConcurrentHashMap<>(); + private final HashMap lockedTasksToOwner = new HashMap<>(); private FileChannel stateDirLockChannel; private FileLock stateDirLock; - private final StreamsConfig config; - private final ConcurrentMap tasksForLocalState = new ConcurrentHashMap<>(); - /** * Ensures that the state base directory as well as the application's sub-directory are created. * @@ -132,7 +118,6 @@ public StateDirectory(final StreamsConfig config, final Time time, final boolean this.hasPersistentStores = hasPersistentStores; this.hasNamedTopologies = hasNamedTopologies; this.appId = config.getString(StreamsConfig.APPLICATION_ID_CONFIG); - this.config = config; final String stateDirName = config.getString(StreamsConfig.STATE_DIR_CONFIG); final File baseDir = new File(stateDirName); stateDir = new File(baseDir, appId); @@ -197,109 +182,6 @@ private boolean lockStateDirectory() { return stateDirLock != null; } - public void initializeStartupTasks(final TopologyMetadata topologyMetadata, - final StreamsMetricsImpl streamsMetrics, - final LogContext logContext) { - final List nonEmptyTaskDirectories = listNonEmptyTaskDirectories(); - if (hasPersistentStores && !nonEmptyTaskDirectories.isEmpty()) { - final ThreadCache dummyCache = new ThreadCache(logContext, 0, streamsMetrics); - final boolean eosEnabled = StreamsConfigUtils.eosEnabled(config); - final boolean stateUpdaterEnabled = StreamsConfig.InternalConfig.stateUpdaterEnabled(config.originals()); - - // discover all non-empty task directories in StateDirectory - for (final TaskDirectory taskDirectory : nonEmptyTaskDirectories) { - final String dirName = taskDirectory.file().getName(); - final TaskId id = parseTaskDirectoryName(dirName, taskDirectory.namedTopology()); - final ProcessorTopology subTopology = topologyMetadata.buildSubtopology(id); - - // we still check if the task's sub-topology is stateful, even though we know its directory contains state, - // because it's possible that the topology has changed since that data was written, and is now stateless - // this therefore prevents us from creating unnecessary Tasks just because of some left-over state - if (subTopology.hasStateWithChangelogs()) { - final Set inputPartitions = topologyMetadata.nodeToSourceTopics(id).values().stream() - .flatMap(Collection::stream) - .map(t -> new TopicPartition(t, id.partition())) - .collect(Collectors.toSet()); - final ProcessorStateManager stateManager = ProcessorStateManager.createStartupTaskStateManager( - id, - eosEnabled, - logContext, - this, - subTopology.storeToChangelogTopic(), - inputPartitions, - stateUpdaterEnabled - ); - - final InternalProcessorContext context = new ProcessorContextImpl( - id, - config, - stateManager, - streamsMetrics, - dummyCache - ); - - final Task task = new StandbyTask( - id, - inputPartitions, - subTopology, - topologyMetadata.taskConfig(id), - streamsMetrics, - stateManager, - this, - dummyCache, - context - ); - - try { - task.initializeIfNeeded(); - - tasksForLocalState.put(id, task); - } catch (final TaskCorruptedException e) { - // Task is corrupt - wipe it out (under EOS) and don't initialize a Standby for it - task.suspend(); - task.closeDirty(); - } - } - } - } - } - - public boolean hasStartupTasks() { - return !tasksForLocalState.isEmpty(); - } - - public Task removeStartupTask(final TaskId taskId) { - final Task task = tasksForLocalState.remove(taskId); - if (task != null) { - lockedTasksToOwner.replace(taskId, Thread.currentThread()); - } - return task; - } - - public void closeStartupTasks() { - closeStartupTasks(t -> true); - } - - private void closeStartupTasks(final Predicate predicate) { - if (!tasksForLocalState.isEmpty()) { - // "drain" Tasks first to ensure that we don't try to close Tasks that another thread is attempting to close - final Set drainedTasks = new HashSet<>(tasksForLocalState.size()); - for (final Map.Entry entry : tasksForLocalState.entrySet()) { - if (predicate.test(entry.getValue()) && removeStartupTask(entry.getKey()) != null) { - // only add to our list of drained Tasks if we exclusively "claimed" a Task from tasksForLocalState - // to ensure we don't accidentally try to drain the same Task multiple times from concurrent threads - drainedTasks.add(entry.getValue()); - } - } - - // now that we have exclusive ownership of the drained tasks, close them - for (final Task task : drainedTasks) { - task.suspend(); - task.closeClean(); - } - } - } - public UUID initializeProcessId() { if (!hasPersistentStores) { final UUID processId = UUID.randomUUID(); @@ -496,17 +378,9 @@ synchronized void unlock(final TaskId taskId) { } } - /** - * Expose for tests. - */ - Thread lockOwner(final TaskId taskId) { - return lockedTasksToOwner.get(taskId); - } - @Override public void close() { if (hasPersistentStores) { - closeStartupTasks(); try { stateDirLock.release(); stateDirLockChannel.close(); @@ -624,7 +498,6 @@ private IOException maybeCleanEmptyNamedTopologyDirs(final boolean logExceptionA ); if (namedTopologyDirs != null) { for (final File namedTopologyDir : namedTopologyDirs) { - closeStartupTasks(task -> task.id().topologyName().equals(parseNamedTopologyFromDirectory(namedTopologyDir.getName()))); final File[] contents = namedTopologyDir.listFiles(); if (contents != null && contents.length == 0) { try { @@ -662,7 +535,6 @@ public void clearLocalStateForNamedTopology(final String topologyName) { log.debug("Tried to clear out the local state for NamedTopology {} but none was found", topologyName); } try { - closeStartupTasks(task -> task.id().topologyName().equals(topologyName)); Utils.delete(namedTopologyDir); } catch (final IOException e) { log.error("Hit an unexpected error while clearing local state for topology " + topologyName, e); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 82e9c8d7fb110..8c3b6cc506b0f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -866,8 +866,8 @@ private void doProcess(final long wallClockTime) { final Record toProcess = new Record<>( record.key(), record.value(), - processorContext.recordContext().timestamp(), - processorContext.recordContext().headers() + processorContext.timestamp(), + processorContext.headers() ); maybeMeasureLatency(() -> currNode.process(toProcess), time, processLatencySensor); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 86e12bf3f65df..ad66a822e7269 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -454,7 +454,7 @@ public static StreamThread create(final TopologyMetadata topologyMetadata, final DefaultTaskManager schedulingTaskManager = maybeCreateSchedulingTaskManager(processingThreadsEnabled, stateUpdaterEnabled, topologyMetadata, time, threadId, tasks); final StateUpdater stateUpdater = - maybeCreateStateUpdater( + maybeCreateAndStartStateUpdater( stateUpdaterEnabled, streamsMetrics, config, @@ -633,7 +633,7 @@ private static DefaultTaskManager maybeCreateSchedulingTaskManager(final boolean return null; } - private static StateUpdater maybeCreateStateUpdater(final boolean stateUpdaterEnabled, + private static StateUpdater maybeCreateAndStartStateUpdater(final boolean stateUpdaterEnabled, final StreamsMetricsImpl streamsMetrics, final StreamsConfig streamsConfig, final Consumer restoreConsumer, @@ -644,15 +644,17 @@ private static StateUpdater maybeCreateStateUpdater(final boolean stateUpdaterEn final int threadIdx) { if (stateUpdaterEnabled) { final String name = clientId + STATE_UPDATER_ID_SUBSTRING + threadIdx; - return new DefaultStateUpdater( + final StateUpdater stateUpdater = new DefaultStateUpdater( name, - streamsMetrics.metricsRegistry(), + streamsMetrics, streamsConfig, restoreConsumer, changelogReader, topologyMetadata, time ); + stateUpdater.start(); + return stateUpdater; } else { return null; } @@ -881,9 +883,6 @@ public void run() { } boolean cleanRun = false; try { - if (stateUpdaterEnabled) { - taskManager.init(); - } cleanRun = runLoop(); } catch (final Throwable e) { failedStreamThreadSensor.record(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java index 67d009b037f78..9207ec74a7cb0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java @@ -149,11 +149,6 @@ public class TaskManager { ); } - void init() { - if (stateUpdater != null) { - this.stateUpdater.start(); - } - } void setMainConsumer(final Consumer mainConsumer) { this.mainConsumer = mainConsumer; } @@ -333,31 +328,6 @@ private void closeDirtyAndRevive(final Collection taskWithChangelogs, fina } } - private Map> assignStartupTasks(final Map> tasksToAssign, - final String threadLogPrefix, - final TopologyMetadata topologyMetadata, - final ChangelogRegister changelogReader) { - if (stateDirectory.hasStartupTasks()) { - final Map> assignedTasks = new HashMap<>(tasksToAssign.size()); - for (final Map.Entry> entry : tasksToAssign.entrySet()) { - final TaskId taskId = entry.getKey(); - final Task task = stateDirectory.removeStartupTask(taskId); - if (task != null) { - // replace our dummy values with the real ones, now we know our thread and assignment - final Set inputPartitions = entry.getValue(); - task.stateManager().assignToStreamThread(new LogContext(threadLogPrefix), changelogReader, inputPartitions); - updateInputPartitionsOfStandbyTaskIfTheyChanged(task, inputPartitions); - - assignedTasks.put(task, inputPartitions); - } - } - - return assignedTasks; - } else { - return Collections.emptyMap(); - } - } - /** * @throws TaskMigratedException if the task producer got fenced (EOS only) * @throws StreamsException fatal error while creating / initializing the task @@ -487,15 +457,6 @@ private void handleTasksWithoutStateUpdater(final Map> standbyTasksToCreate, final Map> tasksToRecycle, final Set tasksToCloseClean) { - final Map> startupStandbyTasksToRecycle = assignStartupTasks(activeTasksToCreate, logPrefix, topologyMetadata, changelogReader); - final Map> startupStandbyTasksToUse = assignStartupTasks(standbyTasksToCreate, logPrefix, topologyMetadata, changelogReader); - - // recycle the startup standbys to active - tasks.addStandbyTasks(startupStandbyTasksToRecycle.keySet()); - - // use startup Standbys as real Standby tasks - tasks.addStandbyTasks(startupStandbyTasksToUse.keySet()); - for (final Task task : tasks.allTasks()) { final TaskId taskId = task.id(); if (activeTasksToCreate.containsKey(taskId)) { @@ -550,7 +511,6 @@ private void handleTasksWithStateUpdater(final Map> final Set tasksToCloseClean, final Map failedTasks) { handleTasksPendingInitialization(); - handleStartupTaskReuse(activeTasksToCreate, standbyTasksToCreate, failedTasks); handleRestoringAndUpdatingTasks(activeTasksToCreate, standbyTasksToCreate, failedTasks); handleRunningAndSuspendedTasks(activeTasksToCreate, standbyTasksToCreate, tasksToRecycle, tasksToCloseClean); } @@ -568,34 +528,6 @@ private void handleTasksPendingInitialization() { } } - private void handleStartupTaskReuse(final Map> activeTasksToCreate, - final Map> standbyTasksToCreate, - final Map failedTasks) { - final Map> startupStandbyTasksToRecycle = assignStartupTasks(activeTasksToCreate, logPrefix, topologyMetadata, changelogReader); - final Map> startupStandbyTasksToUse = assignStartupTasks(standbyTasksToCreate, logPrefix, topologyMetadata, changelogReader); - - // recycle the startup standbys to active, and remove them from the set of actives that need to be created - if (!startupStandbyTasksToRecycle.isEmpty()) { - final Set tasksToCloseDirty = new HashSet<>(); - for (final Map.Entry> entry : startupStandbyTasksToRecycle.entrySet()) { - final Task task = entry.getKey(); - recycleTaskFromStateUpdater(task, entry.getValue(), tasksToCloseDirty, failedTasks); - activeTasksToCreate.remove(task.id()); - } - - // if any standby tasks failed to recycle, close them dirty - tasksToCloseDirty.forEach(task -> - closeTaskDirty(task, false) - ); - } - - // use startup Standbys as real Standby tasks - if (!startupStandbyTasksToUse.isEmpty()) { - tasks.addPendingTasksToInit(startupStandbyTasksToUse.keySet()); - startupStandbyTasksToUse.keySet().forEach(task -> standbyTasksToCreate.remove(task.id())); - } - } - private void handleRunningAndSuspendedTasks(final Map> activeTasksToCreate, final Map> standbyTasksToCreate, final Map> tasksToRecycle, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImpl.java index 0dd3f77f199c7..7c22c94e9af5e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImpl.java @@ -335,6 +335,10 @@ public final void removeAllThreadLevelMetrics(final String threadId) { } } + public void removeMetric(final MetricName metricName) { + metrics.removeMetric(metricName); + } + public Map taskLevelTagMap(final String threadId, final String taskId) { final Map tagMap = new LinkedHashMap<>(); tagMap.put(THREAD_ID_TAG, threadId); @@ -517,13 +521,13 @@ public final Sensor storeLevelSensor(final String taskId, return getSensors(storeLevelSensors, sensorSuffix, sensorPrefix, recordingLevel, parents); } - public void addStoreLevelMutableMetric(final String taskId, - final String metricsScope, - final String storeName, - final String name, - final String description, - final RecordingLevel recordingLevel, - final Gauge valueProvider) { + public MetricName addStoreLevelMutableMetric(final String taskId, + final String metricsScope, + final String storeName, + final String name, + final String description, + final RecordingLevel recordingLevel, + final Gauge valueProvider) { final MetricName metricName = metrics.metricName( name, STATE_STORE_LEVEL_GROUP, @@ -535,6 +539,8 @@ public void addStoreLevelMutableMetric(final String taskId, final String key = storeSensorPrefix(Thread.currentThread().getName(), taskId, storeName); storeLevelMetrics.computeIfAbsent(key, ignored -> new LinkedList<>()).push(metricName); } + + return metricName; } public final void removeAllStoreLevelSensorsAndMetrics(final String taskId, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java index 83343d04494d6..389cf688f4a41 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java @@ -272,12 +272,12 @@ private void putInternal(final Bytes key, key, new LRUCacheEntry( value, - internalContext.recordContext().headers(), + internalContext.headers(), true, - internalContext.recordContext().offset(), - internalContext.recordContext().timestamp(), - internalContext.recordContext().partition(), - internalContext.recordContext().topic(), + internalContext.offset(), + internalContext.timestamp(), + internalContext.partition(), + internalContext.topic(), internalContext.recordContext().sourceRawKey(), internalContext.recordContext().sourceRawValue() ) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java index ec0c1bd077d6f..7bb615ea4ee32 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java @@ -135,12 +135,12 @@ public void put(final Windowed key, final byte[] value) { final LRUCacheEntry entry = new LRUCacheEntry( value, - internalContext.recordContext().headers(), + internalContext.headers(), true, - internalContext.recordContext().offset(), - internalContext.recordContext().timestamp(), - internalContext.recordContext().partition(), - internalContext.recordContext().topic(), + internalContext.offset(), + internalContext.timestamp(), + internalContext.partition(), + internalContext.topic(), internalContext.recordContext().sourceRawKey(), internalContext.recordContext().sourceRawValue() ); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java index 0432c1726cb3e..38d98b58d7e6a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java @@ -153,12 +153,12 @@ public synchronized void put(final Bytes key, final LRUCacheEntry entry = new LRUCacheEntry( value, - internalContext.recordContext().headers(), + internalContext.headers(), true, - internalContext.recordContext().offset(), - internalContext.recordContext().timestamp(), - internalContext.recordContext().partition(), - internalContext.recordContext().topic(), + internalContext.offset(), + internalContext.timestamp(), + internalContext.partition(), + internalContext.topic(), internalContext.recordContext().sourceRawKey(), internalContext.recordContext().sourceRawValue() ); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java index 9c1c3f9ae7639..b21b102cdfc3b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java @@ -52,7 +52,7 @@ private void maybeSetEvictionListener() { if (wrapped() instanceof MemoryLRUCache) { ((MemoryLRUCache) wrapped()).setWhenEldestRemoved((key, value) -> { // pass null to indicate removal - log(key, null, internalContext.recordContext().timestamp()); + log(key, null, internalContext.timestamp()); }); } } @@ -66,7 +66,7 @@ public long approximateNumEntries() { public void put(final Bytes key, final byte[] value) { wrapped().put(key, value); - log(key, value, internalContext.recordContext().timestamp()); + log(key, value, internalContext.timestamp()); } @Override @@ -75,7 +75,7 @@ public byte[] putIfAbsent(final Bytes key, final byte[] previous = wrapped().putIfAbsent(key, value); if (previous == null) { // then it was absent - log(key, value, internalContext.recordContext().timestamp()); + log(key, value, internalContext.timestamp()); } return previous; } @@ -84,7 +84,7 @@ public byte[] putIfAbsent(final Bytes key, public void putAll(final List> entries) { wrapped().putAll(entries); for (final KeyValue entry : entries) { - log(entry.key, entry.value, internalContext.recordContext().timestamp()); + log(entry.key, entry.value, internalContext.timestamp()); } } @@ -97,7 +97,7 @@ public , P> KeyValueIterator prefixScan( @Override public byte[] delete(final Bytes key) { final byte[] oldValue = wrapped().delete(key); - log(key, null, internalContext.recordContext().timestamp()); + log(key, null, internalContext.timestamp()); return oldValue; } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingListValueBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingListValueBytesStore.java index ba43ba30b17a9..9070fc8da5f10 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingListValueBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingListValueBytesStore.java @@ -32,9 +32,9 @@ public void put(final Bytes key, final byte[] value) { // we need to log the full new list and thus call get() on the inner store below // if the value is a tombstone, we delete the whole list and thus can save the get call if (value == null) { - log(key, null, internalContext.recordContext().timestamp()); + log(key, null, internalContext.timestamp()); } else { - log(key, wrapped().get(key), internalContext.recordContext().timestamp()); + log(key, wrapped().get(key), internalContext.timestamp()); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStore.java index 248889211c339..06097aa7a714f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStore.java @@ -73,13 +73,13 @@ public KeyValueIterator, byte[]> backwardFindSessions(final Byte @Override public void remove(final Windowed sessionKey) { wrapped().remove(sessionKey); - internalContext.logChange(name(), SessionKeySchema.toBinary(sessionKey), null, internalContext.recordContext().timestamp(), wrapped().getPosition()); + internalContext.logChange(name(), SessionKeySchema.toBinary(sessionKey), null, internalContext.timestamp(), wrapped().getPosition()); } @Override public void put(final Windowed sessionKey, final byte[] aggregate) { wrapped().put(sessionKey, aggregate); - internalContext.logChange(name(), SessionKeySchema.toBinary(sessionKey), aggregate, internalContext.recordContext().timestamp(), wrapped().getPosition()); + internalContext.logChange(name(), SessionKeySchema.toBinary(sessionKey), aggregate, internalContext.timestamp(), wrapped().getPosition()); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedKeyValueBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedKeyValueBytesStore.java index b95ede1bba85e..916c9547184a4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedKeyValueBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedKeyValueBytesStore.java @@ -35,7 +35,7 @@ public class ChangeLoggingTimestampedKeyValueBytesStore extends ChangeLoggingKey public void put(final Bytes key, final byte[] valueAndTimestamp) { wrapped().put(key, valueAndTimestamp); - log(key, rawValue(valueAndTimestamp), valueAndTimestamp == null ? internalContext.recordContext().timestamp() : timestamp(valueAndTimestamp)); + log(key, rawValue(valueAndTimestamp), valueAndTimestamp == null ? internalContext.timestamp() : timestamp(valueAndTimestamp)); } @Override @@ -44,7 +44,7 @@ public byte[] putIfAbsent(final Bytes key, final byte[] previous = wrapped().putIfAbsent(key, valueAndTimestamp); if (previous == null) { // then it was absent - log(key, rawValue(valueAndTimestamp), valueAndTimestamp == null ? internalContext.recordContext().timestamp() : timestamp(valueAndTimestamp)); + log(key, rawValue(valueAndTimestamp), valueAndTimestamp == null ? internalContext.timestamp() : timestamp(valueAndTimestamp)); } return previous; } @@ -54,7 +54,7 @@ public void putAll(final List> entries) { wrapped().putAll(entries); for (final KeyValue entry : entries) { final byte[] valueAndTimestamp = entry.value; - log(entry.key, rawValue(valueAndTimestamp), valueAndTimestamp == null ? internalContext.recordContext().timestamp() : timestamp(valueAndTimestamp)); + log(entry.key, rawValue(valueAndTimestamp), valueAndTimestamp == null ? internalContext.timestamp() : timestamp(valueAndTimestamp)); } } } \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedWindowBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedWindowBytesStore.java index 5ae334f95cca1..2bf87f9d2a8b5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedWindowBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedWindowBytesStore.java @@ -36,7 +36,7 @@ void log(final Bytes key, name(), key, rawValue(valueAndTimestamp), - valueAndTimestamp != null ? timestamp(valueAndTimestamp) : internalContext.recordContext().timestamp(), + valueAndTimestamp != null ? timestamp(valueAndTimestamp) : internalContext.timestamp(), wrapped().getPosition() ); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java index 0d0f378af7585..d5857d0456e66 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java @@ -129,7 +129,7 @@ public void put(final Bytes key, } void log(final Bytes key, final byte[] value) { - internalContext.logChange(name(), key, value, internalContext.recordContext().timestamp(), wrapped().getPosition()); + internalContext.logChange(name(), key, value, internalContext.timestamp(), wrapped().getPosition()); } private int maybeUpdateSeqnumForDups() { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java index 7622f904c17a5..8678111f989e4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java @@ -23,6 +23,7 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.errors.ProcessorStateException; +import org.apache.kafka.streams.internals.metrics.OpenIterators; import org.apache.kafka.streams.kstream.internals.Change; import org.apache.kafka.streams.kstream.internals.WrappingNullableUtils; import org.apache.kafka.streams.processor.StateStore; @@ -48,14 +49,9 @@ import org.apache.kafka.streams.state.internals.metrics.StateStoreMetrics; import java.util.ArrayList; -import java.util.Comparator; -import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.NavigableSet; import java.util.Objects; -import java.util.concurrent.ConcurrentSkipListSet; -import java.util.concurrent.atomic.LongAdder; import java.util.function.Function; import static org.apache.kafka.common.utils.Utils.mkEntry; @@ -96,8 +92,7 @@ public class MeteredKeyValueStore private StreamsMetricsImpl streamsMetrics; private TaskId taskId; - protected LongAdder numOpenIterators = new LongAdder(); - protected NavigableSet openIterators = new ConcurrentSkipListSet<>(Comparator.comparingLong(MeteredIterator::startTimestamp)); + protected OpenIterators openIterators; @SuppressWarnings("rawtypes") private final Map queryHandlers = @@ -153,13 +148,8 @@ private void registerMetrics() { e2eLatencySensor = StateStoreMetrics.e2ELatencySensor(taskId.toString(), metricsScope, name(), streamsMetrics); iteratorDurationSensor = StateStoreMetrics.iteratorDurationSensor(taskId.toString(), metricsScope, name(), streamsMetrics); StateStoreMetrics.addNumOpenIteratorsGauge(taskId.toString(), metricsScope, name(), streamsMetrics, - (config, now) -> numOpenIterators.sum()); - StateStoreMetrics.addOldestOpenIteratorGauge(taskId.toString(), metricsScope, name(), streamsMetrics, - (config, now) -> { - final Iterator openIteratorsIterator = openIterators.iterator(); - return openIteratorsIterator.hasNext() ? openIteratorsIterator.next().startTimestamp() : null; - } - ); + (config, now) -> openIterators.sum()); + openIterators = new OpenIterators(taskId, metricsScope, name(), streamsMetrics); } protected Serde prepareValueSerdeForStore(final Serde valueSerde, final SerdeGetter getter) { @@ -427,7 +417,7 @@ protected void maybeRecordE2ELatency() { // In that case, we _can't_ get the current timestamp, so we don't record anything. if (e2eLatencySensor.shouldRecord() && internalContext != null) { final long currentTime = time.milliseconds(); - final long e2eLatency = currentTime - internalContext.recordContext().timestamp(); + final long e2eLatency = currentTime - internalContext.timestamp(); e2eLatencySensor.record(e2eLatency, currentTime); } } @@ -445,7 +435,6 @@ private MeteredKeyValueIterator(final KeyValueIterator iter, this.sensor = sensor; this.startTimestamp = time.milliseconds(); this.startNs = time.nanoseconds(); - numOpenIterators.increment(); openIterators.add(this); } @@ -475,7 +464,6 @@ public void close() { final long duration = time.nanoseconds() - startNs; sensor.record(duration); iteratorDurationSensor.record(duration); - numOpenIterators.decrement(); openIterators.remove(this); } } @@ -502,7 +490,6 @@ private MeteredKeyValueTimestampedIterator(final KeyValueIterator this.valueDeserializer = valueDeserializer; this.startTimestamp = time.milliseconds(); this.startNs = time.nanoseconds(); - numOpenIterators.increment(); openIterators.add(this); } @@ -532,7 +519,6 @@ public void close() { final long duration = time.nanoseconds() - startNs; sensor.record(duration); iteratorDurationSensor.record(duration); - numOpenIterators.decrement(); openIterators.remove(this); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredMultiVersionedKeyQueryIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredMultiVersionedKeyQueryIterator.java index b1bb0b6315821..b27e6a78d86bf 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredMultiVersionedKeyQueryIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredMultiVersionedKeyQueryIterator.java @@ -18,39 +18,34 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.internals.metrics.OpenIterators; import org.apache.kafka.streams.state.VersionedRecord; import org.apache.kafka.streams.state.VersionedRecordIterator; -import java.util.Set; -import java.util.concurrent.atomic.LongAdder; import java.util.function.Function; class MeteredMultiVersionedKeyQueryIterator implements VersionedRecordIterator, MeteredIterator { private final VersionedRecordIterator iterator; private final Function, VersionedRecord> deserializeValue; - private final LongAdder numOpenIterators; private final Sensor sensor; private final Time time; private final long startNs; private final long startTimestampMs; - private final Set openIterators; + private final OpenIterators openIterators; public MeteredMultiVersionedKeyQueryIterator(final VersionedRecordIterator iterator, final Sensor sensor, final Time time, final Function, VersionedRecord> deserializeValue, - final LongAdder numOpenIterators, - final Set openIterators) { + final OpenIterators openIterators) { this.iterator = iterator; this.deserializeValue = deserializeValue; - this.numOpenIterators = numOpenIterators; this.openIterators = openIterators; this.sensor = sensor; this.time = time; this.startNs = time.nanoseconds(); this.startTimestampMs = time.milliseconds(); - numOpenIterators.increment(); openIterators.add(this); } @@ -65,7 +60,6 @@ public void close() { iterator.close(); } finally { sensor.record(time.nanoseconds() - startNs); - numOpenIterators.decrement(); openIterators.remove(this); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java index 234ac1220f7bc..546959a9269a4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java @@ -494,7 +494,7 @@ private void maybeRecordE2ELatency() { // In that case, we _can't_ get the current timestamp, so we don't record anything. if (e2eLatencySensor.shouldRecord() && internalContext != null) { final long currentTime = time.milliseconds(); - final long e2eLatency = currentTime - internalContext.recordContext().timestamp(); + final long e2eLatency = currentTime - internalContext.timestamp(); e2eLatencySensor.record(e2eLatency, currentTime); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStore.java index 6f90ef56d868e..0cfb8936a5e21 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStore.java @@ -326,7 +326,6 @@ private MeteredTimestampedKeyValueStoreIterator(final KeyValueIterator QueryResult runMultiVersionedKeyQuery(final Query query, final iteratorDurationSensor, time, StoreQueryUtils.deserializeValue(plainValueSerdes), - numOpenIterators, openIterators ); final QueryResult> typedQueryResult = diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java index 2da877453ced2..783c16b2f4f4a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java @@ -510,7 +510,7 @@ private void maybeRecordE2ELatency() { // In that case, we _can't_ get the current timestamp, so we don't record anything. if (e2eLatencySensor.shouldRecord() && internalContext != null) { final long currentTime = time.milliseconds(); - final long e2eLatency = currentTime - internalContext.recordContext().timestamp(); + final long e2eLatency = currentTime - internalContext.timestamp(); e2eLatencySensor.record(e2eLatency, currentTime); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedCachingWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedCachingWindowStore.java index 646cbf2ca3557..15a728ba0d07b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedCachingWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedCachingWindowStore.java @@ -97,13 +97,12 @@ private void enforceWrappedStore(final WindowStore underlying) { hasIndex = timeOrderedWindowStore.hasIndex(); } - @SuppressWarnings("unchecked") private RocksDBTimeOrderedWindowStore getWrappedStore(final StateStore wrapped) { if (wrapped instanceof RocksDBTimeOrderedWindowStore) { return (RocksDBTimeOrderedWindowStore) wrapped; } if (wrapped instanceof WrappedStateStore) { - return getWrappedStore(((WrappedStateStore) wrapped).wrapped()); + return getWrappedStore(((WrappedStateStore) wrapped).wrapped()); } return null; } @@ -256,12 +255,12 @@ public synchronized void put(final Bytes key, final LRUCacheEntry entry = new LRUCacheEntry( value, - internalContext.recordContext().headers(), + internalContext.headers(), true, - internalContext.recordContext().offset(), - internalContext.recordContext().timestamp(), - internalContext.recordContext().partition(), - internalContext.recordContext().topic(), + internalContext.offset(), + internalContext.timestamp(), + internalContext.partition(), + internalContext.topic(), internalContext.recordContext().sourceRawKey(), internalContext.recordContext().sourceRawValue() ); @@ -278,9 +277,9 @@ public synchronized void put(final Bytes key, new byte[0], new RecordHeaders(), true, - internalContext.recordContext().offset(), - internalContext.recordContext().timestamp(), - internalContext.recordContext().partition(), + internalContext.offset(), + internalContext.timestamp(), + internalContext.partition(), "", internalContext.recordContext().sourceRawKey(), internalContext.recordContext().sourceRawValue() diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/StateStoreMetrics.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/StateStoreMetrics.java index 2422fa9d5e321..bb60c304684e1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/StateStoreMetrics.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/StateStoreMetrics.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.state.internals.metrics; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.Gauge; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.metrics.Sensor.RecordingLevel; @@ -455,12 +456,12 @@ public static void addNumOpenIteratorsGauge(final String taskId, } - public static void addOldestOpenIteratorGauge(final String taskId, - final String storeType, - final String storeName, - final StreamsMetricsImpl streamsMetrics, - final Gauge oldestOpenIteratorGauge) { - streamsMetrics.addStoreLevelMutableMetric( + public static MetricName addOldestOpenIteratorGauge(final String taskId, + final String storeType, + final String storeName, + final StreamsMetricsImpl streamsMetrics, + final Gauge oldestOpenIteratorGauge) { + return streamsMetrics.addStoreLevelMutableMetric( taskId, storeType, storeName, diff --git a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java index d417b5be2edc6..e1212b7f3088e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java @@ -400,44 +400,6 @@ public void testShouldTransitToNotRunningIfCloseRightAfterCreated() { } } - @Test - public void shouldInitializeTasksForLocalStateOnStart() { - prepareStreams(); - prepareStreamThread(streamThreadOne, 1); - prepareStreamThread(streamThreadTwo, 2); - - try (final MockedConstruction constructed = mockConstruction(StateDirectory.class, - (mock, context) -> when(mock.initializeProcessId()).thenReturn(UUID.randomUUID()))) { - try (final KafkaStreams streams = new KafkaStreams(getBuilderWithSource().build(), props, supplier, time)) { - assertEquals(1, constructed.constructed().size()); - final StateDirectory stateDirectory = constructed.constructed().get(0); - verify(stateDirectory, times(0)).initializeStartupTasks(any(), any(), any()); - streams.start(); - verify(stateDirectory, times(1)).initializeStartupTasks(any(), any(), any()); - } - } - } - - @Test - public void shouldCloseStartupTasksAfterFirstRebalance() throws Exception { - prepareStreams(); - final AtomicReference state1 = prepareStreamThread(streamThreadOne, 1); - final AtomicReference state2 = prepareStreamThread(streamThreadTwo, 2); - prepareThreadState(streamThreadOne, state1); - prepareThreadState(streamThreadTwo, state2); - try (final MockedConstruction constructed = mockConstruction(StateDirectory.class, - (mock, context) -> when(mock.initializeProcessId()).thenReturn(UUID.randomUUID()))) { - try (final KafkaStreams streams = new KafkaStreams(getBuilderWithSource().build(), props, supplier, time)) { - assertEquals(1, constructed.constructed().size()); - final StateDirectory stateDirectory = constructed.constructed().get(0); - streams.setStateListener(streamsStateListener); - streams.start(); - waitForCondition(() -> streams.state() == State.RUNNING, "Streams never started."); - verify(stateDirectory, times(1)).closeStartupTasks(); - } - } - } - @Test public void stateShouldTransitToRunningIfNonDeadThreadsBackToRunning() throws Exception { prepareStreams(); diff --git a/streams/src/test/java/org/apache/kafka/streams/internals/metrics/OpenIteratorsTest.java b/streams/src/test/java/org/apache/kafka/streams/internals/metrics/OpenIteratorsTest.java new file mode 100644 index 0000000000000..3464ecbdaee04 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/internals/metrics/OpenIteratorsTest.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.internals.metrics; + +import org.apache.kafka.common.metrics.Gauge; +import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.apache.kafka.streams.state.internals.MeteredIterator; + +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; + +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.not; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.reset; +import static org.mockito.Mockito.verify; + +public class OpenIteratorsTest { + + private final StreamsMetricsImpl streamsMetrics = mock(StreamsMetricsImpl.class); + + @SuppressWarnings("unchecked") + @Test + public void shouldCalculateOldestStartTimestampCorrectly() { + final OpenIterators openIterators = new OpenIterators(new TaskId(0, 0), "scope", "name", streamsMetrics); + + final MeteredIterator meteredIterator1 = () -> 5; + final MeteredIterator meteredIterator2 = () -> 2; + final MeteredIterator meteredIterator3 = () -> 6; + + openIterators.add(meteredIterator1); + final ArgumentCaptor> gaugeCaptor = ArgumentCaptor.forClass(Gauge.class); + verify(streamsMetrics).addStoreLevelMutableMetric(any(), any(), any(), any(), any(), any(), gaugeCaptor.capture()); + final Gauge gauge = gaugeCaptor.getValue(); + assertThat(gauge.value(null, 0), is(5L)); + reset(streamsMetrics); + + openIterators.add(meteredIterator2); + verify(streamsMetrics, never()).addStoreLevelMutableMetric(any(), any(), any(), any(), any(), any(), gaugeCaptor.capture()); + assertThat(gauge.value(null, 0), is(2L)); + + openIterators.remove(meteredIterator2); + verify(streamsMetrics, never()).removeMetric(any()); + assertThat(gauge.value(null, 0), is(5L)); + + openIterators.remove(meteredIterator1); + verify(streamsMetrics).removeMetric(any()); + assertThat(gauge.value(null, 0), is(5L)); + + openIterators.add(meteredIterator3); + verify(streamsMetrics).addStoreLevelMutableMetric(any(), any(), any(), any(), any(), any(), gaugeCaptor.capture()); + assertThat(gaugeCaptor.getValue(), not(gauge)); + assertThat(gaugeCaptor.getValue().value(null, 0), is(6L)); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilderTest.java index 9ac3ff776ca4b..0a13a4100e4b1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilderTest.java @@ -81,7 +81,7 @@ public class InternalStreamsBuilderTest { private static final String APP_ID = "app-id"; - private final InternalStreamsBuilder builder = new InternalStreamsBuilder(new InternalTopologyBuilder()); + private final InternalStreamsBuilder builder = new InternalStreamsBuilder(new InternalTopologyBuilder(), false); private final ConsumedInternal consumed = new ConsumedInternal<>(Consumed.with(null, null)); private final String storePrefix = "prefix-"; private final MaterializedInternal> materialized = new MaterializedInternal<>(Materialized.as("test-store"), builder, storePrefix); @@ -93,7 +93,7 @@ public void testNewName() { assertEquals("Y-0000000001", builder.newProcessorName("Y-")); assertEquals("Z-0000000002", builder.newProcessorName("Z-")); - final InternalStreamsBuilder newBuilder = new InternalStreamsBuilder(new InternalTopologyBuilder()); + final InternalStreamsBuilder newBuilder = new InternalStreamsBuilder(new InternalTopologyBuilder(), false); assertEquals("X-0000000000", newBuilder.newProcessorName("X-")); assertEquals("Y-0000000001", newBuilder.newProcessorName("Y-")); @@ -106,7 +106,7 @@ public void testNewStoreName() { assertEquals("Y-STATE-STORE-0000000001", builder.newStoreName("Y-")); assertEquals("Z-STATE-STORE-0000000002", builder.newStoreName("Z-")); - final InternalStreamsBuilder newBuilder = new InternalStreamsBuilder(new InternalTopologyBuilder()); + final InternalStreamsBuilder newBuilder = new InternalStreamsBuilder(new InternalTopologyBuilder(), false); assertEquals("X-STATE-STORE-0000000000", newBuilder.newStoreName("X-")); assertEquals("Y-STATE-STORE-0000000001", newBuilder.newStoreName("Y-")); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/MaterializedInternalTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/MaterializedInternalTest.java index 24600c57fec1a..a3eaadd27db68 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/MaterializedInternalTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/MaterializedInternalTest.java @@ -95,7 +95,7 @@ public void shouldUseStoreTypeWhenProvidedViaTopologyConfig() { final InternalTopologyBuilder topologyBuilder = new InternalTopologyBuilder( new TopologyConfig("my-topology", config, topologyOverrides)); - final InternalStreamsBuilder internalStreamsBuilder = new InternalStreamsBuilder(topologyBuilder); + final InternalStreamsBuilder internalStreamsBuilder = new InternalStreamsBuilder(topologyBuilder, false); final MaterializedInternal> materialized = new MaterializedInternal<>(Materialized.as(supplier), internalStreamsBuilder, prefix); @@ -113,7 +113,7 @@ public void shouldPreferStoreSupplierWhenProvidedWithStoreTypeViaTopologyConfig( final InternalTopologyBuilder topologyBuilder = new InternalTopologyBuilder( new TopologyConfig("my-topology", config, topologyOverrides)); - final InternalStreamsBuilder internalStreamsBuilder = new InternalStreamsBuilder(topologyBuilder); + final InternalStreamsBuilder internalStreamsBuilder = new InternalStreamsBuilder(topologyBuilder, false); final MaterializedInternal> materialized = new MaterializedInternal<>(Materialized.as(supplier), internalStreamsBuilder, prefix); @@ -129,7 +129,7 @@ public void shouldReturnEmptyWhenOriginalsAndOverridesDontHaveSuppliersSpecified final InternalTopologyBuilder topologyBuilder = new InternalTopologyBuilder( new TopologyConfig("my-topology", config, topologyOverrides)); - final InternalStreamsBuilder internalStreamsBuilder = new InternalStreamsBuilder(topologyBuilder); + final InternalStreamsBuilder internalStreamsBuilder = new InternalStreamsBuilder(topologyBuilder, false); final MaterializedInternal> materialized = new MaterializedInternal<>(Materialized.as(supplier), internalStreamsBuilder, prefix); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java index e046ba1953355..1675619480f61 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java @@ -22,6 +22,7 @@ import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.Topology; +import org.apache.kafka.streams.TopologyConfig; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.Branched; import org.apache.kafka.streams.kstream.Consumed; @@ -36,6 +37,8 @@ import org.apache.kafka.streams.kstream.Suppressed; import org.apache.kafka.streams.kstream.TimeWindows; import org.apache.kafka.streams.kstream.ValueJoiner; +import org.apache.kafka.streams.processor.api.ContextualFixedKeyProcessor; +import org.apache.kafka.streams.processor.api.FixedKeyRecord; import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.ProcessorSupplier; @@ -127,7 +130,7 @@ public void shouldNotThrowNPEWithMergeNodes() { initializer = () -> ""; aggregator = (aggKey, value, aggregate) -> aggregate + value.length(); final ProcessorSupplier processorSupplier = - () -> new Processor() { + () -> new Processor<>() { private ProcessorContext context; @Override @@ -185,14 +188,163 @@ public void process(final Record record) { } @Test - public void shouldNotOptimizeWithValueOrKeyChangingOperatorsAfterInitialKeyChange() { + public void shouldPartiallyOptimizeWithValueOrKeyChangingOperatorsAfterInitialKeyChangeWithFixDisabled() { + final Topology attemptedOptimize = getTopologyWithChangingValuesAfterChangingKey(StreamsConfig.OPTIMIZE, false); + final Topology noOptimization = getTopologyWithChangingValuesAfterChangingKey(StreamsConfig.NO_OPTIMIZATION, false); + + System.out.println(attemptedOptimize.describe().toString()); + System.out.println(noOptimization.describe().toString()); + assertEquals("Topologies:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input])\n" + + " --> KSTREAM-KEY-SELECT-0000000001\n" + + " Processor: KSTREAM-KEY-SELECT-0000000001 (stores: [])\n" + + " --> KSTREAM-FLATMAPVALUES-0000000010, KSTREAM-MAPVALUES-0000000002, KSTREAM-PROCESSVALUES-0000000018\n" + + " <-- KSTREAM-SOURCE-0000000000\n" + + " Processor: KSTREAM-FLATMAPVALUES-0000000010 (stores: [])\n" + + " --> KSTREAM-FILTER-0000000014\n" + + " <-- KSTREAM-KEY-SELECT-0000000001\n" + + " Processor: KSTREAM-MAPVALUES-0000000002 (stores: [])\n" + + " --> KSTREAM-FILTER-0000000006\n" + + " <-- KSTREAM-KEY-SELECT-0000000001\n" + + " Processor: KSTREAM-PROCESSVALUES-0000000018 (stores: [])\n" + + " --> KSTREAM-FILTER-0000000022\n" + + " <-- KSTREAM-KEY-SELECT-0000000001\n" + + " Processor: KSTREAM-FILTER-0000000006 (stores: [])\n" + + " --> KSTREAM-SINK-0000000005\n" + + " <-- KSTREAM-MAPVALUES-0000000002\n" + + " Processor: KSTREAM-FILTER-0000000014 (stores: [])\n" + + " --> KSTREAM-SINK-0000000013\n" + + " <-- KSTREAM-FLATMAPVALUES-0000000010\n" + + " Processor: KSTREAM-FILTER-0000000022 (stores: [])\n" + + " --> KSTREAM-SINK-0000000021\n" + + " <-- KSTREAM-PROCESSVALUES-0000000018\n" + + " Sink: KSTREAM-SINK-0000000005 (topic: KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition)\n" + + " <-- KSTREAM-FILTER-0000000006\n" + + " Sink: KSTREAM-SINK-0000000013 (topic: KSTREAM-AGGREGATE-STATE-STORE-0000000011-repartition)\n" + + " <-- KSTREAM-FILTER-0000000014\n" + + " Sink: KSTREAM-SINK-0000000021 (topic: KSTREAM-AGGREGATE-STATE-STORE-0000000019-repartition)\n" + + " <-- KSTREAM-FILTER-0000000022\n" + + "\n" + + " Sub-topology: 1\n" + + " Source: KSTREAM-SOURCE-0000000007 (topics: [KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition])\n" + + " --> KSTREAM-AGGREGATE-0000000004\n" + + " Processor: KSTREAM-AGGREGATE-0000000004 (stores: [KSTREAM-AGGREGATE-STATE-STORE-0000000003])\n" + + " --> KTABLE-TOSTREAM-0000000008\n" + + " <-- KSTREAM-SOURCE-0000000007\n" + + " Processor: KTABLE-TOSTREAM-0000000008 (stores: [])\n" + + " --> KSTREAM-SINK-0000000009\n" + + " <-- KSTREAM-AGGREGATE-0000000004\n" + + " Sink: KSTREAM-SINK-0000000009 (topic: output)\n" + + " <-- KTABLE-TOSTREAM-0000000008\n" + + "\n" + + " Sub-topology: 2\n" + + " Source: KSTREAM-SOURCE-0000000015 (topics: [KSTREAM-AGGREGATE-STATE-STORE-0000000011-repartition])\n" + + " --> KSTREAM-AGGREGATE-0000000012\n" + + " Processor: KSTREAM-AGGREGATE-0000000012 (stores: [KSTREAM-AGGREGATE-STATE-STORE-0000000011])\n" + + " --> KTABLE-TOSTREAM-0000000016\n" + + " <-- KSTREAM-SOURCE-0000000015\n" + + " Processor: KTABLE-TOSTREAM-0000000016 (stores: [])\n" + + " --> KSTREAM-SINK-0000000017\n" + + " <-- KSTREAM-AGGREGATE-0000000012\n" + + " Sink: KSTREAM-SINK-0000000017 (topic: windowed-output)\n" + + " <-- KTABLE-TOSTREAM-0000000016\n" + + "\n" + + " Sub-topology: 3\n" + + " Source: KSTREAM-SOURCE-0000000023 (topics: [KSTREAM-AGGREGATE-STATE-STORE-0000000019-repartition])\n" + + " --> KSTREAM-AGGREGATE-0000000020\n" + + " Processor: KSTREAM-AGGREGATE-0000000020 (stores: [KSTREAM-AGGREGATE-STATE-STORE-0000000019])\n" + + " --> KTABLE-TOSTREAM-0000000024\n" + + " <-- KSTREAM-SOURCE-0000000023\n" + + " Processor: KTABLE-TOSTREAM-0000000024 (stores: [])\n" + + " --> KSTREAM-SINK-0000000025\n" + + " <-- KSTREAM-AGGREGATE-0000000020\n" + + " Sink: KSTREAM-SINK-0000000025 (topic: output)\n" + + " <-- KTABLE-TOSTREAM-0000000024\n" + + "\n", + noOptimization.describe().toString() + ); + assertEquals("Topologies:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input])\n" + + " --> KSTREAM-KEY-SELECT-0000000001\n" + + " Processor: KSTREAM-KEY-SELECT-0000000001 (stores: [])\n" + + " --> KSTREAM-FLATMAPVALUES-0000000010, KSTREAM-MAPVALUES-0000000002, KSTREAM-PROCESSVALUES-0000000018\n" + + " <-- KSTREAM-SOURCE-0000000000\n" + + " Processor: KSTREAM-FLATMAPVALUES-0000000010 (stores: [])\n" + + " --> KSTREAM-FILTER-0000000014\n" + + " <-- KSTREAM-KEY-SELECT-0000000001\n" + + " Processor: KSTREAM-MAPVALUES-0000000002 (stores: [])\n" + + " --> KSTREAM-FILTER-0000000006\n" + + " <-- KSTREAM-KEY-SELECT-0000000001\n" + + " Processor: KSTREAM-PROCESSVALUES-0000000018 (stores: [])\n" + + " --> KSTREAM-FILTER-0000000022\n" + + " <-- KSTREAM-KEY-SELECT-0000000001\n" + + " Processor: KSTREAM-FILTER-0000000006 (stores: [])\n" + + " --> KSTREAM-SINK-0000000005\n" + + " <-- KSTREAM-MAPVALUES-0000000002\n" + + " Processor: KSTREAM-FILTER-0000000014 (stores: [])\n" + + " --> KSTREAM-SINK-0000000013\n" + + " <-- KSTREAM-FLATMAPVALUES-0000000010\n" + + " Processor: KSTREAM-FILTER-0000000022 (stores: [])\n" + + " --> KSTREAM-SINK-0000000021\n" + + " <-- KSTREAM-PROCESSVALUES-0000000018\n" + + " Sink: KSTREAM-SINK-0000000005 (topic: KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition)\n" + + " <-- KSTREAM-FILTER-0000000006\n" + + " Sink: KSTREAM-SINK-0000000013 (topic: KSTREAM-AGGREGATE-STATE-STORE-0000000011-repartition)\n" + + " <-- KSTREAM-FILTER-0000000014\n" + + " Sink: KSTREAM-SINK-0000000021 (topic: KSTREAM-AGGREGATE-STATE-STORE-0000000019-repartition)\n" + + " <-- KSTREAM-FILTER-0000000022\n" + + "\n" + + " Sub-topology: 1\n" + + " Source: KSTREAM-SOURCE-0000000007 (topics: [KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition])\n" + + " --> KSTREAM-AGGREGATE-0000000004\n" + + " Processor: KSTREAM-AGGREGATE-0000000004 (stores: [KSTREAM-AGGREGATE-STATE-STORE-0000000003])\n" + + " --> KTABLE-TOSTREAM-0000000008\n" + + " <-- KSTREAM-SOURCE-0000000007\n" + + " Processor: KTABLE-TOSTREAM-0000000008 (stores: [])\n" + + " --> KSTREAM-SINK-0000000009\n" + + " <-- KSTREAM-AGGREGATE-0000000004\n" + + " Sink: KSTREAM-SINK-0000000009 (topic: output)\n" + + " <-- KTABLE-TOSTREAM-0000000008\n" + + "\n" + + " Sub-topology: 2\n" + + " Source: KSTREAM-SOURCE-0000000015 (topics: [KSTREAM-AGGREGATE-STATE-STORE-0000000011-repartition])\n" + + " --> KSTREAM-AGGREGATE-0000000012\n" + + " Processor: KSTREAM-AGGREGATE-0000000012 (stores: [KSTREAM-AGGREGATE-STATE-STORE-0000000011])\n" + + " --> KTABLE-TOSTREAM-0000000016\n" + + " <-- KSTREAM-SOURCE-0000000015\n" + + " Processor: KTABLE-TOSTREAM-0000000016 (stores: [])\n" + + " --> KSTREAM-SINK-0000000017\n" + + " <-- KSTREAM-AGGREGATE-0000000012\n" + + " Sink: KSTREAM-SINK-0000000017 (topic: windowed-output)\n" + + " <-- KTABLE-TOSTREAM-0000000016\n" + + "\n" + + " Sub-topology: 3\n" + + " Source: KSTREAM-SOURCE-0000000023 (topics: [KSTREAM-AGGREGATE-STATE-STORE-0000000019-repartition])\n" + + " --> KSTREAM-AGGREGATE-0000000020\n" + + " Processor: KSTREAM-AGGREGATE-0000000020 (stores: [KSTREAM-AGGREGATE-STATE-STORE-0000000019])\n" + + " --> KTABLE-TOSTREAM-0000000024\n" + + " <-- KSTREAM-SOURCE-0000000023\n" + + " Processor: KTABLE-TOSTREAM-0000000024 (stores: [])\n" + + " --> KSTREAM-SINK-0000000025\n" + + " <-- KSTREAM-AGGREGATE-0000000020\n" + + " Sink: KSTREAM-SINK-0000000025 (topic: output)\n" + + " <-- KTABLE-TOSTREAM-0000000024\n\n", + noOptimization.describe().toString() + ); + assertEquals(3, getCountOfRepartitionTopicsFound(attemptedOptimize.describe().toString())); + assertEquals(3, getCountOfRepartitionTopicsFound(noOptimization.describe().toString())); + } - final Topology attemptedOptimize = getTopologyWithChangingValuesAfterChangingKey(StreamsConfig.OPTIMIZE); - final Topology noOptimization = getTopologyWithChangingValuesAfterChangingKey(StreamsConfig.NO_OPTIMIZATION); + @Test + public void shouldNotOptimizeWithValueOrKeyChangingOperatorsAfterInitialKeyChangeWithFixEnabled() { + final Topology attemptedOptimize = getTopologyWithChangingValuesAfterChangingKey(StreamsConfig.OPTIMIZE, true); + final Topology noOptimization = getTopologyWithChangingValuesAfterChangingKey(StreamsConfig.NO_OPTIMIZATION, true); assertEquals(attemptedOptimize.describe().toString(), noOptimization.describe().toString()); - assertEquals(2, getCountOfRepartitionTopicsFound(attemptedOptimize.describe().toString())); - assertEquals(2, getCountOfRepartitionTopicsFound(noOptimization.describe().toString())); + assertEquals(3, getCountOfRepartitionTopicsFound(attemptedOptimize.describe().toString())); + assertEquals(3, getCountOfRepartitionTopicsFound(noOptimization.describe().toString())); } @Test @@ -227,20 +379,30 @@ public void shouldNotOptimizeWhenRepartitionOperationIsDone() { assertEquals(2, getCountOfRepartitionTopicsFound(noOptimization.describe().toString())); } - private Topology getTopologyWithChangingValuesAfterChangingKey(final String optimizeConfig) { - - final StreamsBuilder builder = new StreamsBuilder(); + private Topology getTopologyWithChangingValuesAfterChangingKey(final String optimizeConfig, + final boolean enableFix) { final Properties properties = new Properties(); + properties.put(StreamsConfig.APPLICATION_ID_CONFIG, "application-id"); + properties.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); properties.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, optimizeConfig); + properties.put(TopologyConfig.InternalConfig.ENABLE_PROCESS_PROCESSVALUE_FIX, enableFix); + + final StreamsBuilder builder = new StreamsBuilder(new TopologyConfig(new StreamsConfig(properties))); final KStream inputStream = builder.stream("input"); final KStream mappedKeyStream = inputStream.selectKey((k, v) -> k + v); mappedKeyStream.mapValues(v -> v.toUpperCase(Locale.getDefault())).groupByKey().count().toStream().to("output"); mappedKeyStream.flatMapValues(v -> Arrays.asList(v.split("\\s"))).groupByKey().windowedBy(TimeWindows.ofSizeWithNoGrace(ofMillis(5000))).count().toStream().to("windowed-output"); + mappedKeyStream.processValues( + () -> new ContextualFixedKeyProcessor<>() { + @Override + public void process(final FixedKeyRecord record) { + context().forward(record.withValue(record.value().toUpperCase(Locale.getDefault()))); + } + }).groupByKey().count().toStream().to("output"); return builder.build(properties); - } private Topology getTopologyWithRepartitionOperation(final String optimizeConfig) { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdaterTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdaterTest.java index abb128698a081..b6d41966257a5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdaterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdaterTest.java @@ -28,6 +28,7 @@ import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.StateUpdater.ExceptionAndTask; import org.apache.kafka.streams.processor.internals.Task.State; +import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.hamcrest.Matcher; import org.junit.jupiter.api.AfterEach; @@ -105,7 +106,7 @@ class DefaultStateUpdaterTest { // need an auto-tick timer to work for draining with timeout private final Time time = new MockTime(1L); - private final Metrics metrics = new Metrics(time); + private final StreamsMetricsImpl metrics = new StreamsMetricsImpl(new Metrics(time), "", "", time); private final StreamsConfig config = new StreamsConfig(configProps(COMMIT_INTERVAL)); private final ChangelogReader changelogReader = mock(ChangelogReader.class); private final TopologyMetadata topologyMetadata = unnamedTopology().build(); @@ -1680,8 +1681,59 @@ public void shouldRecordMetrics() throws Exception { assertThat(metrics.metrics().size(), is(1)); } + @Test + public void shouldRemoveMetricsWithoutInterference() { + final DefaultStateUpdater stateUpdater2 = + new DefaultStateUpdater("test-state-updater2", metrics, config, null, changelogReader, topologyMetadata, time); + final List threadMetrics = getMetricNames("test-state-updater"); + final List threadMetrics2 = getMetricNames("test-state-updater2"); + + stateUpdater.start(); + stateUpdater2.start(); + + for (final MetricName metricName : threadMetrics) { + assertTrue(metrics.metrics().containsKey(metricName)); + } + for (final MetricName metricName : threadMetrics2) { + assertTrue(metrics.metrics().containsKey(metricName)); + } + + stateUpdater2.shutdown(Duration.ofMinutes(1)); + + for (final MetricName metricName : threadMetrics) { + assertTrue(metrics.metrics().containsKey(metricName)); + } + for (final MetricName metricName : threadMetrics2) { + assertFalse(metrics.metrics().containsKey(metricName)); + } + + stateUpdater.shutdown(Duration.ofMinutes(1)); + + for (final MetricName metricName : threadMetrics) { + assertFalse(metrics.metrics().containsKey(metricName)); + } + for (final MetricName metricName : threadMetrics2) { + assertFalse(metrics.metrics().containsKey(metricName)); + } + } + + private static List getMetricNames(final String threadId) { + final Map tagMap = Map.of("thread-id", threadId); + return List.of( + new MetricName("active-restoring-tasks", "stream-state-updater-metrics", "", tagMap), + new MetricName("standby-updating-tasks", "stream-state-updater-metrics", "", tagMap), + new MetricName("active-paused-tasks", "stream-state-updater-metrics", "", tagMap), + new MetricName("standby-paused-tasks", "stream-state-updater-metrics", "", tagMap), + new MetricName("idle-ratio", "stream-state-updater-metrics", "", tagMap), + new MetricName("standby-update-ratio", "stream-state-updater-metrics", "", tagMap), + new MetricName("checkpoint-ratio", "stream-state-updater-metrics", "", tagMap), + new MetricName("restore-records-rate", "stream-state-updater-metrics", "", tagMap), + new MetricName("restore-call-rate", "stream-state-updater-metrics", "", tagMap) + ); + } + @SuppressWarnings("unchecked") - private static void verifyMetric(final Metrics metrics, + private static void verifyMetric(final StreamsMetricsImpl metrics, final MetricName metricName, final Matcher matcher) { assertThat(metrics.metrics().get(metricName).metricName().description(), is(metricName.description())); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java index 9410ca5a97896..42c466c2120fe 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java @@ -593,8 +593,6 @@ public void shouldThrowUnsupportedOperationExceptionOnGetStateStore() { @Test public void shouldThrowUnsupportedOperationExceptionOnForward() { context = getStandbyContext(); - context.recordContext = mock(ProcessorRecordContext.class); - assertThrows( UnsupportedOperationException.class, () -> context.forward("key", "value") @@ -604,8 +602,6 @@ public void shouldThrowUnsupportedOperationExceptionOnForward() { @Test public void shouldThrowUnsupportedOperationExceptionOnForwardWithTo() { context = getStandbyContext(); - context.recordContext = mock(ProcessorRecordContext.class); - assertThrows( UnsupportedOperationException.class, () -> context.forward("key", "value", To.child("child-name")) diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java index 5341cd25f0d5d..ce5fddb870a5d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java @@ -362,7 +362,7 @@ public ProcessingExceptionHandler.ProcessingHandlerResponse handle(final ErrorHa assertEquals(internalProcessorContext.offset(), context.offset()); assertEquals(internalProcessorContext.currentNode().name(), context.processorNodeId()); assertEquals(internalProcessorContext.taskId(), context.taskId()); - assertEquals(internalProcessorContext.recordContext().timestamp(), context.timestamp()); + assertEquals(internalProcessorContext.timestamp(), context.timestamp()); assertEquals(internalProcessorContext.recordContext().sourceRawKey(), context.sourceRawKey()); assertEquals(internalProcessorContext.recordContext().sourceRawValue(), context.sourceRawValue()); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java index afb89fa3c8f3c..57753df91e83f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java @@ -30,6 +30,7 @@ import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.streams.TestOutputTopic; import org.apache.kafka.streams.Topology; +import org.apache.kafka.streams.TopologyConfig; import org.apache.kafka.streams.TopologyTestDriver; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.Consumed; @@ -42,6 +43,8 @@ import org.apache.kafka.streams.kstream.Produced; import org.apache.kafka.streams.kstream.Reducer; import org.apache.kafka.streams.kstream.StreamJoined; +import org.apache.kafka.streams.processor.api.ContextualFixedKeyProcessor; +import org.apache.kafka.streams.processor.api.FixedKeyRecord; import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.Stores; @@ -220,6 +223,38 @@ private void runTest(final String optimizationConfig, final int expectedNumberRe assertThat(joinedOutputTopic.readKeyValuesToMap(), equalTo(keyValueListToMap(expectedJoinKeyValues))); } + @Test + public void shouldNotPushRepartitionAcrossValueChangingOperation() { + streamsConfiguration.setProperty(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, StreamsConfig.OPTIMIZE); + streamsConfiguration.setProperty(TopologyConfig.InternalConfig.ENABLE_PROCESS_PROCESSVALUE_FIX, "true"); + + final StreamsBuilder builder = new StreamsBuilder(new TopologyConfig(new StreamsConfig(streamsConfiguration))); + + builder.stream(INPUT_TOPIC, Consumed.with(Serdes.String(), Serdes.String()).withName("sourceStream")) + .map((k, v) -> KeyValue.pair(k.toUpperCase(Locale.getDefault()), v)) + .processValues(() -> new ContextualFixedKeyProcessor() { + @Override + public void process(final FixedKeyRecord record) { + context().forward(record.withValue(record.value().length())); + } + }) + .groupByKey(Grouped.valueSerde(new Serdes.IntegerSerde())) + .reduce(Integer::sum) + .toStream() + .to(AGGREGATION_TOPIC); + + final Topology topology = builder.build(streamsConfiguration); + + topologyTestDriver = new TopologyTestDriver(topology, streamsConfiguration); + + final TestInputTopic inputTopic = topologyTestDriver.createInputTopic(INPUT_TOPIC, stringSerializer, stringSerializer); + final TestOutputTopic outputTopic = topologyTestDriver.createOutputTopic(AGGREGATION_TOPIC, stringDeserializer, new IntegerDeserializer()); + + inputTopic.pipeKeyValueList(getKeyValues()); + + assertThat(outputTopic.readKeyValuesToMap(), equalTo(keyValueListToMap(expectedAggKeyValues))); + } + private Map keyValueListToMap(final List> keyValuePairs) { final Map map = new HashMap<>(); for (final KeyValue pair : keyValuePairs) { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java index f850093d389d4..515bff1df1600 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java @@ -17,20 +17,14 @@ package org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.utils.LogCaptureAppender; -import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.TopologyConfig; import org.apache.kafka.streams.errors.ProcessorStateException; -import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.StateDirectory.TaskDirectory; -import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.internals.OffsetCheckpoint; -import org.apache.kafka.test.MockKeyValueStore; import org.apache.kafka.test.TestUtils; import com.fasterxml.jackson.annotation.JsonProperty; @@ -39,8 +33,6 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.mockito.ArgumentMatchers; -import org.mockito.Mockito; import java.io.BufferedWriter; import java.io.File; @@ -79,7 +71,6 @@ import static org.hamcrest.CoreMatchers.endsWith; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.hasItem; -import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.CoreMatchers.not; import static org.hamcrest.MatcherAssert.assertThat; @@ -96,7 +87,6 @@ public class StateDirectoryTest { private final MockTime time = new MockTime(); private File stateDir; private final String applicationId = "applicationId"; - private StreamsConfig config; private StateDirectory directory; private File appDir; @@ -105,14 +95,15 @@ private void initializeStateDirectory(final boolean createStateDirectory, final if (!createStateDirectory) { cleanup(); } - config = new StreamsConfig(new Properties() { - { - put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); - put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "dummy:1234"); - put(StreamsConfig.STATE_DIR_CONFIG, stateDir.getPath()); - } - }); - directory = new StateDirectory(config, time, createStateDirectory, hasNamedTopology); + directory = new StateDirectory( + new StreamsConfig(new Properties() { + { + put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); + put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "dummy:1234"); + put(StreamsConfig.STATE_DIR_CONFIG, stateDir.getPath()); + } + }), + time, createStateDirectory, hasNamedTopology); appDir = new File(stateDir, applicationId); } @@ -821,144 +812,6 @@ public void shouldReadFutureProcessFileFormat() throws Exception { assertThat(directory.initializeProcessId(), equalTo(processId)); } - @Test - public void shouldNotInitializeStandbyTasksWhenNoLocalState() { - final TaskId taskId = new TaskId(0, 0); - initializeStartupTasks(new TaskId(0, 0), false); - assertFalse(directory.hasStartupTasks()); - assertNull(directory.removeStartupTask(taskId)); - assertFalse(directory.hasStartupTasks()); - } - - @Test - public void shouldInitializeStandbyTasksForLocalState() { - final TaskId taskId = new TaskId(0, 0); - initializeStartupTasks(new TaskId(0, 0), true); - assertTrue(directory.hasStartupTasks()); - assertNotNull(directory.removeStartupTask(taskId)); - assertFalse(directory.hasStartupTasks()); - assertNull(directory.removeStartupTask(taskId)); - } - - @Test - public void shouldNotAssignStartupTasksWeDontHave() { - final TaskId taskId = new TaskId(0, 0); - initializeStartupTasks(taskId, false); - final Task task = directory.removeStartupTask(taskId); - assertNull(task); - } - - private class FakeStreamThread extends Thread { - private final TaskId taskId; - private final AtomicReference result; - - private FakeStreamThread(final TaskId taskId, final AtomicReference result) { - this.taskId = taskId; - this.result = result; - } - - @Override - public void run() { - result.set(directory.removeStartupTask(taskId)); - } - } - - @Test - public void shouldAssignStartupTaskToStreamThread() throws InterruptedException { - final TaskId taskId = new TaskId(0, 0); - - initializeStartupTasks(taskId, true); - - // main thread owns the newly initialized tasks - assertThat(directory.lockOwner(taskId), is(Thread.currentThread())); - - // spawn off a "fake" StreamThread, so we can verify the lock was updated to the correct thread - final AtomicReference result = new AtomicReference<>(); - final Thread streamThread = new FakeStreamThread(taskId, result); - streamThread.start(); - streamThread.join(); - final Task task = result.get(); - - assertNotNull(task); - assertThat(task, instanceOf(StandbyTask.class)); - - // verify the owner of the task directory lock has been shifted over to our assigned StreamThread - assertThat(directory.lockOwner(taskId), is(instanceOf(FakeStreamThread.class))); - } - - @Test - public void shouldUnlockStartupTasksOnClose() { - final TaskId taskId = new TaskId(0, 0); - initializeStartupTasks(taskId, true); - - assertEquals(Thread.currentThread(), directory.lockOwner(taskId)); - directory.closeStartupTasks(); - assertNull(directory.lockOwner(taskId)); - } - - @Test - public void shouldCloseStartupTasksOnDirectoryClose() { - final StateStore store = initializeStartupTasks(new TaskId(0, 0), true); - - assertTrue(directory.hasStartupTasks()); - assertTrue(store.isOpen()); - - directory.close(); - - assertFalse(directory.hasStartupTasks()); - assertFalse(store.isOpen()); - } - - @Test - public void shouldNotCloseStartupTasksOnAutoCleanUp() { - // we need to set this because the auto-cleanup uses the last-modified time from the filesystem, - // which can't be mocked - time.setCurrentTimeMs(System.currentTimeMillis()); - - final StateStore store = initializeStartupTasks(new TaskId(0, 0), true); - - assertTrue(directory.hasStartupTasks()); - assertTrue(store.isOpen()); - - time.sleep(10000); - - directory.cleanRemovedTasks(1000); - - assertTrue(directory.hasStartupTasks()); - assertTrue(store.isOpen()); - } - - private StateStore initializeStartupTasks(final TaskId taskId, final boolean createTaskDir) { - directory.initializeProcessId(); - final TopologyMetadata metadata = Mockito.mock(TopologyMetadata.class); - final TopologyConfig topologyConfig = new TopologyConfig(config); - - final StateStore store = new MockKeyValueStore("test", true); - - if (createTaskDir) { - final File taskDir = directory.getOrCreateDirectoryForTask(taskId); - final File storeDir = new File(taskDir, store.name()); - storeDir.mkdir(); - } - - final ProcessorTopology processorTopology = new ProcessorTopology( - Collections.emptyList(), - Collections.emptyMap(), - Collections.emptyMap(), - Collections.singletonList(store), - Collections.emptyList(), - Collections.singletonMap(store.name(), store.name() + "-changelog"), - Collections.emptySet(), - Collections.emptyMap() - ); - Mockito.when(metadata.buildSubtopology(ArgumentMatchers.any())).thenReturn(processorTopology); - Mockito.when(metadata.taskConfig(ArgumentMatchers.any())).thenReturn(topologyConfig.getTaskConfig()); - - directory.initializeStartupTasks(metadata, new StreamsMetricsImpl(new Metrics(), "test", "processId", time), new LogContext("test")); - - return store; - } - private static class FutureStateDirectoryProcessFile { @JsonProperty diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index 1b4143eceee84..740d7d2c8f968 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -109,7 +109,6 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; -import org.junit.jupiter.params.provider.ValueSource; import org.mockito.InOrder; import org.mockito.Mock; import org.mockito.Mockito; @@ -204,7 +203,7 @@ public class StreamThreadTest { private final ChangelogReader changelogReader = new MockChangelogReader(); private StateDirectory stateDirectory = null; private final InternalTopologyBuilder internalTopologyBuilder = new InternalTopologyBuilder(); - private final InternalStreamsBuilder internalStreamsBuilder = new InternalStreamsBuilder(internalTopologyBuilder); + private final InternalStreamsBuilder internalStreamsBuilder = new InternalStreamsBuilder(internalTopologyBuilder, false); private StreamThread thread = null; @@ -917,7 +916,6 @@ public void shouldRespectNumIterationsInMainLoopWithoutProcessingThreads(final b thread = createStreamThread(CLIENT_ID, config); thread.setState(StreamThread.State.STARTING); - thread.taskManager().init(); thread.setState(StreamThread.State.PARTITIONS_REVOKED); final TaskId task1 = new TaskId(0, t1p1.partition()); @@ -1163,7 +1161,6 @@ public void shouldRecordCommitLatency(final boolean stateUpdaterEnabled, final b new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID.toString(), mockTime); final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); topologyMetadata.buildAndRewriteTopology(); - stateDirectory = new StateDirectory(config, mockTime, true, false); final TaskManager taskManager = new TaskManager( new MockTime(), @@ -1175,7 +1172,7 @@ public void shouldRecordCommitLatency(final boolean stateUpdaterEnabled, final b new Tasks(new LogContext()), topologyMetadata, null, - stateDirectory, + null, stateUpdater, schedulingTaskManager ) { @@ -1292,7 +1289,6 @@ public void shouldInjectProducerPerThreadUsingClientSupplierOnCreateIfEosV2Enabl thread = createStreamThread(CLIENT_ID, new StreamsConfig(props)); thread.setState(StreamThread.State.STARTING); - thread.taskManager().init(); thread.rebalanceListener().onPartitionsRevoked(Collections.emptyList()); final Map> activeTasks = new HashMap<>(); @@ -1550,7 +1546,6 @@ public void shouldNotCloseTaskAndRemoveFromTaskManagerIfProducerWasFencedWhilePr consumer.updatePartitions(topic1, Collections.singletonList(new PartitionInfo(topic1, 1, null, null, null))); thread.setState(StreamThread.State.STARTING); - thread.taskManager().init(); thread.rebalanceListener().onPartitionsRevoked(Collections.emptySet()); final Map> activeTasks = new HashMap<>(); @@ -1616,7 +1611,6 @@ private void testThrowingDurringCommitTransactionException(final RuntimeExceptio internalTopologyBuilder.addSink("out", "output", null, null, null, "name"); thread.setState(StreamThread.State.STARTING); - thread.taskManager().init(); thread.rebalanceListener().onPartitionsRevoked(Collections.emptySet()); final Map> activeTasks = new HashMap<>(); @@ -1699,7 +1693,6 @@ public void shouldReinitializeRevivedTasksInAnyState(final boolean stateUpdaterE internalTopologyBuilder.buildTopology(); thread.setState(StreamThread.State.STARTING); - thread.taskManager().init(); thread.rebalanceListener().onPartitionsRevoked(Collections.emptySet()); final Map> activeTasks = new HashMap<>(); @@ -1794,7 +1787,6 @@ private void testNotCloseTaskAndRemoveFromTaskManagerInCommitTransactionWhenComm consumer.updatePartitions(topic1, Collections.singletonList(new PartitionInfo(topic1, 1, null, null, null))); thread.setState(StreamThread.State.STARTING); - thread.taskManager().init(); thread.rebalanceListener().onPartitionsRevoked(Collections.emptySet()); final Map> activeTasks = new HashMap<>(); @@ -1859,7 +1851,6 @@ public void shouldNotCloseTaskProducerWhenSuspending(final boolean stateUpdaterE internalTopologyBuilder.addSink("out", "output", null, null, null, "name"); thread.setState(StreamThread.State.STARTING); - thread.taskManager().init(); thread.rebalanceListener().onPartitionsRevoked(Collections.emptySet()); final Map> activeTasks = new HashMap<>(); @@ -1941,7 +1932,6 @@ public void shouldReturnActiveTaskMetadataWhileRunningState(final boolean stateU ); thread.setState(StreamThread.State.STARTING); - thread.taskManager().init(); thread.rebalanceListener().onPartitionsRevoked(Collections.emptySet()); final Map> activeTasks = new HashMap<>(); @@ -2002,7 +1992,6 @@ public void shouldReturnStandbyTaskMetadataWhileRunningState(final boolean state restoreConsumer.updateBeginningOffsets(offsets); thread.setState(StreamThread.State.STARTING); - thread.taskManager().init(); thread.rebalanceListener().onPartitionsRevoked(Collections.emptySet()); final Map> standbyTasks = new HashMap<>(); @@ -2266,7 +2255,6 @@ public void process(final Record record) {} thread = createStreamThread(CLIENT_ID, config); thread.setState(StreamThread.State.STARTING); - thread.taskManager().init(); thread.rebalanceListener().onPartitionsRevoked(Collections.emptySet()); final List assignedPartitions = new ArrayList<>(); @@ -2346,7 +2334,6 @@ public void process(final Record record) {} thread = createStreamThread(CLIENT_ID, stateUpdaterEnabled, processingThreadsEnabled); thread.setState(StreamThread.State.STARTING); - thread.taskManager().init(); thread.rebalanceListener().onPartitionsRevoked(Collections.emptySet()); final List assignedPartitions = new ArrayList<>(); @@ -2544,7 +2531,6 @@ public void shouldLogAndRecordSkippedMetricForDeserializationException(final boo thread = createStreamThread(CLIENT_ID, new StreamsConfig(properties)); thread.setState(StreamThread.State.STARTING); - thread.taskManager().init(); thread.setState(StreamThread.State.PARTITIONS_REVOKED); final TaskId task1 = new TaskId(0, t1p1.partition()); @@ -3031,7 +3017,6 @@ public void shouldLogAndRecordSkippedRecordsForInvalidTimestamps( thread = createStreamThread(CLIENT_ID, config); thread.setState(StreamThread.State.STARTING); - thread.taskManager().init(); thread.setState(StreamThread.State.PARTITIONS_REVOKED); final TaskId task1 = new TaskId(0, t1p1.partition()); @@ -3405,7 +3390,6 @@ private void getClientInstanceId(final boolean injectTimeException, final boolea thread = createStreamThread("clientId", stateUpdaterEnabled, processingThreadsEnabled); thread.setState(State.STARTING); - thread.taskManager().init(); final Map> clientInstanceIdFutures = thread.clientInstanceIds(Duration.ZERO); @@ -3430,7 +3414,6 @@ private void getClientInstanceId(final boolean injectTimeException, final boolea public void shouldReturnErrorIfMainConsumerInstanceIdNotInitialized(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) { thread = createStreamThread("clientId", stateUpdaterEnabled, processingThreadsEnabled); thread.setState(State.STARTING); - thread.taskManager().init(); final Map> consumerFutures = thread.clientInstanceIds(Duration.ZERO); @@ -3447,7 +3430,6 @@ public void shouldReturnErrorIfMainConsumerInstanceIdNotInitialized(final boolea public void shouldReturnErrorIfRestoreConsumerInstanceIdNotInitialized(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) { thread = createStreamThread("clientId", stateUpdaterEnabled, processingThreadsEnabled); thread.setState(State.STARTING); - thread.taskManager().init(); final Map> consumerFutures = thread.clientInstanceIds(Duration.ZERO); @@ -3464,7 +3446,6 @@ public void shouldReturnErrorIfRestoreConsumerInstanceIdNotInitialized(final boo public void shouldReturnErrorIfProducerInstanceIdNotInitialized(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) { thread = createStreamThread("clientId", stateUpdaterEnabled, processingThreadsEnabled); thread.setState(State.STARTING); - thread.taskManager().init(); final Map> producerFutures = thread.clientInstanceIds(Duration.ZERO); @@ -3482,7 +3463,6 @@ public void shouldReturnNullIfMainConsumerTelemetryDisabled(final boolean stateU clientSupplier.consumer.disableTelemetry(); thread = createStreamThread("clientId", stateUpdaterEnabled, processingThreadsEnabled); thread.setState(State.STARTING); - thread.taskManager().init(); final Map> consumerFutures = thread.clientInstanceIds(Duration.ZERO); @@ -3500,7 +3480,6 @@ public void shouldReturnNullIfRestoreConsumerTelemetryDisabled(final boolean sta thread = createStreamThread("clientId", stateUpdaterEnabled, processingThreadsEnabled); thread.setState(State.STARTING); - thread.taskManager().init(); final Map> consumerFutures = thread.clientInstanceIds(Duration.ZERO); @@ -3520,7 +3499,6 @@ public void shouldReturnNullIfProducerTelemetryDisabled(final boolean stateUpdat thread = createStreamThread("clientId", stateUpdaterEnabled, processingThreadsEnabled); thread.setState(State.STARTING); - thread.taskManager().init(); final Map> producerFutures = thread.clientInstanceIds(Duration.ZERO); @@ -3538,7 +3516,6 @@ public void shouldTimeOutOnMainConsumerInstanceId(final boolean stateUpdaterEnab clientSupplier.consumer.injectTimeoutException(-1); thread = createStreamThread("clientId", stateUpdaterEnabled, processingThreadsEnabled); thread.setState(State.STARTING); - thread.taskManager().init(); final Map> consumerFutures = thread.clientInstanceIds(Duration.ZERO); @@ -3563,7 +3540,6 @@ public void shouldTimeOutOnRestoreConsumerInstanceId(final boolean stateUpdaterE clientSupplier.restoreConsumer.injectTimeoutException(-1); thread = createStreamThread("clientId", stateUpdaterEnabled, processingThreadsEnabled); thread.setState(State.STARTING); - thread.taskManager().init(); final Map> consumerFutures = thread.clientInstanceIds(Duration.ZERO); @@ -3591,7 +3567,6 @@ public void shouldTimeOutOnProducerInstanceId(final boolean stateUpdaterEnabled, thread = createStreamThread("clientId", stateUpdaterEnabled, processingThreadsEnabled); thread.setState(State.STARTING); - thread.taskManager().init(); final Map> producerFutures = thread.clientInstanceIds(Duration.ZERO); @@ -3608,10 +3583,9 @@ public void shouldTimeOutOnProducerInstanceId(final boolean stateUpdaterEnabled, ); } - @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testNamedTopologyWithStreamsProtocol(final boolean stateUpdaterEnabled) { - final Properties props = configProps(false, stateUpdaterEnabled, false); + @Test + public void testNamedTopologyWithStreamsProtocol() { + final Properties props = configProps(false, false, false); props.setProperty(StreamsConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.STREAMS.toString()); final StreamsConfig config = new StreamsConfig(props); final InternalTopologyBuilder topologyBuilder = new InternalTopologyBuilder( @@ -3668,10 +3642,9 @@ public void testStreamsRebalanceDataWithClassicProtocol() { assertTrue(thread.streamsRebalanceData().isEmpty()); } - @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testStreamsRebalanceDataWithExtraCopartition(final boolean stateUpdaterEnabled) { - final Properties props = configProps(false, stateUpdaterEnabled, false); + @Test + public void testStreamsRebalanceDataWithExtraCopartition() { + final Properties props = configProps(false, false, false); props.setProperty(StreamsConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.STREAMS.toString()); internalTopologyBuilder.addSource(null, "source1", null, null, null, topic1); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java index b34d1408c5620..52253f78b2329 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java @@ -2622,7 +2622,7 @@ public void shouldThrowTaskAssignmentExceptionWhenUnableToResolvePartitionCount( builder = new CorruptedInternalTopologyBuilder(); topologyMetadata = new TopologyMetadata(builder, new StreamsConfig(configProps(parameterizedConfig))); - final InternalStreamsBuilder streamsBuilder = new InternalStreamsBuilder(builder); + final InternalStreamsBuilder streamsBuilder = new InternalStreamsBuilder(builder, false); final KStream inputTopic = streamsBuilder.stream(singleton("topic1"), new ConsumedInternal<>(Consumed.with(null, null))); final KTable inputTable = streamsBuilder.table("topic2", new ConsumedInternal<>(Consumed.with(null, null)), new MaterializedInternal<>(Materialized.as("store"))); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java index 26a1523131b38..00b4d5d8a07fc 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java @@ -61,8 +61,6 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; import org.mockito.InOrder; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; @@ -4719,138 +4717,6 @@ public void shouldListNotPausedTasks() { assertEquals(taskManager.notPausedTasks().size(), 0); } - @Test - public void shouldRecycleStartupTasksFromStateDirectoryAsActive() { - final StandbyTask startupTask = standbyTask(taskId00, taskId00ChangelogPartitions).build(); - final StreamTask activeTask = statefulTask(taskId00, taskId00ChangelogPartitions).build(); - when(activeTaskCreator.createActiveTaskFromStandby(eq(startupTask), eq(taskId00Partitions), any())) - .thenReturn(activeTask); - - when(stateDirectory.hasStartupTasks()).thenReturn(true, false); - when(stateDirectory.removeStartupTask(taskId00)).thenReturn(startupTask, (Task) null); - - taskManager.handleAssignment(taskId00Assignment, Collections.emptyMap()); - - // ensure we recycled our existing startup Standby into an Active task - verify(activeTaskCreator).createActiveTaskFromStandby(eq(startupTask), eq(taskId00Partitions), any()); - - // ensure we didn't construct any new Tasks - verify(activeTaskCreator).createTasks(any(), eq(Collections.emptyMap())); - verify(standbyTaskCreator).createTasks(Collections.emptyMap()); - verifyNoMoreInteractions(activeTaskCreator); - verifyNoMoreInteractions(standbyTaskCreator); - - // verify the recycled task is now being used as an assigned Active - assertEquals(Collections.singletonMap(taskId00, activeTask), taskManager.activeTaskMap()); - assertEquals(Collections.emptyMap(), taskManager.standbyTaskMap()); - } - - @Test - public void shouldUseStartupTasksFromStateDirectoryAsStandby() { - final StandbyTask startupTask = standbyTask(taskId00, taskId00ChangelogPartitions).build(); - - when(stateDirectory.hasStartupTasks()).thenReturn(true, true, false); - when(stateDirectory.removeStartupTask(taskId00)).thenReturn(startupTask, (Task) null); - - taskManager.handleAssignment(Collections.emptyMap(), taskId00Assignment); - - // ensure we used our existing startup Task directly as a Standby - verify(startupTask).resume(); - - // ensure we didn't construct any new Tasks, or recycle an existing Task; we only used the one we already have - verify(activeTaskCreator).createTasks(any(), eq(Collections.emptyMap())); - verify(standbyTaskCreator).createTasks(Collections.emptyMap()); - verifyNoMoreInteractions(activeTaskCreator); - verifyNoMoreInteractions(standbyTaskCreator); - - // verify the startup Standby is now being used as an assigned Standby - assertEquals(Collections.emptyMap(), taskManager.activeTaskMap()); - assertEquals(Collections.singletonMap(taskId00, startupTask), taskManager.standbyTaskMap()); - } - - @Test - public void shouldRecycleStartupTasksFromStateDirectoryAsActiveWithStateUpdater() { - final Tasks taskRegistry = new Tasks(new LogContext()); - final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, taskRegistry, true); - final StandbyTask startupTask = standbyTask(taskId00, taskId00ChangelogPartitions).build(); - - final StreamTask activeTask = statefulTask(taskId00, taskId00ChangelogPartitions).build(); - when(activeTaskCreator.createActiveTaskFromStandby(eq(startupTask), eq(taskId00Partitions), any())) - .thenReturn(activeTask); - - when(stateDirectory.hasStartupTasks()).thenReturn(true, false); - when(stateDirectory.removeStartupTask(taskId00)).thenReturn(startupTask, (Task) null); - - taskManager.handleAssignment(taskId00Assignment, Collections.emptyMap()); - - // ensure we used our existing startup Task directly as a Standby - assertTrue(taskRegistry.hasPendingTasksToInit()); - assertEquals(Collections.singleton(activeTask), taskRegistry.drainPendingTasksToInit()); - - // we're using a mock StateUpdater here, so now that we've drained the task from the queue of startup tasks to init - // let's "add" it to our mock StateUpdater - when(stateUpdater.tasks()).thenReturn(Collections.singleton(activeTask)); - when(stateUpdater.standbyTasks()).thenReturn(Collections.emptySet()); - - // ensure we recycled our existing startup Standby into an Active task - verify(activeTaskCreator).createActiveTaskFromStandby(eq(startupTask), eq(taskId00Partitions), any()); - - // ensure we didn't construct any new Tasks - verify(activeTaskCreator).createTasks(any(), eq(Collections.emptyMap())); - verify(standbyTaskCreator).createTasks(Collections.emptyMap()); - verifyNoMoreInteractions(activeTaskCreator); - verifyNoMoreInteractions(standbyTaskCreator); - - // verify the recycled task is now being used as an assigned Active - assertEquals(Collections.singletonMap(taskId00, activeTask), taskManager.activeTaskMap()); - assertEquals(Collections.emptyMap(), taskManager.standbyTaskMap()); - } - - @Test - public void shouldUseStartupTasksFromStateDirectoryAsStandbyWithStateUpdater() { - final Tasks taskRegistry = new Tasks(new LogContext()); - final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, taskRegistry, true); - final StandbyTask startupTask = standbyTask(taskId00, taskId00ChangelogPartitions).build(); - - when(stateDirectory.hasStartupTasks()).thenReturn(true, true, false); - when(stateDirectory.removeStartupTask(taskId00)).thenReturn(startupTask, (Task) null); - - assertFalse(taskRegistry.hasPendingTasksToInit()); - - taskManager.handleAssignment(Collections.emptyMap(), taskId00Assignment); - - // ensure we used our existing startup Task directly as a Standby - assertTrue(taskRegistry.hasPendingTasksToInit()); - assertEquals(Collections.singleton(startupTask), taskRegistry.drainPendingTasksToInit()); - - // we're using a mock StateUpdater here, so now that we've drained the task from the queue of startup tasks to init - // let's "add" it to our mock StateUpdater - when(stateUpdater.tasks()).thenReturn(Collections.singleton(startupTask)); - when(stateUpdater.standbyTasks()).thenReturn(Collections.singleton(startupTask)); - - // ensure we didn't construct any new Tasks, or recycle an existing Task; we only used the one we already have - verify(activeTaskCreator).createTasks(any(), eq(Collections.emptyMap())); - verify(standbyTaskCreator).createTasks(Collections.emptyMap()); - verifyNoMoreInteractions(activeTaskCreator); - verifyNoMoreInteractions(standbyTaskCreator); - - // verify the startup Standby is now being used as an assigned Standby - assertEquals(Collections.emptyMap(), taskManager.activeTaskMap()); - assertEquals(Collections.singletonMap(taskId00, startupTask), taskManager.standbyTaskMap()); - } - - @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void shouldStartStateUpdaterOnInit(final boolean stateUpdaterEnabled) { - final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, stateUpdaterEnabled); - taskManager.init(); - if (stateUpdaterEnabled) { - verify(stateUpdater).start(); - } else { - verify(stateUpdater, never()).start(); - } - } - private static KafkaFutureImpl completedFuture() { final KafkaFutureImpl futureDeletedRecords = new KafkaFutureImpl<>(); futureDeletedRecords.complete(null); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStoreTest.java index 9a23e657600cf..d3243ef2fc669 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStoreTest.java @@ -16,12 +16,10 @@ */ package org.apache.kafka.streams.state.internals; -import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.internals.SessionWindow; import org.apache.kafka.streams.processor.internals.ProcessorContextImpl; -import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; import org.apache.kafka.streams.query.Position; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.SessionStore; @@ -77,7 +75,6 @@ public void shouldDelegateInit() { public void shouldLogPuts() { final Bytes binaryKey = SessionKeySchema.toBinary(key1); when(inner.getPosition()).thenReturn(Position.emptyPosition()); - when(context.recordContext()).thenReturn(new ProcessorRecordContext(0, 0, 0, "topic", new RecordHeaders())); store.put(key1, value1); @@ -89,7 +86,6 @@ public void shouldLogPuts() { public void shouldLogPutsWithPosition() { final Bytes binaryKey = SessionKeySchema.toBinary(key1); when(inner.getPosition()).thenReturn(POSITION); - when(context.recordContext()).thenReturn(new ProcessorRecordContext(0, 0, 0, "topic", new RecordHeaders())); store.put(key1, value1); @@ -101,7 +97,6 @@ public void shouldLogPutsWithPosition() { public void shouldLogRemoves() { final Bytes binaryKey = SessionKeySchema.toBinary(key1); when(inner.getPosition()).thenReturn(Position.emptyPosition()); - when(context.recordContext()).thenReturn(new ProcessorRecordContext(0, 0, 0, "topic", new RecordHeaders())); store.remove(key1); store.remove(key1); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedWindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedWindowBytesStoreTest.java index 1c1b713ce21a0..03701bdcb0086 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedWindowBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedWindowBytesStoreTest.java @@ -17,11 +17,9 @@ package org.apache.kafka.streams.state.internals; -import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.processor.internals.ProcessorContextImpl; -import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; import org.apache.kafka.streams.query.Position; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.WindowStore; @@ -79,9 +77,8 @@ public void shouldDelegateInit() { public void shouldLogPuts() { final Bytes key = WindowKeySchema.toStoreKeyBinary(bytesKey, 0, 0); when(inner.getPosition()).thenReturn(Position.emptyPosition()); - when(context.recordContext()).thenReturn(new ProcessorRecordContext(0, 0, 0, "topic", new RecordHeaders())); - store.put(bytesKey, valueAndTimestamp, context.recordContext().timestamp()); + store.put(bytesKey, valueAndTimestamp, context.timestamp()); verify(inner).put(bytesKey, valueAndTimestamp, 0); verify(context).logChange(store.name(), key, value, 42, Position.emptyPosition()); @@ -91,9 +88,8 @@ public void shouldLogPuts() { public void shouldLogPutsWithPosition() { final Bytes key = WindowKeySchema.toStoreKeyBinary(bytesKey, 0, 0); when(inner.getPosition()).thenReturn(POSITION); - when(context.recordContext()).thenReturn(new ProcessorRecordContext(0, 0, 0, "topic", new RecordHeaders())); - store.put(bytesKey, valueAndTimestamp, context.recordContext().timestamp()); + store.put(bytesKey, valueAndTimestamp, context.timestamp()); verify(inner).put(bytesKey, valueAndTimestamp, 0); verify(context).logChange(store.name(), key, value, 42, POSITION); @@ -122,10 +118,9 @@ public void shouldRetainDuplicatesWhenSet() { final Bytes key1 = WindowKeySchema.toStoreKeyBinary(bytesKey, 0, 1); final Bytes key2 = WindowKeySchema.toStoreKeyBinary(bytesKey, 0, 2); when(inner.getPosition()).thenReturn(Position.emptyPosition()); - when(context.recordContext()).thenReturn(new ProcessorRecordContext(0, 0, 0, "topic", new RecordHeaders())); - store.put(bytesKey, valueAndTimestamp, context.recordContext().timestamp()); - store.put(bytesKey, valueAndTimestamp, context.recordContext().timestamp()); + store.put(bytesKey, valueAndTimestamp, context.timestamp()); + store.put(bytesKey, valueAndTimestamp, context.timestamp()); verify(inner, times(2)).put(bytesKey, valueAndTimestamp, 0); verify(context).logChange(store.name(), key1, value, 42L, Position.emptyPosition()); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java index e80a2325a2a54..2607e56ad9ff1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java @@ -17,11 +17,9 @@ package org.apache.kafka.streams.state.internals; -import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.processor.internals.ProcessorContextImpl; -import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; import org.apache.kafka.streams.query.Position; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.WindowStore; @@ -78,9 +76,8 @@ public void shouldDelegateInit() { public void shouldLogPuts() { final Bytes key = WindowKeySchema.toStoreKeyBinary(bytesKey, 0, 0); when(inner.getPosition()).thenReturn(Position.emptyPosition()); - when(context.recordContext()).thenReturn(new ProcessorRecordContext(0, 0, 0, "topic", new RecordHeaders())); - store.put(bytesKey, value, context.recordContext().timestamp()); + store.put(bytesKey, value, context.timestamp()); verify(inner).put(bytesKey, value, 0); verify(context).logChange(store.name(), key, value, 0L, Position.emptyPosition()); @@ -90,9 +87,8 @@ public void shouldLogPuts() { public void shouldLogPutsWithPosition() { final Bytes key = WindowKeySchema.toStoreKeyBinary(bytesKey, 0, 0); when(inner.getPosition()).thenReturn(POSITION); - when(context.recordContext()).thenReturn(new ProcessorRecordContext(0, 0, 0, "topic", new RecordHeaders())); - store.put(bytesKey, value, context.recordContext().timestamp()); + store.put(bytesKey, value, context.timestamp()); verify(inner).put(bytesKey, value, 0); verify(context).logChange(store.name(), key, value, 0L, POSITION); @@ -135,13 +131,12 @@ public void shouldRetainDuplicatesWhenSet() { store = new ChangeLoggingWindowBytesStore(inner, true, WindowKeySchema::toStoreKeyBinary); store.init(context, store); when(inner.getPosition()).thenReturn(Position.emptyPosition()); - when(context.recordContext()).thenReturn(new ProcessorRecordContext(0, 0, 0, "topic", new RecordHeaders())); final Bytes key1 = WindowKeySchema.toStoreKeyBinary(bytesKey, 0, 1); final Bytes key2 = WindowKeySchema.toStoreKeyBinary(bytesKey, 0, 2); - store.put(bytesKey, value, context.recordContext().timestamp()); - store.put(bytesKey, value, context.recordContext().timestamp()); + store.put(bytesKey, value, context.timestamp()); + store.put(bytesKey, value, context.timestamp()); verify(inner, times(2)).put(bytesKey, value, 0); verify(context).logChange(store.name(), key1, value, 0L, Position.emptyPosition()); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java index 4a8c891355dd1..294af3944f2b0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java @@ -525,15 +525,16 @@ public void shouldTrackOldestOpenIteratorTimestamp() { when(inner.all()).thenReturn(KeyValueIterators.emptyIterator()); init(); - final KafkaMetric oldestIteratorTimestampMetric = metric("oldest-iterator-open-since-ms"); - assertThat(oldestIteratorTimestampMetric, not(nullValue())); - - assertThat(oldestIteratorTimestampMetric.metricValue(), nullValue()); + KafkaMetric oldestIteratorTimestampMetric = metric("oldest-iterator-open-since-ms"); + assertThat(oldestIteratorTimestampMetric, nullValue()); KeyValueIterator second = null; final long secondTimestamp; try { try (final KeyValueIterator unused = metered.all()) { + oldestIteratorTimestampMetric = metric("oldest-iterator-open-since-ms"); + assertThat(oldestIteratorTimestampMetric, not(nullValue())); + final long oldestTimestamp = mockTime.milliseconds(); assertThat((Long) oldestIteratorTimestampMetric.metricValue(), equalTo(oldestTimestamp)); mockTime.sleep(100); @@ -553,7 +554,8 @@ public void shouldTrackOldestOpenIteratorTimestamp() { } } - assertThat((Integer) oldestIteratorTimestampMetric.metricValue(), nullValue()); + oldestIteratorTimestampMetric = metric("oldest-iterator-open-since-ms"); + assertThat(oldestIteratorTimestampMetric, nullValue()); } private KafkaMetric metric(final MetricName metricName) { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStoreTest.java index fa42cb0728335..2e3c470387c2b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStoreTest.java @@ -503,15 +503,16 @@ public void shouldTrackOldestOpenIteratorTimestamp() { when(inner.all()).thenReturn(KeyValueIterators.emptyIterator()); init(); - final KafkaMetric oldestIteratorTimestampMetric = metric("oldest-iterator-open-since-ms"); - assertThat(oldestIteratorTimestampMetric, not(nullValue())); - - assertThat(oldestIteratorTimestampMetric.metricValue(), nullValue()); + KafkaMetric oldestIteratorTimestampMetric = metric("oldest-iterator-open-since-ms"); + assertThat(oldestIteratorTimestampMetric, nullValue()); KeyValueIterator> second = null; final long secondTimestamp; try { try (final KeyValueIterator> unused = metered.all()) { + oldestIteratorTimestampMetric = metric("oldest-iterator-open-since-ms"); + assertThat(oldestIteratorTimestampMetric, not(nullValue())); + final long oldestTimestamp = mockTime.milliseconds(); assertThat((Long) oldestIteratorTimestampMetric.metricValue(), equalTo(oldestTimestamp)); mockTime.sleep(100); @@ -531,6 +532,7 @@ public void shouldTrackOldestOpenIteratorTimestamp() { } } - assertThat((Integer) oldestIteratorTimestampMetric.metricValue(), nullValue()); + oldestIteratorTimestampMetric = metric("oldest-iterator-open-since-ms"); + assertThat(oldestIteratorTimestampMetric, nullValue()); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStoreTest.java index f0a7f23c09c78..8e8e02b272277 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStoreTest.java @@ -426,16 +426,17 @@ public void shouldTrackOldestOpenIteratorTimestamp() { when(inner.query(any(), any(), any())).thenReturn( QueryResult.forResult(new LogicalSegmentIterator(Collections.emptyListIterator(), RAW_KEY, 0L, 0L, ResultOrder.ANY))); - final KafkaMetric oldestIteratorTimestampMetric = getMetric("oldest-iterator-open-since-ms"); - assertThat(oldestIteratorTimestampMetric, not(nullValue())); - - assertThat(oldestIteratorTimestampMetric.metricValue(), nullValue()); + KafkaMetric oldestIteratorTimestampMetric = getMetric("oldest-iterator-open-since-ms"); + assertThat(oldestIteratorTimestampMetric, nullValue()); final QueryResult> first = store.query(query, bound, config); VersionedRecordIterator secondIterator = null; final long secondTime; try { try (final VersionedRecordIterator unused = first.getResult()) { + oldestIteratorTimestampMetric = getMetric("oldest-iterator-open-since-ms"); + assertThat(oldestIteratorTimestampMetric, not(nullValue())); + final long oldestTimestamp = mockTime.milliseconds(); assertThat((Long) oldestIteratorTimestampMetric.metricValue(), equalTo(oldestTimestamp)); mockTime.sleep(100); @@ -457,7 +458,8 @@ public void shouldTrackOldestOpenIteratorTimestamp() { } } - assertThat((Integer) oldestIteratorTimestampMetric.metricValue(), nullValue()); + oldestIteratorTimestampMetric = getMetric("oldest-iterator-open-since-ms"); + assertThat(oldestIteratorTimestampMetric, nullValue()); } private KafkaMetric getMetric(final String name) { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java index 1c8935d1e1c7c..ba557104ebdef 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java @@ -223,10 +223,10 @@ public void shouldRecordRestoreLatencyOnInit() { @Test public void shouldPutToInnerStoreAndRecordPutMetrics() { final byte[] bytes = "a".getBytes(); - doNothing().when(innerStoreMock).put(eq(Bytes.wrap(bytes)), any(), eq(context.recordContext().timestamp())); + doNothing().when(innerStoreMock).put(eq(Bytes.wrap(bytes)), any(), eq(context.timestamp())); store.init(context, store); - store.put("a", "a", context.recordContext().timestamp()); + store.put("a", "a", context.timestamp()); // it suffices to verify one put metric since all put metrics are recorded by the same sensor // and the sensor is tested elsewhere diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedCachingPersistentWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedCachingPersistentWindowStoreTest.java index ffa509d518871..21d16b09be419 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedCachingPersistentWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedCachingPersistentWindowStoreTest.java @@ -935,9 +935,9 @@ public void shouldSkipNonExistBaseKeyInCache(final boolean hasIndex) { new byte[0], new RecordHeaders(), true, - context.recordContext().offset(), - context.recordContext().timestamp(), - context.recordContext().partition(), + context.offset(), + context.timestamp(), + context.partition(), "", context.recordContext().sourceRawKey(), context.recordContext().sourceRawValue() diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedWindowStoreTest.java index 9d0db9bae0fbb..f4ff30002ae14 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedWindowStoreTest.java @@ -941,9 +941,9 @@ public void shouldSkipNonExistBaseKeyInCache(final boolean hasIndex) { new byte[0], new RecordHeaders(), true, - context.recordContext().offset(), - context.recordContext().timestamp(), - context.recordContext().partition(), + context.offset(), + context.timestamp(), + context.partition(), "", context.recordContext().sourceRawKey(), context.recordContext().sourceRawValue() diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java index 68e6c27592cfa..59698607912c4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java @@ -48,11 +48,13 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Properties; import java.util.Random; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiPredicate; import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -512,21 +514,34 @@ private static VerificationResult verifyAll(final Map> inpu final boolean printResults, final boolean eosEnabled) { final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + final BiPredicate validationPredicate; + if (eosEnabled) { + validationPredicate = Objects::equals; + } else { + validationPredicate = SmokeTestDriver::lessEquals; + } boolean pass; try (final PrintStream resultStream = new PrintStream(byteArrayOutputStream)) { - pass = verifyTAgg(resultStream, inputs, events.get("tagg"), printResults); + pass = true; + if (eosEnabled) { + // TAGG is computing "Count-by-count", which may produce keys that are not in the input data in ALOS, so we skip validation in this case. + pass = verifyTAgg(resultStream, inputs, events.get("tagg"), printResults); + } pass &= verifySuppressed(resultStream, "min-suppressed", events, printResults); pass &= verify(resultStream, "min-suppressed", inputs, events, windowedKey -> { final String unwindowedKey = windowedKey.substring(1, windowedKey.length() - 1).replaceAll("@.*", ""); return getMin(unwindowedKey); - }, printResults, eosEnabled); + }, Object::equals, printResults, eosEnabled); pass &= verifySuppressed(resultStream, "sws-suppressed", events, printResults); - pass &= verify(resultStream, "min", inputs, events, SmokeTestDriver::getMin, printResults, eosEnabled); - pass &= verify(resultStream, "max", inputs, events, SmokeTestDriver::getMax, printResults, eosEnabled); - pass &= verify(resultStream, "dif", inputs, events, key -> getMax(key).intValue() - getMin(key).intValue(), printResults, eosEnabled); - pass &= verify(resultStream, "sum", inputs, events, SmokeTestDriver::getSum, printResults, eosEnabled); - pass &= verify(resultStream, "cnt", inputs, events, key1 -> getMax(key1).intValue() - getMin(key1).intValue() + 1L, printResults, eosEnabled); - pass &= verify(resultStream, "avg", inputs, events, SmokeTestDriver::getAvg, printResults, eosEnabled); + pass &= verify(resultStream, "min", inputs, events, SmokeTestDriver::getMin, Object::equals, printResults, eosEnabled); + pass &= verify(resultStream, "max", inputs, events, SmokeTestDriver::getMax, Object::equals, printResults, eosEnabled); + pass &= verify(resultStream, "dif", inputs, events, key -> getMax(key).intValue() - getMin(key).intValue(), Object::equals, printResults, eosEnabled); + pass &= verify(resultStream, "sum", inputs, events, SmokeTestDriver::getSum, validationPredicate, printResults, eosEnabled); + pass &= verify(resultStream, "cnt", inputs, events, key1 -> getMax(key1).intValue() - getMin(key1).intValue() + 1L, validationPredicate, printResults, eosEnabled); + if (eosEnabled) { + // Average can overcount and undercount in ALOS, so we skip validation in that case. + pass &= verify(resultStream, "avg", inputs, events, SmokeTestDriver::getAvg, Object::equals, printResults, eosEnabled); + } } return new VerificationResult(pass, new String(byteArrayOutputStream.toByteArray(), StandardCharsets.UTF_8)); } @@ -536,6 +551,7 @@ private static boolean verify(final PrintStream resultStream, final Map> inputData, final Map>>> events, final Function keyToExpectation, + final BiPredicate validationPredicate, final boolean printResults, final boolean eosEnabled) { resultStream.printf("verifying topic '%s'%n", topic); @@ -561,18 +577,17 @@ private static boolean verify(final PrintStream resultStream, } } - for (final Map.Entry>> entry : outputEvents.entrySet()) { final String key = entry.getKey(); final Number expected = keyToExpectation.apply(key); final Number actual = entry.getValue().getLast().value(); - if (!expected.equals(actual)) { + if (!validationPredicate.test(expected, actual)) { resultStream.printf("%s fail: key=%s actual=%s expected=%s%n", topic, key, actual, expected); if (printResults) { resultStream.printf("\t inputEvents=%n%s%n\t" + "echoEvents=%n%s%n\tmaxEvents=%n%s%n\tminEvents=%n%s%n\tdifEvents=%n%s%n\tcntEvents=%n%s%n\ttaggEvents=%n%s%n", - indent("\t\t", observedInputEvents.get(key)), + indent("\t\t", observedInputEvents.getOrDefault(key, new LinkedList<>())), indent("\t\t", events.getOrDefault("echo", emptyMap()).getOrDefault(key, new LinkedList<>())), indent("\t\t", events.getOrDefault("max", emptyMap()).getOrDefault(key, new LinkedList<>())), indent("\t\t", events.getOrDefault("min", emptyMap()).getOrDefault(key, new LinkedList<>())), @@ -591,6 +606,18 @@ private static boolean verify(final PrintStream resultStream, } } + private static boolean lessEquals(final Number expected, final Number actual) { + if (actual instanceof Integer && expected instanceof Integer) { + return actual.intValue() >= expected.intValue(); + } else if (actual instanceof Long && expected instanceof Long) { + return actual.longValue() >= expected.longValue(); + } else if (actual instanceof Double && expected instanceof Double) { + return actual.doubleValue() >= expected.doubleValue(); + } else { + throw new IllegalArgumentException("Unexpected type: " + actual.getClass()); + } + } + private static boolean verifySuppressed(final PrintStream resultStream, @SuppressWarnings("SameParameterValue") final String topic, final Map>>> events, diff --git a/tests/README.md b/tests/README.md index 7a1d4e733945b..a64946c892a71 100644 --- a/tests/README.md +++ b/tests/README.md @@ -49,12 +49,13 @@ TC_PATHS="tests/kafkatest/tests/streams/streams_upgrade_test.py::StreamsUpgradeT ``` * Run tests with a specific image name ``` -image_name="ducker-ak-openjdk:17-buster" bash tests/docker/run_tests.sh +image_name="ducker-ak-openjdk:17" bash tests/docker/run_tests.sh ``` * Run tests with a different JVM ``` -bash tests/docker/ducker-ak up -j 'openjdk:17-buster'; tests/docker/run_tests.sh +bash tests/docker/ducker-ak up -j ''; tests/docker/run_tests.sh ``` + You can customize the OpenJDK base image using the `-j` or `--jdk` parameter, otherwise a default value will be used. * Remove ducker-ak containers ``` bash tests/docker/ducker-ak down -f diff --git a/tests/docker/Dockerfile b/tests/docker/Dockerfile index 87078db195b92..01c1a0533d3b1 100644 --- a/tests/docker/Dockerfile +++ b/tests/docker/Dockerfile @@ -13,9 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -# The base image of openjdk:17 is typically oraclelinux:8-slim, which doesn't include apt-get. -# Therefore, use openjdk:17-buster instead. -ARG jdk_version=openjdk:17-buster +ARG jdk_version FROM $jdk_version AS build-native-image WORKDIR /build diff --git a/tests/docker/ducker-ak b/tests/docker/ducker-ak index 05af5ac9bb122..68aed30923f96 100755 --- a/tests/docker/ducker-ak +++ b/tests/docker/ducker-ak @@ -45,7 +45,9 @@ docker_run_memory_limit="2000m" default_num_nodes=14 # The default OpenJDK base image. -default_jdk="openjdk:17-buster" +# The base image of openjdk:17 is typically oraclelinux:8-slim, which doesn't include apt-get. +# Therefore, use openjdk:17-bullseye instead. +default_jdk="openjdk:17-bullseye" # The default ducker-ak image name. default_image_name="ducker-ak" @@ -70,7 +72,7 @@ help|-h|--help Display this help message up [-n|--num-nodes NUM_NODES] [-f|--force] [docker-image] - [-C|--custom-ducktape DIR] [-e|--expose-ports ports] [--ipv6] + [-C|--custom-ducktape DIR] [-e|--expose-ports ports] [-j|--jdk JDK_VERSION] [--ipv6] Bring up a cluster with the specified amount of nodes (defaults to ${default_num_nodes}). The docker image name defaults to ${default_image_name}. If --force is specified, we will attempt to bring up an image even some parameters are not valid. @@ -84,6 +86,9 @@ up [-n|--num-nodes NUM_NODES] [-f|--force] [docker-image] or a combination of port/port-range separated by comma (like 2181,9092 or 2181,5005-5008). By default no port is exposed. See README.md for more detail on this option. + If -j|--jdk is specified, you can customize the OpenJDK base image used for building + the ducker container. Defaults to ${default_jdk}. Example: -j openjdk:17-bullseye + If --ipv6 is specified, we will create a Docker network with IPv6 enabled. Note that port 5678 will be automatically exposed for ducker01 node and will be mapped to 5678 diff --git a/tests/kafkatest/__init__.py b/tests/kafkatest/__init__.py index 99c9dfe22e7f8..9457c4a6322e7 100644 --- a/tests/kafkatest/__init__.py +++ b/tests/kafkatest/__init__.py @@ -22,4 +22,4 @@ # Instead, in development branches, the version should have a suffix of the form ".devN" # # For example, when Kafka is at version 1.0.0-SNAPSHOT, this should be something like "1.0.0.dev0" -__version__ = '4.1.0.dev0' +__version__ = '4.1.1.dev0' diff --git a/tests/kafkatest/tests/streams/base_streams_test.py b/tests/kafkatest/tests/streams/base_streams_test.py index 81cad7a4d1b8d..da00e0895f21e 100644 --- a/tests/kafkatest/tests/streams/base_streams_test.py +++ b/tests/kafkatest/tests/streams/base_streams_test.py @@ -82,8 +82,8 @@ def assert_produce_consume(self, self.assert_consume(client_id, test_state, streams_sink_topic, num_messages, timeout_sec) - def assert_produce(self, topic, test_state, num_messages=5, timeout_sec=60): - producer = self.get_producer(topic, num_messages) + def assert_produce(self, topic, test_state, num_messages=5, timeout_sec=60, repeating_keys=None): + producer = self.get_producer(topic, num_messages, repeating_keys=repeating_keys) producer.start() wait_until(lambda: producer.num_acked >= num_messages, diff --git a/tests/kafkatest/tests/streams/streams_application_upgrade_test.py b/tests/kafkatest/tests/streams/streams_application_upgrade_test.py index 78ea4141496bf..6831ec896e821 100644 --- a/tests/kafkatest/tests/streams/streams_application_upgrade_test.py +++ b/tests/kafkatest/tests/streams/streams_application_upgrade_test.py @@ -108,10 +108,10 @@ def test_app_upgrade(self, from_version, bounce_type, metadata_quorum): random.shuffle(self.processors) for p in self.processors: p.CLEAN_NODE_ENABLED = False - self.do_stop_start_bounce(p, None, to_version, counter) + self.do_stop_start_bounce(p, None, from_version, to_version, counter) counter = counter + 1 elif bounce_type == "full": - self.restart_all_nodes_with(to_version) + self.restart_all_nodes_with(from_version, to_version) else: raise Exception("Unrecognized bounce_type: " + str(bounce_type)) @@ -157,30 +157,36 @@ def start_all_nodes_with(self, version): self.wait_for_verification(self.processor2, self.processed_msg, self.processor2.STDOUT_FILE) self.wait_for_verification(self.processor3, self.processed_msg, self.processor3.STDOUT_FILE) - def restart_all_nodes_with(self, version): + def restart_all_nodes_with(self, from_version, to_version): self.processor1.stop_node(self.processor1.node) self.processor2.stop_node(self.processor2.node) self.processor3.stop_node(self.processor3.node) # make sure the members have stopped - self.wait_for_verification(self.processor1, "SMOKE-TEST-CLIENT-CLOSED", self.processor1.STDOUT_FILE) - self.wait_for_verification(self.processor2, "SMOKE-TEST-CLIENT-CLOSED", self.processor2.STDOUT_FILE) - self.wait_for_verification(self.processor3, "SMOKE-TEST-CLIENT-CLOSED", self.processor3.STDOUT_FILE) + if from_version.startswith("2."): + # some older versions crash on shutdown, so we allow crashes here. + self.wait_for_verification(self.processor1, "SMOKE-TEST-CLIENT-(EXCEPTION|CLOSED)", self.processor1.STDOUT_FILE) + self.wait_for_verification(self.processor2, "SMOKE-TEST-CLIENT-(EXCEPTION|CLOSED)", self.processor2.STDOUT_FILE) + self.wait_for_verification(self.processor3, "SMOKE-TEST-CLIENT-(EXCEPTION|CLOSED)", self.processor3.STDOUT_FILE) + else: + self.wait_for_verification(self.processor1, "SMOKE-TEST-CLIENT-CLOSED", self.processor1.STDOUT_FILE) + self.wait_for_verification(self.processor2, "SMOKE-TEST-CLIENT-CLOSED", self.processor2.STDOUT_FILE) + self.wait_for_verification(self.processor3, "SMOKE-TEST-CLIENT-CLOSED", self.processor3.STDOUT_FILE) self.roll_logs(self.processor1, ".1-1") self.roll_logs(self.processor2, ".1-1") self.roll_logs(self.processor3, ".1-1") - self.set_version(self.processor1, version) - self.set_version(self.processor2, version) - self.set_version(self.processor3, version) + self.set_version(self.processor1, to_version) + self.set_version(self.processor2, to_version) + self.set_version(self.processor3, to_version) self.processor1.start_node(self.processor1.node) self.processor2.start_node(self.processor2.node) self.processor3.start_node(self.processor3.node) # double-check the version - kafka_version_str = self.get_version_string(version) + kafka_version_str = self.get_version_string(to_version) self.wait_for_verification(self.processor1, kafka_version_str, self.processor1.LOG_FILE) self.wait_for_verification(self.processor2, kafka_version_str, self.processor2.LOG_FILE) self.wait_for_verification(self.processor3, kafka_version_str, self.processor3.LOG_FILE) @@ -226,8 +232,8 @@ def set_version(self, processor, version): def purge_state_dir(self, processor): processor.node.account.ssh("rm -rf " + processor.PERSISTENT_ROOT, allow_fail=False) - def do_stop_start_bounce(self, processor, upgrade_from, new_version, counter): - kafka_version_str = self.get_version_string(new_version) + def do_stop_start_bounce(self, processor, upgrade_from, from_version, to_version, counter): + kafka_version_str = self.get_version_string(to_version) first_other_processor = None second_other_processor = None @@ -252,7 +258,12 @@ def do_stop_start_bounce(self, processor, upgrade_from, new_version, counter): second_other_monitor.wait_until(self.processed_msg, timeout_sec=60, err_msg="Never saw output '%s' on " % self.processed_msg + str(second_other_node.account)) - node.account.ssh_capture("grep SMOKE-TEST-CLIENT-CLOSED %s" % processor.STDOUT_FILE, allow_fail=False) + + if from_version.startswith("2."): + # some older versions crash on shutdown, so we allow crashes here. + node.account.ssh_capture("grep -E 'SMOKE-TEST-CLIENT-(EXCEPTION|CLOSED)' %s" % processor.STDOUT_FILE, allow_fail=False) + else: + node.account.ssh_capture("grep -E 'SMOKE-TEST-CLIENT-CLOSED' %s" % processor.STDOUT_FILE, allow_fail=False) if upgrade_from is None: # upgrade disabled -- second round of rolling bounces roll_counter = ".1-" # second round of rolling bounces @@ -261,7 +272,7 @@ def do_stop_start_bounce(self, processor, upgrade_from, new_version, counter): self.roll_logs(processor, roll_counter + str(counter)) - self.set_version(processor, new_version) + self.set_version(processor, to_version) processor.set_upgrade_from(upgrade_from) grep_metadata_error = "grep \"org.apache.kafka.streams.errors.TaskAssignmentException: unable to decode subscription data: version=2\" " @@ -273,7 +284,7 @@ def do_stop_start_bounce(self, processor, upgrade_from, new_version, counter): log_monitor.wait_until(kafka_version_str, timeout_sec=60, - err_msg="Could not detect Kafka Streams version " + new_version + " on " + str(node.account)) + err_msg="Could not detect Kafka Streams version " + to_version + " on " + str(node.account)) first_other_monitor.wait_until(self.processed_msg, timeout_sec=60, err_msg="Never saw output '%s' on " % self.processed_msg + str(first_other_node.account)) diff --git a/tests/kafkatest/tests/streams/streams_broker_bounce_test.py b/tests/kafkatest/tests/streams/streams_broker_bounce_test.py index e9cd1315cf5e0..3d7e40fdf0dbd 100644 --- a/tests/kafkatest/tests/streams/streams_broker_bounce_test.py +++ b/tests/kafkatest/tests/streams/streams_broker_bounce_test.py @@ -112,8 +112,8 @@ def __init__(self, test_context): 'configs': {"min.insync.replicas": 2} }, 'tagg' : { 'partitions': self.partitions, 'replication-factor': self.replication, 'configs': {"min.insync.replicas": 2} }, - '__consumer_offsets' : { 'partitions': 50, 'replication-factor': self.replication, - 'configs': {"min.insync.replicas": 2} } + '__consumer_offsets' : { 'partitions': self.partitions, 'replication-factor': self.replication, + 'configs': {"min.insync.replicas": 2} } } def fail_broker_type(self, failure_mode, broker_type): @@ -152,7 +152,10 @@ def confirm_topics_on_all_brokers(self, expected_topic_set): def setup_system(self, start_processor=True, num_threads=3, group_protocol='classic'): # Setup phase use_streams_groups = True if group_protocol == 'streams' else False - self.kafka = KafkaService(self.test_context, num_nodes=self.replication, zk=None, topics=self.topics, use_streams_groups=use_streams_groups) + self.kafka = KafkaService(self.test_context, num_nodes=self.replication, zk=None, topics=self.topics, server_prop_overrides=[ + ["offsets.topic.num.partitions", self.partitions], + ["offsets.topic.replication.factor", self.replication] + ], use_streams_groups=use_streams_groups) self.kafka.start() # allow some time for topics to be created @@ -285,7 +288,7 @@ def test_all_brokers_bounce(self, failure_mode, num_failures, metadata_quorum, g # Set min.insync.replicas to 1 because in the last stage of the test there is only one broker left. # Otherwise the last offset commit will never succeed and time out and potentially take longer as # duration passed to the close method of the Kafka Streams client. - self.topics['__consumer_offsets'] = { 'partitions': 50, 'replication-factor': self.replication, + self.topics['__consumer_offsets'] = { 'partitions': self.partitions, 'replication-factor': self.replication, 'configs': {"min.insync.replicas": 1} } self.setup_system(group_protocol=group_protocol) diff --git a/tests/kafkatest/tests/streams/streams_broker_down_resilience_test.py b/tests/kafkatest/tests/streams/streams_broker_down_resilience_test.py index 3b9d0b43bf7fc..94df6e3747338 100644 --- a/tests/kafkatest/tests/streams/streams_broker_down_resilience_test.py +++ b/tests/kafkatest/tests/streams/streams_broker_down_resilience_test.py @@ -129,10 +129,12 @@ def test_streams_runs_with_broker_down_initially(self, metadata_quorum, group_pr with processor_2.node.account.monitor_log(processor_2.STDOUT_FILE) as monitor_2: with processor_3.node.account.monitor_log(processor_3.STDOUT_FILE) as monitor_3: + # repeating_keys enables production of records with keys, ensuring that we produce to all 3 partitions self.assert_produce(self.inputTopic, "sending_message_after_broker_down_initially", num_messages=self.num_messages, - timeout_sec=120) + timeout_sec=120, + repeating_keys=self.num_messages) monitor_1.wait_until(self.message, timeout_sec=120, @@ -189,10 +191,12 @@ def test_streams_should_scale_in_while_brokers_down(self, metadata_quorum, group with processor_2.node.account.monitor_log(processor_2.STDOUT_FILE) as monitor_2: with processor_3.node.account.monitor_log(processor_3.STDOUT_FILE) as monitor_3: + # repeating_keys enables production of records with keys, ensuring that we produce to all 3 partitions self.assert_produce(self.inputTopic, "sending_message_normal_broker_start", num_messages=self.num_messages, - timeout_sec=120) + timeout_sec=120, + repeating_keys=self.num_messages) monitor_1.wait_until(self.message, timeout_sec=120, @@ -273,10 +277,12 @@ def test_streams_should_failover_while_brokers_down(self, metadata_quorum, group with processor_2.node.account.monitor_log(processor_2.STDOUT_FILE) as monitor_2: with processor_3.node.account.monitor_log(processor_3.STDOUT_FILE) as monitor_3: + # repeating_keys enables production of records with keys, ensuring that we produce to all 3 partitions self.assert_produce(self.inputTopic, "sending_message_after_normal_broker_start", num_messages=self.num_messages, - timeout_sec=120) + timeout_sec=120, + repeating_keys=self.num_messages) monitor_1.wait_until(self.message, timeout_sec=120, @@ -320,10 +326,12 @@ def test_streams_should_failover_while_brokers_down(self, metadata_quorum, group with processor_2.node.account.monitor_log(processor_2.STDOUT_FILE) as monitor_2: with processor_3.node.account.monitor_log(processor_3.STDOUT_FILE) as monitor_3: + # repeating_keys enables production of records with keys, ensuring that we produce to all 3 partitions self.assert_produce(self.inputTopic, "sending_message_after_hard_bouncing_streams_instance_bouncing_broker", num_messages=self.num_messages, - timeout_sec=120) + timeout_sec=120, + repeating_keys=self.num_messages) monitor_1.wait_until(self.message, timeout_sec=120, diff --git a/tests/kafkatest/version.py b/tests/kafkatest/version.py index 0ee8f1aded1c5..15b30560a81d2 100644 --- a/tests/kafkatest/version.py +++ b/tests/kafkatest/version.py @@ -110,11 +110,11 @@ def get_version(node=None): return DEV_BRANCH DEV_BRANCH = KafkaVersion("dev") -DEV_VERSION = KafkaVersion("4.1.0-SNAPSHOT") +DEV_VERSION = KafkaVersion("4.1.1-SNAPSHOT") LATEST_STABLE_TRANSACTION_VERSION = 2 # This should match the LATEST_PRODUCTION version defined in MetadataVersion.java -LATEST_STABLE_METADATA_VERSION = "4.0-IV3" +LATEST_STABLE_METADATA_VERSION = "4.1-IV1" # 2.1.x versions V_2_1_0 = KafkaVersion("2.1.0") diff --git a/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java b/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java index 72c49410e13eb..4893bb2583010 100644 --- a/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java @@ -1271,7 +1271,7 @@ static Entry, Set> cancelAssignment(A Set targetPartsSet = targetParts.stream().map(t -> t.getKey()).collect(Collectors.toSet()); Set curReassigningParts = new HashSet<>(); adminClient.listPartitionReassignments(targetPartsSet).reassignments().get().forEach((part, reassignment) -> { - if (reassignment.addingReplicas().isEmpty() || !reassignment.removingReplicas().isEmpty()) + if (!reassignment.addingReplicas().isEmpty() || !reassignment.removingReplicas().isEmpty()) curReassigningParts.add(part); }); if (!curReassigningParts.isEmpty()) { @@ -1440,7 +1440,7 @@ static ReassignPartitionsCommandOptions validateAndParseArgs(String[] args) { } OptionSpec action = allActions.get(0); - + if (opts.options.has(opts.bootstrapServerOpt) && opts.options.has(opts.bootstrapControllerOpt)) CommandLineUtils.printUsageAndExit(opts.parser, "Please don't specify both --bootstrap-server and --bootstrap-controller"); else if (!opts.options.has(opts.bootstrapServerOpt) && !opts.options.has(opts.bootstrapControllerOpt)) diff --git a/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java index c416cfb67d541..23794fde622db 100644 --- a/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java @@ -121,8 +121,16 @@ public void testDowngradeMetadataVersionWithKRaft(ClusterInstance cluster) { "disable", "--feature", "metadata.version")) ); // Change expected message to reflect possible MetadataVersion range 1-N (N increases when adding a new version) - assertEquals("Could not disable metadata.version. The update failed for all features since the following " + - "feature had an error: Invalid update version 0 for feature metadata.version. Local controller 3000 only supports versions 7-28", commandOutput); + assertEquals( + String.format( + "Could not disable metadata.version. The update failed for all features since the " + + "following feature had an error: Invalid update version 0 for feature " + + "metadata.version. Local controller 3000 only supports versions %s-%s", + MetadataVersion.MINIMUM_VERSION.featureLevel(), + MetadataVersion.latestTesting().featureLevel() + ), + commandOutput + ); commandOutput = ToolsTestUtils.captureStandardOut(() -> assertEquals(1, FeatureCommand.mainNoExit("--bootstrap-server", cluster.bootstrapServers(), diff --git a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandTest.java index a96bccd36eddd..3c5497b380664 100644 --- a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandTest.java @@ -316,7 +316,7 @@ public void testCancellationWithBootstrapController() throws Exception { } @ClusterTest - public void testCancellationWithAddingReplicaInIsr() throws Exception { + public void testCancellationWithAddingAndRemovingReplicaInIsr() throws Exception { createTopics(); TopicPartition foo0 = new TopicPartition("foo", 0); produceMessages(foo0.topic(), foo0.partition(), 200); @@ -351,6 +351,42 @@ public void testCancellationWithAddingReplicaInIsr() throws Exception { verifyReplicaDeleted(new TopicPartitionReplica(foo0.topic(), foo0.partition(), 4)); } + @ClusterTest + public void testCancellationWithAddingReplicaInIsr() throws Exception { + createTopics(); + TopicPartition foo0 = new TopicPartition("foo", 0); + produceMessages(foo0.topic(), foo0.partition(), 200); + + // The reassignment will bring replicas 3 and 4 into the replica set. + String assignment = "{\"version\":1,\"partitions\":" + + "[{\"topic\":\"foo\",\"partition\":0,\"replicas\":[0,1,2,3,4],\"log_dirs\":[\"any\",\"any\",\"any\",\"any\",\"any\"]}" + + "]}"; + + // We will throttle replica 4 so that only replica 3 joins the ISR + setReplicationThrottleForPartitions(foo0); + + // Execute the assignment and wait for replica 3 (only) to join the ISR + runExecuteAssignment(false, assignment, -1L, -1L); + try (Admin admin = Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { + TestUtils.waitForCondition( + () -> { + Set isr = admin.describeTopics(Collections.singleton(foo0.topic())) + .allTopicNames().get().get(foo0.topic()).partitions().stream() + .filter(p -> p.partition() == foo0.partition()) + .flatMap(p -> p.isr().stream()) + .map(Node::id).collect(Collectors.toSet()); + return isr.containsAll(Arrays.asList(0, 1, 2, 3)); + }, + "Timed out while waiting for replica 3 to join the ISR" + ); + } + + // Now cancel the assignment and verify that the partition is removed from cancelled replicas + assertEquals(new AbstractMap.SimpleImmutableEntry<>(singleton(foo0), Collections.emptySet()), runCancelAssignment(assignment, true, true)); + verifyReplicaDeleted(new TopicPartitionReplica(foo0.topic(), foo0.partition(), 3)); + verifyReplicaDeleted(new TopicPartitionReplica(foo0.topic(), foo0.partition(), 4)); + } + /** * Test moving partitions between directories. */