From 03db088518c553f52f61937792c9cb66453fd869 Mon Sep 17 00:00:00 2001 From: Antoine Pourchet Date: Thu, 23 May 2024 13:49:08 -0600 Subject: [PATCH] Squashed commit of the following (during trunk merge): MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit commit 93238ae312ef7bea79160e59bb7a06623cc94a1b Author: Antoine Pourchet Date: Thu May 23 13:45:29 2024 -0600 KAFKA-15045: (KIP-924 pt. 10) Topic partition rack annotation simplified (#16034) This PR uses the new TaskTopicPartition structure to simplify the build process for the ApplicationState, which is the input to the new TaskAssignor#assign call. Reviewers: Anna Sophie Blee-Goldman commit 4020307ac2842a35cac8c974ec3ab90153893194 Author: Kuan-Po (Cooper) Tseng Date: Fri May 24 02:51:26 2024 +0800 KAFKA-16795 Fix broken compatibility in kafka.tools.NoOpMessageFormatter, kafka.tools.DefaultMessageFormatter, and kafka.tools.LoggingMessageFormatter (#16020) This commit allows users to apply the scala version Formatters, but users will receive the warning messages about deprecation. This compatibility support will be removed from 4.0.0 Reviewers: Chia-Ping Tsai commit c3018ef55ec97f6b2bb08baaf141ff840559e10e Author: TingIāu "Ting" Kì <51072200+frankvicky@users.noreply.github.com> Date: Fri May 24 01:15:56 2024 +0800 KAFKA-16804: Replace archivesBaseName with archivesName (#16016) Reviewers: Chia-Ping Tsai , Greg Harris commit 0ba15ad4d1bd7318d626430035a1666eb16f844d Author: Edoardo Comar Date: Thu May 23 17:17:56 2024 +0100 KAFKA-15905 Restarts of MirrorCheckpointTask should not permanently i… (#15910) * KAFKA-15905 Restarts of MirrorCheckpointTask should not permanently interrupt offset translation MirrorCheckpointTask reloads the last checkpoint at start, OffsetSyncStore stores OffsetSyncs before reading till end. If CheckpointTask cannot read checkpoints at startup, use previous OffsetSyncStore load logic, with warning log message about degraded offset translation. Also addresses KAFKA-16622 Mirromaker2 first Checkpoint not emitted until consumer group fully catches up once because the OffsetSyncStore store is populated before reading to log end. Co-Authored-By: Adrian Preston Reviewers: Greg Harris commit 5a4898450d0125f819f8b5ca5cce53e26ea053af Author: Viktor Somogyi-Vass Date: Thu May 23 17:36:39 2024 +0200 KAFKA-15649: Handle directory failure timeout (#15697) A broker that is unable to communicate with the controller will shut down after the configurable log.dir.failure.timeout.ms. The implementation adds a new event to the Kafka EventQueue. This event is deferred by the configured timeout and will execute the shutdown if the heartbeat communication containing the failed log dir is still pending with the controller. Reviewers: Igor Soarez commit 8d117a188df7a135ad5595d0302ad903f9241027 Author: Mickael Maison Date: Thu May 23 17:03:24 2024 +0200 KAFKA-16825: Update netty/jetty/jackson/zstd dependencies (#16038) Reviewers: Luke Chen commit ab0cc72499eedd5b5da7232d4a799103d17802fd Author: Mickael Maison Date: Thu May 23 16:01:45 2024 +0200 MINOR: Move parseCsvList to server-common (#16029) Reviewers: Chia-Ping Tsai commit 14b5c4d1e8589ff61faf855e6b64766001e06ecf Author: Dongnuo Lyu <139248811+dongnuo123@users.noreply.github.com> Date: Thu May 23 02:27:00 2024 -0400 KAFKA-16793; Heartbeat API for upgrading ConsumerGroup (#15988) This patch implements the heartbeat api to the members that use the classic protocol in a ConsumerGroup. Reviewers: Jeff Kim , David Jacot commit e692feed34c59c71fe0277d08141e023c0d28d73 Author: Jeff Kim Date: Thu May 23 02:24:23 2024 -0400 MINOR: fix flaky testRecordThreadIdleRatio (#15987) DelayEventAccumulator should return immediately if there are no events in the queue. Also removed some unused fields inside EventProcessorThread. Reviewers: Gaurav Narula , Chia-Ping Tsai , David Jacot commit bef83ce89bb6a869cf884b37d5b5f86d8f243488 Author: Nick Telford Date: Thu May 23 05:34:31 2024 +0100 KAFKA-15541: Add iterator-duration metrics (#16028) Part of [KIP-989](https://cwiki.apache.org/confluence/x/9KCzDw). This new `StateStore` metric tracks the average and maximum amount of time between creating and closing Iterators. Iterators with very high durations can indicate to users performance problems that should be addressed. If a store reports no data for these metrics, despite the user opening Iterators on the store, it suggests those iterators are not being closed, and have therefore leaked. Reviewers: Matthias J. Sax --- build.gradle | 215 ++++++--- .../kafka/connect/mirror/Checkpoint.java | 14 + .../kafka/connect/mirror/CheckpointStore.java | 203 ++++++++ .../mirror/MirrorCheckpointConfig.java | 1 + .../connect/mirror/MirrorCheckpointTask.java | 51 +- .../kafka/connect/mirror/OffsetSyncStore.java | 22 +- .../connect/mirror/CheckpointStoreTest.java | 97 ++++ .../mirror/MirrorCheckpointTaskTest.java | 125 ++++- .../connect/mirror/OffsetSyncStoreTest.java | 108 ++++- .../kafka/connect/util/KafkaBasedLog.java | 12 +- .../kafka/metrics/KafkaMetricsConfig.scala | 8 +- .../kafka/server/BrokerLifecycleManager.scala | 42 +- .../scala/kafka/server/BrokerServer.scala | 7 +- .../main/scala/kafka/server/KafkaConfig.scala | 8 +- .../scala/kafka/server/ReplicaManager.scala | 6 +- .../main/scala/kafka/utils/CoreUtils.scala | 16 +- .../server/BrokerLifecycleManagerTest.scala | 6 +- .../unit/kafka/server/LogDirFailureTest.scala | 30 +- .../unit/kafka/utils/CoreUtilsTest.scala | 13 - gradle/dependencies.gradle | 8 +- .../group/GroupCoordinatorService.java | 7 +- .../group/GroupCoordinatorShard.java | 5 +- .../group/GroupMetadataManager.java | 272 ++++++++--- .../group/consumer/ConsumerGroup.java | 23 + .../group/runtime/CoordinatorRuntime.java | 13 + .../group/runtime/CoordinatorTimer.java | 12 + .../runtime/MultiThreadedEventProcessor.java | 3 - .../group/GroupCoordinatorServiceTest.java | 9 +- .../group/GroupMetadataManagerTest.java | 444 +++++++++++++++++- .../GroupMetadataManagerTestContext.java | 26 +- .../group/MockCoordinatorTimer.java | 13 + .../group/consumer/ConsumerGroupTest.java | 36 ++ .../group/runtime/CoordinatorRuntimeTest.java | 51 ++ .../MultiThreadedEventProcessorTest.java | 9 +- .../kafka/server/config/ServerLogConfigs.java | 5 + .../org/apache/kafka/server/util/Csv.java | 16 + .../org/apache/kafka/server/util/CsvTest.java | 22 +- .../processor/assignment/TaskInfo.java | 26 +- .../internals/StreamsPartitionAssignor.java | 65 ++- .../internals/assignment/DefaultTaskInfo.java | 30 +- .../assignment/DefaultTaskTopicPartition.java | 32 +- .../internals/assignment/RackUtils.java | 51 +- .../state/internals/MeteredKeyValueStore.java | 10 +- ...MeteredMultiVersionedKeyQueryIterator.java | 12 + .../state/internals/MeteredSessionStore.java | 12 + .../MeteredTimestampedKeyValueStore.java | 4 +- .../MeteredVersionedKeyValueStore.java | 8 +- .../state/internals/MeteredWindowStore.java | 12 + .../internals/MeteredWindowStoreIterator.java | 13 +- .../MeteredWindowedKeyValueIterator.java | 13 +- .../internals/metrics/StateStoreMetrics.java | 25 + .../internals/MeteredKeyValueStoreTest.java | 36 +- .../internals/MeteredSessionStoreTest.java | 34 +- .../MeteredTimestampedKeyValueStoreTest.java | 35 +- .../MeteredVersionedKeyValueStoreTest.java | 36 ++ .../internals/MeteredWindowStoreTest.java | 34 +- .../consumer/ConsoleConsumerOptions.java | 21 +- .../consumer/ConsoleConsumerOptionsTest.java | 27 ++ 58 files changed, 2090 insertions(+), 404 deletions(-) create mode 100644 connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointStore.java create mode 100644 connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointStoreTest.java diff --git a/build.gradle b/build.gradle index 031389f62c6e..0836f7af962c 100644 --- a/build.gradle +++ b/build.gradle @@ -356,7 +356,7 @@ subprojects { artifact task } - artifactId = archivesBaseName + artifactId = base.archivesName.get() pom { name = 'Apache Kafka' url = 'https://kafka.apache.org' @@ -850,7 +850,9 @@ tasks.create(name: "jarConnect", dependsOn: connectPkgs.collect { it + ":jar" }) tasks.create(name: "testConnect", dependsOn: connectPkgs.collect { it + ":test" }) {} project(':server') { - archivesBaseName = "kafka-server" + base { + archivesName = "kafka-server" + } dependencies { implementation project(':clients') @@ -922,7 +924,10 @@ project(':core') { } if (userEnableTestCoverage) apply plugin: "org.scoverage" - archivesBaseName = "kafka_${versions.baseScala}" + + base { + archivesName = "kafka_${versions.baseScala}" + } configurations { generator @@ -1250,7 +1255,9 @@ project(':core') { } project(':metadata') { - archivesBaseName = "kafka-metadata" + base { + archivesName = "kafka-metadata" + } configurations { generator @@ -1317,7 +1324,9 @@ project(':metadata') { } project(':group-coordinator') { - archivesBaseName = "kafka-group-coordinator" + base { + archivesName = "kafka-group-coordinator" + } configurations { generator @@ -1384,7 +1393,9 @@ project(':group-coordinator') { } project(':transaction-coordinator') { - archivesBaseName = "kafka-transaction-coordinator" + base { + archivesName = "kafka-transaction-coordinator" + } sourceSets { main { @@ -1405,7 +1416,9 @@ project(':transaction-coordinator') { } project(':examples') { - archivesBaseName = "kafka-examples" + base { + archivesName = "kafka-examples" + } dependencies { implementation project(':clients') @@ -1435,7 +1448,9 @@ project(':generator') { } project(':clients') { - archivesBaseName = "kafka-clients" + base { + archivesName = "kafka-clients" + } configurations { generator @@ -1610,7 +1625,9 @@ project(':clients') { } project(':raft') { - archivesBaseName = "kafka-raft" + base { + archivesName = "kafka-raft" + } configurations { generator @@ -1706,7 +1723,9 @@ project(':raft') { } project(':server-common') { - archivesBaseName = "kafka-server-common" + base { + archivesName = "kafka-server-common" + } dependencies { api project(':clients') @@ -1764,7 +1783,9 @@ project(':server-common') { } project(':storage:storage-api') { - archivesBaseName = "kafka-storage-api" + base { + archivesName = "kafka-storage-api" + } dependencies { implementation project(':clients') @@ -1832,7 +1853,9 @@ project(':storage:storage-api') { } project(':storage') { - archivesBaseName = "kafka-storage" + base { + archivesName = "kafka-storage" + } configurations { generator @@ -1954,7 +1977,9 @@ project(':storage') { } project(':tools:tools-api') { - archivesBaseName = "kafka-tools-api" + base { + archivesName = "kafka-tools-api" + } dependencies { implementation project(':clients') @@ -2009,7 +2034,10 @@ project(':tools:tools-api') { } project(':tools') { - archivesBaseName = "kafka-tools" + base { + archivesName = "kafka-tools" + } + dependencies { implementation project(':clients') implementation project(':storage') @@ -2073,7 +2101,9 @@ project(':tools') { } project(':trogdor') { - archivesBaseName = "trogdor" + base { + archivesName = "trogdor" + } dependencies { implementation project(':clients') @@ -2123,7 +2153,9 @@ project(':trogdor') { } project(':shell') { - archivesBaseName = "kafka-shell" + base { + archivesName = "kafka-shell" + } dependencies { implementation libs.argparse4j @@ -2173,7 +2205,10 @@ project(':shell') { } project(':streams') { - archivesBaseName = "kafka-streams" + base { + archivesName = "kafka-streams" + } + ext.buildStreamsVersionFileName = "kafka-streams-version.properties" configurations { @@ -2335,7 +2370,11 @@ project(':streams') { project(':streams:streams-scala') { apply plugin: 'scala' - archivesBaseName = "kafka-streams-scala_${versions.baseScala}" + + base { + archivesName = "kafka-streams-scala_${versions.baseScala}" + } + dependencies { api project(':streams') @@ -2397,7 +2436,9 @@ project(':streams:streams-scala') { } project(':streams:test-utils') { - archivesBaseName = "kafka-streams-test-utils" + base { + archivesName = "kafka-streams-test-utils" + } dependencies { api project(':streams') @@ -2432,7 +2473,9 @@ project(':streams:test-utils') { } project(':streams:examples') { - archivesBaseName = "kafka-streams-examples" + base { + archivesName = "kafka-streams-examples" + } dependencies { // this dependency should be removed after we unify data API @@ -2469,7 +2512,9 @@ project(':streams:examples') { } project(':streams:upgrade-system-tests-0100') { - archivesBaseName = "kafka-streams-upgrade-system-tests-0100" + base { + archivesName = "kafka-streams-upgrade-system-tests-0100" + } dependencies { testImplementation(libs.kafkaStreams_0100) { @@ -2485,7 +2530,9 @@ project(':streams:upgrade-system-tests-0100') { } project(':streams:upgrade-system-tests-0101') { - archivesBaseName = "kafka-streams-upgrade-system-tests-0101" + base { + archivesName = "kafka-streams-upgrade-system-tests-0101" + } dependencies { testImplementation(libs.kafkaStreams_0101) { @@ -2501,7 +2548,9 @@ project(':streams:upgrade-system-tests-0101') { } project(':streams:upgrade-system-tests-0102') { - archivesBaseName = "kafka-streams-upgrade-system-tests-0102" + base { + archivesName = "kafka-streams-upgrade-system-tests-0102" + } dependencies { testImplementation libs.kafkaStreams_0102 @@ -2514,7 +2563,9 @@ project(':streams:upgrade-system-tests-0102') { } project(':streams:upgrade-system-tests-0110') { - archivesBaseName = "kafka-streams-upgrade-system-tests-0110" + base{ + archivesName = "kafka-streams-upgrade-system-tests-0110" + } dependencies { testImplementation libs.kafkaStreams_0110 @@ -2527,7 +2578,9 @@ project(':streams:upgrade-system-tests-0110') { } project(':streams:upgrade-system-tests-10') { - archivesBaseName = "kafka-streams-upgrade-system-tests-10" + base { + archivesName = "kafka-streams-upgrade-system-tests-10" + } dependencies { testImplementation libs.kafkaStreams_10 @@ -2540,7 +2593,9 @@ project(':streams:upgrade-system-tests-10') { } project(':streams:upgrade-system-tests-11') { - archivesBaseName = "kafka-streams-upgrade-system-tests-11" + base { + archivesName = "kafka-streams-upgrade-system-tests-11" + } dependencies { testImplementation libs.kafkaStreams_11 @@ -2553,7 +2608,9 @@ project(':streams:upgrade-system-tests-11') { } project(':streams:upgrade-system-tests-20') { - archivesBaseName = "kafka-streams-upgrade-system-tests-20" + base { + archivesName = "kafka-streams-upgrade-system-tests-20" + } dependencies { testImplementation libs.kafkaStreams_20 @@ -2566,7 +2623,9 @@ project(':streams:upgrade-system-tests-20') { } project(':streams:upgrade-system-tests-21') { - archivesBaseName = "kafka-streams-upgrade-system-tests-21" + base { + archivesName = "kafka-streams-upgrade-system-tests-21" + } dependencies { testImplementation libs.kafkaStreams_21 @@ -2579,7 +2638,9 @@ project(':streams:upgrade-system-tests-21') { } project(':streams:upgrade-system-tests-22') { - archivesBaseName = "kafka-streams-upgrade-system-tests-22" + base { + archivesName = "kafka-streams-upgrade-system-tests-22" + } dependencies { testImplementation libs.kafkaStreams_22 @@ -2592,7 +2653,9 @@ project(':streams:upgrade-system-tests-22') { } project(':streams:upgrade-system-tests-23') { - archivesBaseName = "kafka-streams-upgrade-system-tests-23" + base { + archivesName = "kafka-streams-upgrade-system-tests-23" + } dependencies { testImplementation libs.kafkaStreams_23 @@ -2605,7 +2668,9 @@ project(':streams:upgrade-system-tests-23') { } project(':streams:upgrade-system-tests-24') { - archivesBaseName = "kafka-streams-upgrade-system-tests-24" + base { + archivesName = "kafka-streams-upgrade-system-tests-24" + } dependencies { testImplementation libs.kafkaStreams_24 @@ -2618,7 +2683,9 @@ project(':streams:upgrade-system-tests-24') { } project(':streams:upgrade-system-tests-25') { - archivesBaseName = "kafka-streams-upgrade-system-tests-25" + base { + archivesName = "kafka-streams-upgrade-system-tests-25" + } dependencies { testImplementation libs.kafkaStreams_25 @@ -2631,7 +2698,9 @@ project(':streams:upgrade-system-tests-25') { } project(':streams:upgrade-system-tests-26') { - archivesBaseName = "kafka-streams-upgrade-system-tests-26" + base { + archivesName = "kafka-streams-upgrade-system-tests-26" + } dependencies { testImplementation libs.kafkaStreams_26 @@ -2644,7 +2713,9 @@ project(':streams:upgrade-system-tests-26') { } project(':streams:upgrade-system-tests-27') { - archivesBaseName = "kafka-streams-upgrade-system-tests-27" + base { + archivesName = "kafka-streams-upgrade-system-tests-27" + } dependencies { testImplementation libs.kafkaStreams_27 @@ -2657,7 +2728,9 @@ project(':streams:upgrade-system-tests-27') { } project(':streams:upgrade-system-tests-28') { - archivesBaseName = "kafka-streams-upgrade-system-tests-28" + base { + archivesName = "kafka-streams-upgrade-system-tests-28" + } dependencies { testImplementation libs.kafkaStreams_28 @@ -2670,7 +2743,9 @@ project(':streams:upgrade-system-tests-28') { } project(':streams:upgrade-system-tests-30') { - archivesBaseName = "kafka-streams-upgrade-system-tests-30" + base { + archivesName = "kafka-streams-upgrade-system-tests-30" + } dependencies { testImplementation libs.kafkaStreams_30 @@ -2683,7 +2758,9 @@ project(':streams:upgrade-system-tests-30') { } project(':streams:upgrade-system-tests-31') { - archivesBaseName = "kafka-streams-upgrade-system-tests-31" + base { + archivesName = "kafka-streams-upgrade-system-tests-31" + } dependencies { testImplementation libs.kafkaStreams_31 @@ -2696,7 +2773,9 @@ project(':streams:upgrade-system-tests-31') { } project(':streams:upgrade-system-tests-32') { - archivesBaseName = "kafka-streams-upgrade-system-tests-32" + base { + archivesName = "kafka-streams-upgrade-system-tests-32" + } dependencies { testImplementation libs.kafkaStreams_32 @@ -2709,7 +2788,9 @@ project(':streams:upgrade-system-tests-32') { } project(':streams:upgrade-system-tests-33') { - archivesBaseName = "kafka-streams-upgrade-system-tests-33" + base { + archivesName = "kafka-streams-upgrade-system-tests-33" + } dependencies { testImplementation libs.kafkaStreams_33 @@ -2722,7 +2803,9 @@ project(':streams:upgrade-system-tests-33') { } project(':streams:upgrade-system-tests-34') { - archivesBaseName = "kafka-streams-upgrade-system-tests-34" + base { + archivesName = "kafka-streams-upgrade-system-tests-34" + } dependencies { testImplementation libs.kafkaStreams_34 @@ -2735,7 +2818,9 @@ project(':streams:upgrade-system-tests-34') { } project(':streams:upgrade-system-tests-35') { - archivesBaseName = "kafka-streams-upgrade-system-tests-35" + base { + archivesName = "kafka-streams-upgrade-system-tests-35" + } dependencies { testImplementation libs.kafkaStreams_35 @@ -2748,7 +2833,9 @@ project(':streams:upgrade-system-tests-35') { } project(':streams:upgrade-system-tests-36') { - archivesBaseName = "kafka-streams-upgrade-system-tests-36" + base { + archivesName = "kafka-streams-upgrade-system-tests-36" + } dependencies { testImplementation libs.kafkaStreams_36 @@ -2761,7 +2848,9 @@ project(':streams:upgrade-system-tests-36') { } project(':streams:upgrade-system-tests-37') { - archivesBaseName = "kafka-streams-upgrade-system-tests-37" + base { + archivesName = "kafka-streams-upgrade-system-tests-37" + } dependencies { testImplementation libs.kafkaStreams_37 @@ -2846,7 +2935,9 @@ project(':jmh-benchmarks') { } project(':log4j-appender') { - archivesBaseName = "kafka-log4j-appender" + base { + archivesName = "kafka-log4j-appender" + } dependencies { implementation project(':clients') @@ -2865,7 +2956,9 @@ project(':log4j-appender') { } project(':connect:api') { - archivesBaseName = "connect-api" + base { + archivesName = "connect-api" + } dependencies { api project(':clients') @@ -2900,7 +2993,9 @@ project(':connect:api') { } project(':connect:transforms') { - archivesBaseName = "connect-transforms" + base { + archivesName = "connect-transforms" + } dependencies { api project(':connect:api') @@ -2936,7 +3031,9 @@ project(':connect:transforms') { } project(':connect:json') { - archivesBaseName = "connect-json" + base { + archivesName = "connect-json" + } dependencies { api project(':connect:api') @@ -2980,7 +3077,9 @@ project(':connect:runtime') { swagger } - archivesBaseName = "connect-runtime" + base { + archivesName = "connect-runtime" + } dependencies { // connect-runtime is used in tests, use `api` for modules below for backwards compatibility even though @@ -3122,7 +3221,9 @@ project(':connect:runtime') { } project(':connect:file') { - archivesBaseName = "connect-file" + base { + archivesName = "connect-file" + } dependencies { implementation project(':connect:api') @@ -3162,7 +3263,9 @@ project(':connect:file') { } project(':connect:basic-auth-extension') { - archivesBaseName = "connect-basic-auth-extension" + base { + archivesName = "connect-basic-auth-extension" + } dependencies { implementation project(':connect:api') @@ -3202,7 +3305,9 @@ project(':connect:basic-auth-extension') { } project(':connect:mirror') { - archivesBaseName = "connect-mirror" + base { + archivesName = "connect-mirror" + } dependencies { implementation project(':connect:api') @@ -3290,7 +3395,9 @@ project(':connect:mirror') { } project(':connect:mirror-client') { - archivesBaseName = "connect-mirror-client" + base { + archivesName = "connect-mirror-client" + } dependencies { implementation project(':clients') @@ -3325,7 +3432,9 @@ project(':connect:mirror-client') { } project(':connect:test-plugins') { - archivesBaseName = "connect-test-plugins" + base { + archivesName = "connect-test-plugins" + } dependencies { api project(':connect:api') diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Checkpoint.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Checkpoint.java index 8f186400dd29..353d2eedb959 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Checkpoint.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Checkpoint.java @@ -27,6 +27,7 @@ import java.util.Map; import java.util.HashMap; import java.nio.ByteBuffer; +import java.util.Objects; /** Checkpoint records emitted from MirrorCheckpointConnector. Encodes remote consumer group state. */ public class Checkpoint { @@ -180,5 +181,18 @@ byte[] recordKey() { byte[] recordValue() { return serializeValue(VERSION).array(); } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Checkpoint that = (Checkpoint) o; + return upstreamOffset == that.upstreamOffset && downstreamOffset == that.downstreamOffset && Objects.equals(consumerGroupId, that.consumerGroupId) && Objects.equals(topicPartition, that.topicPartition) && Objects.equals(metadata, that.metadata); + } + + @Override + public int hashCode() { + return Objects.hash(consumerGroupId, topicPartition, upstreamOffset, downstreamOffset, metadata); + } } diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointStore.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointStore.java new file mode 100644 index 000000000000..cbe76efecb38 --- /dev/null +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointStore.java @@ -0,0 +1,203 @@ +/* + * 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.connect.mirror; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.AuthorizationException; +import org.apache.kafka.common.protocol.types.SchemaException; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.connect.util.Callback; +import org.apache.kafka.connect.util.KafkaBasedLog; +import org.apache.kafka.connect.util.TopicAdmin; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import static org.apache.kafka.connect.mirror.MirrorCheckpointConfig.CHECKPOINTS_TARGET_CONSUMER_ROLE; + +/** + * Reads once the Kafka log for checkpoints and populates a map of + * checkpoints per consumer group. + * + * The Kafka log is closed after the initial load and only the in memory map is + * used after start. + */ +public class CheckpointStore implements AutoCloseable { + + private static final Logger log = LoggerFactory.getLogger(CheckpointStore.class); + + private final MirrorCheckpointTaskConfig config; + private final Set consumerGroups; + + private TopicAdmin cpAdmin = null; + private KafkaBasedLog backingStore = null; + // accessible for testing + Map> checkpointsPerConsumerGroup; + + private volatile boolean loadSuccess = false; + private volatile boolean isInitialized = false; + + public CheckpointStore(MirrorCheckpointTaskConfig config, Set consumerGroups) { + this.config = config; + this.consumerGroups = new HashSet<>(consumerGroups); + } + + // constructor for testing only + CheckpointStore(Map> checkpointsPerConsumerGroup) { + this.config = null; //ignored by tests + this.consumerGroups = null; //ignored by tests + this.checkpointsPerConsumerGroup = checkpointsPerConsumerGroup; + isInitialized = true; + loadSuccess = true; + } + + // potentially long running + public boolean start() { + checkpointsPerConsumerGroup = readCheckpoints(); + isInitialized = true; + if (log.isTraceEnabled()) { + log.trace("CheckpointStore started, load success={}, map={}", loadSuccess, checkpointsPerConsumerGroup); + } else { + log.debug("CheckpointStore started, load success={}, map.size={}", loadSuccess, checkpointsPerConsumerGroup.size()); + } + return loadSuccess; + } + + public boolean isInitialized() { + return isInitialized; + } + + public void update(String group, Map newCheckpoints) { + Map oldCheckpoints = checkpointsPerConsumerGroup.computeIfAbsent(group, ignored -> new HashMap<>()); + oldCheckpoints.putAll(newCheckpoints); + } + + public Map get(String group) { + Map result = checkpointsPerConsumerGroup.get(group); + return result == null ? null : Collections.unmodifiableMap(result); + } + + public Map> computeConvertedUpstreamOffset() { + Map> result = new HashMap<>(); + + for (Map.Entry> entry : checkpointsPerConsumerGroup.entrySet()) { + String consumerId = entry.getKey(); + Map convertedUpstreamOffset = new HashMap<>(); + for (Checkpoint checkpoint : entry.getValue().values()) { + convertedUpstreamOffset.put(checkpoint.topicPartition(), checkpoint.offsetAndMetadata()); + } + result.put(consumerId, convertedUpstreamOffset); + } + return result; + } + + @Override + public void close() { + releaseResources(); + } + + private void releaseResources() { + Utils.closeQuietly(backingStore != null ? backingStore::stop : null, "backing store for previous Checkpoints"); + Utils.closeQuietly(cpAdmin, "admin client for previous Checkpoints"); + cpAdmin = null; + backingStore = null; + } + + // read the checkpoints topic to initialize the checkpointsPerConsumerGroup state + // the callback may only handle errors thrown by consumer.poll in KafkaBasedLog + // e.g. unauthorized to read from topic (non-retriable) + // if any are encountered, treat the loading of Checkpoints as failed. + private Map> readCheckpoints() { + Map> checkpoints = new HashMap<>(); + Callback> consumedCallback = (error, cpRecord) -> { + if (error != null) { + // if there is no authorization to READ from the topic, we must throw an error + // to stop the KafkaBasedLog forever looping attempting to read to end + checkpoints.clear(); + if (error instanceof RuntimeException) { + throw (RuntimeException) error; + } else { + throw new RuntimeException(error); + } + } else { + try { + Checkpoint cp = Checkpoint.deserializeRecord(cpRecord); + if (consumerGroups.contains(cp.consumerGroupId())) { + Map cps = checkpoints.computeIfAbsent(cp.consumerGroupId(), ignored1 -> new HashMap<>()); + cps.put(cp.topicPartition(), cp); + } + } catch (SchemaException ex) { + log.warn("Ignored invalid checkpoint record at offset {}", cpRecord.offset(), ex); + } + } + }; + + try { + long startTime = System.currentTimeMillis(); + readCheckpointsImpl(config, consumedCallback); + log.debug("starting+stopping KafkaBasedLog took {}ms", System.currentTimeMillis() - startTime); + loadSuccess = true; + } catch (Exception error) { + loadSuccess = false; + if (error instanceof AuthorizationException) { + log.warn("Not authorized to access checkpoints topic {} - " + + "this may degrade offset translation as only checkpoints " + + "for offsets which were mirrored after the task started will be emitted", + config.checkpointsTopic(), error); + } else { + log.info("Exception encountered loading checkpoints topic {} - " + + "this may degrade offset translation as only checkpoints " + + "for offsets which were mirrored after the task started will be emitted", + config.checkpointsTopic(), error); + } + } + return checkpoints; + } + + // accessible for testing + void readCheckpointsImpl(MirrorCheckpointTaskConfig config, Callback> consumedCallback) { + try { + cpAdmin = new TopicAdmin( + config.targetAdminConfig("checkpoint-target-admin"), + config.forwardingAdmin(config.targetAdminConfig("checkpoint-target-admin"))); + + backingStore = KafkaBasedLog.withExistingClients( + config.checkpointsTopic(), + MirrorUtils.newConsumer(config.targetConsumerConfig(CHECKPOINTS_TARGET_CONSUMER_ROLE)), + null, + cpAdmin, + consumedCallback, + Time.SYSTEM, + ignored -> { + }, + topicPartition -> topicPartition.partition() == 0); + + backingStore.start(true); + backingStore.stop(); + } finally { + releaseResources(); + } + } +} diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConfig.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConfig.java index e37cee4a79b8..8be52a9c9be9 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConfig.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConfig.java @@ -75,6 +75,7 @@ public class MirrorCheckpointConfig extends MirrorConnectorConfig { public static final Class GROUP_FILTER_CLASS_DEFAULT = DefaultGroupFilter.class; public static final String OFFSET_SYNCS_SOURCE_CONSUMER_ROLE = "offset-syncs-source-consumer"; public static final String OFFSET_SYNCS_TARGET_CONSUMER_ROLE = "offset-syncs-target-consumer"; + public static final String CHECKPOINTS_TARGET_CONSUMER_ROLE = "checkpoints-target-consumer"; public static final String OFFSET_SYNCS_SOURCE_ADMIN_ROLE = "offset-syncs-source-admin"; public static final String OFFSET_SYNCS_TARGET_ADMIN_ROLE = "offset-syncs-target-admin"; diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java index 96c287add5f1..7f446efea5df 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java @@ -69,21 +69,25 @@ public class MirrorCheckpointTask extends SourceTask { private MirrorCheckpointMetrics metrics; private Scheduler scheduler; private Map> idleConsumerGroupsOffset; - private Map> checkpointsPerConsumerGroup; + private CheckpointStore checkpointStore; + public MirrorCheckpointTask() {} // for testing MirrorCheckpointTask(String sourceClusterAlias, String targetClusterAlias, - ReplicationPolicy replicationPolicy, OffsetSyncStore offsetSyncStore, + ReplicationPolicy replicationPolicy, OffsetSyncStore offsetSyncStore, Set consumerGroups, Map> idleConsumerGroupsOffset, - Map> checkpointsPerConsumerGroup) { + CheckpointStore checkpointStore) { this.sourceClusterAlias = sourceClusterAlias; this.targetClusterAlias = targetClusterAlias; this.replicationPolicy = replicationPolicy; this.offsetSyncStore = offsetSyncStore; + this.consumerGroups = consumerGroups; this.idleConsumerGroupsOffset = idleConsumerGroupsOffset; - this.checkpointsPerConsumerGroup = checkpointsPerConsumerGroup; + this.checkpointStore = checkpointStore; this.topicFilter = topic -> true; + this.interval = Duration.ofNanos(1); + this.pollTimeout = Duration.ofNanos(1); } @Override @@ -103,15 +107,18 @@ public void start(Map props) { targetAdminClient = config.forwardingAdmin(config.targetAdminConfig("checkpoint-target-admin")); metrics = config.metrics(); idleConsumerGroupsOffset = new HashMap<>(); - checkpointsPerConsumerGroup = new HashMap<>(); + checkpointStore = new CheckpointStore(config, consumerGroups); scheduler = new Scheduler(getClass(), config.entityLabel(), config.adminTimeout()); scheduler.execute(() -> { - offsetSyncStore.start(); + // loading the stores are potentially long running operations, so they run asynchronously + // to avoid blocking task::start (until a task has completed starting it cannot be stopped) + boolean checkpointsReadOk = checkpointStore.start(); + offsetSyncStore.start(!checkpointsReadOk); scheduler.scheduleRepeating(this::refreshIdleConsumerGroupOffset, config.syncGroupOffsetsInterval(), "refreshing idle consumers group offsets at target cluster"); scheduler.scheduleRepeatingDelayed(this::syncGroupOffset, config.syncGroupOffsetsInterval(), "sync idle consumer group offset from source to target"); - }, "starting offset sync store"); + }, "starting checkpoint and offset sync stores"); log.info("{} checkpointing {} consumer groups {}->{}: {}.", Thread.currentThread().getName(), consumerGroups.size(), sourceClusterAlias, config.targetClusterAlias(), consumerGroups); } @@ -126,6 +133,7 @@ public void stop() { long start = System.currentTimeMillis(); stopping = true; Utils.closeQuietly(topicFilter, "topic filter"); + Utils.closeQuietly(checkpointStore, "checkpoints store"); Utils.closeQuietly(offsetSyncStore, "offset sync store"); Utils.closeQuietly(sourceAdminClient, "source admin client"); Utils.closeQuietly(targetAdminClient, "target admin client"); @@ -146,8 +154,8 @@ public List poll() throws InterruptedException { while (!stopping && System.currentTimeMillis() < deadline) { Thread.sleep(pollTimeout.toMillis()); } - if (stopping) { - // we are stopping, return early. + if (stopping || !checkpointStore.isInitialized()) { + // we are stopping, or not fully initialized, return early. return null; } List records = new ArrayList<>(); @@ -166,14 +174,13 @@ public List poll() throws InterruptedException { } } - - private List sourceRecordsForGroup(String group) throws InterruptedException { + // visible for testing + List sourceRecordsForGroup(String group) throws InterruptedException { try { long timestamp = System.currentTimeMillis(); Map upstreamGroupOffsets = listConsumerGroupOffsets(group); Map newCheckpoints = checkpointsForGroup(upstreamGroupOffsets, group); - Map oldCheckpoints = checkpointsPerConsumerGroup.computeIfAbsent(group, ignored -> new HashMap<>()); - oldCheckpoints.putAll(newCheckpoints); + checkpointStore.update(group, newCheckpoints); return newCheckpoints.values().stream() .map(x -> checkpointRecord(x, timestamp)) .collect(Collectors.toList()); @@ -195,7 +202,7 @@ Map checkpointsForGroup(Map checkpoints = checkpointsPerConsumerGroup.get(checkpoint.consumerGroupId()); + Map checkpoints = checkpointStore.get(checkpoint.consumerGroupId()); if (checkpoints == null) { log.trace("Emitting {} (first for this group)", checkpoint); return true; @@ -314,7 +321,7 @@ Map> syncGroupOffset() throws Exe Map> offsetToSyncAll = new HashMap<>(); // first, sync offsets for the idle consumers at target - for (Entry> group : getConvertedUpstreamOffset().entrySet()) { + for (Entry> group : checkpointStore.computeConvertedUpstreamOffset().entrySet()) { String consumerGroupId = group.getKey(); // for each idle consumer at target, read the checkpoints (converted upstream offset) // from the pre-populated map @@ -391,18 +398,4 @@ void syncGroupOffset(String consumerGroupId, Map> getConvertedUpstreamOffset() { - Map> result = new HashMap<>(); - - for (Entry> entry : checkpointsPerConsumerGroup.entrySet()) { - String consumerId = entry.getKey(); - Map convertedUpstreamOffset = new HashMap<>(); - for (Checkpoint checkpoint : entry.getValue().values()) { - convertedUpstreamOffset.put(checkpoint.topicPartition(), checkpoint.offsetAndMetadata()); - } - result.put(consumerId, convertedUpstreamOffset); - } - return result; - } } diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java index eca5cc68f997..16038044ddd2 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java @@ -54,7 +54,7 @@ * started after the position of the consumer group, or if relevant offset syncs for the topic were potentially used as * for translation in an earlier generation of the sync store. */ -class OffsetSyncStore implements AutoCloseable { +public class OffsetSyncStore implements AutoCloseable { private static final Logger log = LoggerFactory.getLogger(OffsetSyncStore.class); // Store one offset sync for each bit of the topic offset. @@ -63,8 +63,10 @@ class OffsetSyncStore implements AutoCloseable { private final KafkaBasedLog backingStore; private final Map offsetSyncs = new ConcurrentHashMap<>(); private final TopicAdmin admin; + protected volatile boolean initializationMustReadToEnd = true; protected volatile boolean readToEnd = false; + // package access to avoid Java 21 "this-escape" warning OffsetSyncStore(MirrorCheckpointConfig config) { Consumer consumer = null; TopicAdmin admin = null; @@ -97,6 +99,7 @@ private KafkaBasedLog createBackingStore(MirrorCheckpointConfig ); } + // for testing OffsetSyncStore() { this.admin = null; this.backingStore = null; @@ -105,12 +108,19 @@ private KafkaBasedLog createBackingStore(MirrorCheckpointConfig /** * Start the OffsetSyncStore, blocking until all previous Offset Syncs have been read from backing storage. */ - public void start() { - backingStore.start(); + public void start(boolean initializationMustReadToEnd) { + this.initializationMustReadToEnd = initializationMustReadToEnd; + log.debug("OffsetSyncStore starting - must read to OffsetSync end = {}", initializationMustReadToEnd); + backingStoreStart(); readToEnd = true; } - OptionalLong translateDownstream(String group, TopicPartition sourceTopicPartition, long upstreamOffset) { + // overridable for testing + void backingStoreStart() { + backingStore.start(false); + } + + public OptionalLong translateDownstream(String group, TopicPartition sourceTopicPartition, long upstreamOffset) { if (!readToEnd) { // If we have not read to the end of the syncs topic at least once, decline to translate any offsets. // This prevents emitting stale offsets while initially reading the offset syncs topic. @@ -214,7 +224,9 @@ private void updateSyncArray(OffsetSync[] syncs, OffsetSync[] original, OffsetSy // While reading to the end of the topic, ensure that our earliest sync is later than // any earlier sync that could have been used for translation, to preserve monotonicity // If the upstream offset rewinds, all previous offsets are invalid, so overwrite them all. - if (!readToEnd || syncs[0].upstreamOffset() > upstreamOffset) { + boolean onlyLoadLastOffset = !readToEnd && initializationMustReadToEnd; + boolean upstreamRewind = upstreamOffset < syncs[0].upstreamOffset(); + if (onlyLoadLastOffset || upstreamRewind) { clearSyncArray(syncs, offsetSync); return; } diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointStoreTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointStoreTest.java new file mode 100644 index 000000000000..b7b3904899f1 --- /dev/null +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointStoreTest.java @@ -0,0 +1,97 @@ +/* + * 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.connect.mirror; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TopicAuthorizationException; +import org.apache.kafka.connect.util.Callback; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class CheckpointStoreTest { + + @Test + public void testReadCheckpointsTopic() { + Set consumerGroups = new HashSet<>(); + consumerGroups.add("group1"); + + MirrorCheckpointTaskConfig config = mock(MirrorCheckpointTaskConfig.class); + when(config.checkpointsTopic()).thenReturn("checkpoint.topic"); + + try (CheckpointStore store = new CheckpointStore(config, consumerGroups) { + @Override + void readCheckpointsImpl(MirrorCheckpointTaskConfig config, Callback> consumedCallback) { + consumedCallback.onCompletion(null, newCheckpointRecord("group1", "t1", 0, 0, 0)); + // this record must be ignored as not part of consumerGroups for task + consumedCallback.onCompletion(null, newCheckpointRecord("group2", "t1", 0, 0, 0)); + // this record must be ignored as malformed + consumedCallback.onCompletion(null, + new ConsumerRecord<>("checkpoint.topic", 0, 0L, new byte[0], new byte[0])); + consumedCallback.onCompletion(null, newCheckpointRecord("group1", "t1", 0, 1, 1)); + } + }) { + assertFalse(store.isInitialized()); + + assertTrue(store.start(), "expected start to return success"); + assertTrue(store.isInitialized()); + + Map> expected = new HashMap<>(); + expected.put("group1", Collections.singletonMap(new TopicPartition("t1", 0), + new Checkpoint("group1", new TopicPartition("t1", 0), 1, 1, ""))); + assertEquals(expected, store.checkpointsPerConsumerGroup); + } + } + + @Test + public void testReadCheckpointsTopicError() { + Set consumerGroups = new HashSet<>(); + consumerGroups.add("group1"); + + MirrorCheckpointTaskConfig config = mock(MirrorCheckpointTaskConfig.class); + when(config.checkpointsTopic()).thenReturn("checkpoint.topic"); + + try (CheckpointStore store = new CheckpointStore(config, consumerGroups) { + @Override + void readCheckpointsImpl(MirrorCheckpointTaskConfig config, Callback> consumedCallback) { + consumedCallback.onCompletion(null, newCheckpointRecord("group1", "topic", 1, 0, 0)); + consumedCallback.onCompletion(new TopicAuthorizationException("test"), null); + } + }) { + + assertFalse(store.start(), "expected start to return failure"); + assertTrue(store.isInitialized()); + assertTrue(store.checkpointsPerConsumerGroup.isEmpty()); + } + } + + ConsumerRecord newCheckpointRecord(String gid, String topic, int partition, long upo, long dwo) { + Checkpoint cp = new Checkpoint(gid, new TopicPartition(topic, partition), upo, dwo, ""); + return new ConsumerRecord<>("checkpoint.topic", 0, 0L, cp.recordKey(), cp.recordValue()); + } +} diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java index c7aec3e54788..0afc4f74f2f9 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.connect.mirror; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Collections; import java.util.Optional; @@ -32,14 +33,19 @@ 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.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + public class MirrorCheckpointTaskTest { @Test public void testDownstreamTopicRenaming() { MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), null, Collections.emptyMap(), Collections.emptyMap()); + new DefaultReplicationPolicy(), null, Collections.emptySet(), Collections.emptyMap(), + new CheckpointStore(Collections.emptyMap())); assertEquals(new TopicPartition("source1.topic3", 4), mirrorCheckpointTask.renameTopicPartition(new TopicPartition("topic3", 4)), "Renaming source1.topic3 failed"); @@ -58,9 +64,10 @@ public void testCheckpoint() { long t2UpstreamOffset = 7L; long t2DownstreamOffset = 8L; OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); - offsetSyncStore.start(); + offsetSyncStore.start(true); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptyMap(), Collections.emptyMap()); + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), + Collections.emptyMap(), new CheckpointStore(Collections.emptyMap())); offsetSyncStore.sync(new TopicPartition("topic1", 2), t1UpstreamOffset, t1DownstreamOffset); offsetSyncStore.sync(new TopicPartition("target2.topic5", 6), t2UpstreamOffset, t2DownstreamOffset); Optional optionalCheckpoint1 = mirrorCheckpointTask.checkpoint("group9", new TopicPartition("topic1", 2), @@ -160,7 +167,8 @@ public void testSyncOffset() throws ExecutionException, InterruptedException { checkpointsPerConsumerGroup.put(consumer2, checkpointMapC2); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), null, idleConsumerGroupsOffset, checkpointsPerConsumerGroup); + new DefaultReplicationPolicy(), null, Collections.emptySet(), idleConsumerGroupsOffset, + new CheckpointStore(checkpointsPerConsumerGroup)); Map> output = mirrorCheckpointTask.syncGroupOffset(); @@ -190,7 +198,8 @@ public void testSyncOffsetForTargetGroupWithNullOffsetAndMetadata() throws Execu checkpointsPerConsumerGroup.put(consumer, checkpointMap); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source", "target", - new DefaultReplicationPolicy(), null, idleConsumerGroupsOffset, checkpointsPerConsumerGroup); + new DefaultReplicationPolicy(), null, Collections.emptySet(), idleConsumerGroupsOffset, + new CheckpointStore(checkpointsPerConsumerGroup)); Map> output = mirrorCheckpointTask.syncGroupOffset(); @@ -200,9 +209,10 @@ public void testSyncOffsetForTargetGroupWithNullOffsetAndMetadata() throws Execu @Test public void testNoCheckpointForTopicWithoutOffsetSyncs() { OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); - offsetSyncStore.start(); + offsetSyncStore.start(true); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptyMap(), Collections.emptyMap()); + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), + new CheckpointStore(Collections.emptyMap())); offsetSyncStore.sync(new TopicPartition("topic1", 0), 3L, 4L); Optional checkpoint1 = mirrorCheckpointTask.checkpoint("group9", new TopicPartition("topic1", 1), @@ -216,9 +226,10 @@ public void testNoCheckpointForTopicWithoutOffsetSyncs() { @Test public void testNoCheckpointForTopicWithNullOffsetAndMetadata() { OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); - offsetSyncStore.start(); + offsetSyncStore.start(true); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptyMap(), Collections.emptyMap()); + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), + new CheckpointStore(Collections.emptyMap())); offsetSyncStore.sync(new TopicPartition("topic1", 0), 1L, 3L); Optional checkpoint = mirrorCheckpointTask.checkpoint("g1", new TopicPartition("topic1", 0), null); assertFalse(checkpoint.isPresent()); @@ -227,10 +238,11 @@ public void testNoCheckpointForTopicWithNullOffsetAndMetadata() { @Test public void testCheckpointRecordsMonotonicIfStoreRewinds() { OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); - offsetSyncStore.start(); + offsetSyncStore.start(true); Map> checkpointsPerConsumerGroup = new HashMap<>(); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptyMap(), checkpointsPerConsumerGroup); + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), + new CheckpointStore(checkpointsPerConsumerGroup)); TopicPartition tp = new TopicPartition("topic1", 0); TopicPartition targetTP = new TopicPartition("source1.topic1", 0); @@ -252,7 +264,7 @@ public void testCheckpointRecordsMonotonicIfStoreRewinds() { offsetSyncStore.sync(tp, upstream++, downstream++); offsetSyncStore.sync(tp, upstream++, downstream++); offsetSyncStore.sync(tp, upstream++, downstream++); - offsetSyncStore.sync(tp, upstream++, downstream++); + offsetSyncStore.sync(tp, upstream, downstream); // The OffsetSyncStore will change its translation of the same offset assertNotEquals(OptionalLong.of(expectedDownstreamOffset), offsetSyncStore.translateDownstream("g1", tp, consumerGroupOffset)); // But the task will filter this out and not emit a checkpoint @@ -271,4 +283,93 @@ private Map assertCheckpointForTopic( assertEquals(truth, checkpoints.containsKey(remoteTp), "should" + (truth ? "" : " not") + " emit offset sync"); return checkpoints; } + + @Test + public void testCheckpointsTaskRestartUsesExistingCheckpoints() { + TopicPartition t1p0 = new TopicPartition("t1", 0); + TopicPartition sourceT1p0 = new TopicPartition("source1.t1", 0); + OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore() { + @Override + void backingStoreStart() { + // OffsetSyncStore contains entries for: 100->100, 200->200, 300->300 + for (int i = 100; i <= 300; i += 100) { + sync(t1p0, i, i); + } + } + }; + offsetSyncStore.start(false); + + MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), + new CheckpointStore(Collections.emptyMap())); + + // Generate a checkpoint for upstream offset 250, and assert it maps to downstream 201 + // (as nearest mapping in OffsetSyncStore is 200->200) + Map upstreamGroupOffsets = new HashMap<>(); + upstreamGroupOffsets.put(t1p0, new OffsetAndMetadata(250)); + Map checkpoints = mirrorCheckpointTask.checkpointsForGroup(upstreamGroupOffsets, "group1"); + assertEquals(1, checkpoints.size()); + assertEquals(new Checkpoint("group1", sourceT1p0, 250, 201, ""), checkpoints.get(sourceT1p0)); + + // Simulate task restart, during which more offsets are added to the sync topic, and thus the + // corresponding OffsetSyncStore no longer has a mapping for 100->100 + // Now OffsetSyncStore contains entries for: 175->175, 375->375, 475->475 + OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore2 = new OffsetSyncStoreTest.FakeOffsetSyncStore() { + @Override + void backingStoreStart() { + for (int i = 175; i <= 475; i += 100) { + sync(t1p0, i, i); + } + } + }; + offsetSyncStore2.start(false); + + // Simulate loading existing checkpoints into checkpointsPerConsumerGroup (250->201) + Map> checkpointsPerConsumerGroup = new HashMap<>(); + checkpointsPerConsumerGroup.put("group1", checkpoints); + MirrorCheckpointTask mirrorCheckpointTask2 = new MirrorCheckpointTask("source1", "target2", + new DefaultReplicationPolicy(), offsetSyncStore2, Collections.emptySet(), Collections.emptyMap(), + new CheckpointStore(checkpointsPerConsumerGroup)); + + // Upstream offsets 250 and 370 now have the closest downstream value of 176, but this is + // earlier than the downstream value of the last checkpoint (201) - so they are not emitted. + assertEquals(OptionalLong.of(176), offsetSyncStore2.translateDownstream(null, t1p0, 250)); + assertEquals(OptionalLong.of(176), offsetSyncStore2.translateDownstream(null, t1p0, 370)); + upstreamGroupOffsets.put(t1p0, new OffsetAndMetadata(250)); + assertTrue(mirrorCheckpointTask2.checkpointsForGroup(upstreamGroupOffsets, "group1").isEmpty()); + upstreamGroupOffsets.put(t1p0, new OffsetAndMetadata(370)); + assertTrue(mirrorCheckpointTask2.checkpointsForGroup(upstreamGroupOffsets, "group1").isEmpty()); + + // Upstream offset 400 has a closes downstream value of 376, and is emitted because it has + // a later downstream offset than the last checkpoint's downstream (201) + upstreamGroupOffsets.put(t1p0, new OffsetAndMetadata(400)); + Map checkpoints2 = mirrorCheckpointTask2.checkpointsForGroup(upstreamGroupOffsets, "group1"); + assertEquals(1, checkpoints2.size()); + assertEquals(new Checkpoint("group1", sourceT1p0, 400, 376, ""), checkpoints2.get(sourceT1p0)); + } + + @Test + public void testCheckpointStoreInitialized() throws InterruptedException { + CheckpointStore checkpointStore = mock(CheckpointStore.class); + + MirrorCheckpointTask task = new MirrorCheckpointTask("source1", "target2", + new DefaultReplicationPolicy(), + new OffsetSyncStoreTest.FakeOffsetSyncStore(), + Collections.singleton("group"), + Collections.emptyMap(), + checkpointStore) { + + @Override + List sourceRecordsForGroup(String group) { + SourceRecord sr = new SourceRecord(Collections.emptyMap(), Collections.emptyMap(), "", 0, null, null); + return Collections.singletonList(sr); + } + }; + + assertNull(task.poll()); + + when(checkpointStore.isInitialized()).thenReturn(true); + List polled = task.poll(); + assertEquals(1, polled.size()); + } } diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java index bc76a1994db9..3f2ddbc62e93 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java @@ -34,18 +34,22 @@ public class OffsetSyncStoreTest { static TopicPartition tp = new TopicPartition("topic1", 2); static class FakeOffsetSyncStore extends OffsetSyncStore { + private boolean startCalled = false; - FakeOffsetSyncStore() { - super(); + @Override + public void start(boolean initializationMustReadToEnd) { + startCalled = true; + super.start(initializationMustReadToEnd); } @Override - public void start() { - // do not call super to avoid NPE without a KafkaBasedLog. - readToEnd = true; + void backingStoreStart() { + // do not start KafkaBasedLog } + // simulate OffsetSync load as from KafkaBasedLog void sync(TopicPartition topicPartition, long upstreamOffset, long downstreamOffset) { + assertTrue(startCalled); // sync in tests should only be called after store.start OffsetSync offsetSync = new OffsetSync(topicPartition, upstreamOffset, downstreamOffset); byte[] key = offsetSync.recordKey(); byte[] value = offsetSync.recordValue(); @@ -57,7 +61,7 @@ void sync(TopicPartition topicPartition, long upstreamOffset, long downstreamOff @Test public void testOffsetTranslation() { try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { - store.start(); + store.start(true); // Emit synced downstream offset without dead-reckoning store.sync(tp, 100, 200); @@ -82,20 +86,24 @@ public void testOffsetTranslation() { @Test public void testNoTranslationIfStoreNotStarted() { - try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { + try (FakeOffsetSyncStore store = new FakeOffsetSyncStore() { + @Override + void backingStoreStart() { + // read a sync during startup + sync(tp, 100, 200); + assertEquals(OptionalLong.empty(), translateDownstream(null, tp, 0)); + assertEquals(OptionalLong.empty(), translateDownstream(null, tp, 100)); + assertEquals(OptionalLong.empty(), translateDownstream(null, tp, 200)); + } + }) { // no offsets exist and store is not started assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 0)); assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 100)); assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 200)); - // read a sync during startup - store.sync(tp, 100, 200); - assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 0)); - assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 100)); - assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 200)); - // After the store is started all offsets are visible - store.start(); + store.start(true); + assertEquals(OptionalLong.of(-1), store.translateDownstream(null, tp, 0)); assertEquals(OptionalLong.of(200), store.translateDownstream(null, tp, 100)); assertEquals(OptionalLong.of(201), store.translateDownstream(null, tp, 200)); @@ -105,26 +113,29 @@ public void testNoTranslationIfStoreNotStarted() { @Test public void testNoTranslationIfNoOffsetSync() { try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { - store.start(); + store.start(true); assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 0)); } } @Test public void testPastOffsetTranslation() { - try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { - int maxOffsetLag = 10; - int offset = 0; - for (; offset <= 1000; offset += maxOffsetLag) { - store.sync(tp, offset, offset); - assertSparseSyncInvariant(store, tp); + int maxOffsetLag = 10; + try (FakeOffsetSyncStore store = new FakeOffsetSyncStore() { + @Override + void backingStoreStart() { + for (int offset = 0; offset <= 1000; offset += maxOffsetLag) { + sync(tp, offset, offset); + assertSparseSyncInvariant(this, tp); + } } - store.start(); + }) { + store.start(true); // After starting but before seeing new offsets, only the latest startup offset can be translated assertSparseSync(store, 1000, -1); - for (; offset <= 10000; offset += maxOffsetLag) { + for (int offset = 1000 + maxOffsetLag; offset <= 10000; offset += maxOffsetLag) { store.sync(tp, offset, offset); assertSparseSyncInvariant(store, tp); } @@ -155,6 +166,55 @@ public void testPastOffsetTranslation() { } } + // this test has been written knowing the exact offsets syncs stored + @Test + public void testPastOffsetTranslationWithoutInitializationReadToEnd() { + final int maxOffsetLag = 10; + + try (FakeOffsetSyncStore store = new FakeOffsetSyncStore() { + @Override + void backingStoreStart() { + for (int offset = 0; offset <= 1000; offset += maxOffsetLag) { + sync(tp, offset, offset); + assertSparseSyncInvariant(this, tp); + } + } + }) { + + store.start(false); + + // After starting but before seeing new offsets + assertSparseSync(store, 480, 0); + assertSparseSync(store, 720, 480); + assertSparseSync(store, 1000, 990); + + for (int offset = 1000; offset <= 10000; offset += maxOffsetLag) { + store.sync(tp, offset, offset); + assertSparseSyncInvariant(store, tp); + } + + // After seeing new offsets, 1000 was kicked out of the store, so + // offsets before 3840 can only be translated to 1, only previously stored offset is 0 + assertSparseSync(store, 3840, 0); + assertSparseSync(store, 7680, 3840); + assertSparseSync(store, 8640, 7680); + assertSparseSync(store, 9120, 8640); + assertSparseSync(store, 9600, 9120); + assertSparseSync(store, 9840, 9600); + assertSparseSync(store, 9900, 9840); + assertSparseSync(store, 9960, 9900); + assertSparseSync(store, 9990, 9960); + assertSparseSync(store, 10000, 9990); + + // Rewinding upstream offsets should clear all historical syncs + store.sync(tp, 1500, 11000); + assertSparseSyncInvariant(store, tp); + assertEquals(OptionalLong.of(-1), store.translateDownstream(null, tp, 1499)); + assertEquals(OptionalLong.of(11000), store.translateDownstream(null, tp, 1500)); + assertEquals(OptionalLong.of(11001), store.translateDownstream(null, tp, 2000)); + } + } + @Test public void testConsistentlySpacedSyncs() { // Under normal operation, the incoming syncs will be regularly spaced and the store should keep a set of syncs @@ -215,7 +275,7 @@ public void testDroppedSyncsSpacing() { */ private void assertSyncSpacingHasBoundedExpirations(long firstOffset, LongStream steps, int maximumExpirations) { try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { - store.start(); + store.start(true); store.sync(tp, firstOffset, firstOffset); PrimitiveIterator.OfLong iterator = steps.iterator(); long offset = firstOffset; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java index 36203399766c..c1f19e334910 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java @@ -107,6 +107,8 @@ public class KafkaBasedLog { private boolean stopRequested; private final Queue> readLogEndOffsetCallbacks; private final java.util.function.Consumer initializer; + // initialized as false for backward compatibility + private volatile boolean reportErrorsToCallback = false; /** * Create a new KafkaBasedLog object. This does not start reading the log and writing is not permitted until @@ -243,7 +245,12 @@ public void stop() { } public void start() { - log.info("Starting KafkaBasedLog with topic " + topic); + start(false); + } + + public void start(boolean reportErrorsToCallback) { + this.reportErrorsToCallback = reportErrorsToCallback; + log.info("Starting KafkaBasedLog with topic {} reportErrorsToCallback={}", topic, reportErrorsToCallback); // Create the topic admin client and initialize the topic ... admin = topicAdminSupplier.get(); // may be null @@ -468,6 +475,9 @@ private void poll(long timeoutMs) { throw e; } catch (KafkaException e) { log.error("Error polling: " + e); + if (reportErrorsToCallback) { + consumedCallback.onCompletion(e, null); + } } } diff --git a/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala b/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala index e2064602c81a..83973ede23db 100755 --- a/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala +++ b/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala @@ -20,10 +20,12 @@ package kafka.metrics -import kafka.utils.{CoreUtils, VerifiableProperties} +import kafka.utils.VerifiableProperties import org.apache.kafka.server.metrics.MetricConfigs +import org.apache.kafka.server.util.Csv import scala.collection.Seq +import scala.jdk.CollectionConverters._ class KafkaMetricsConfig(props: VerifiableProperties) { @@ -31,8 +33,8 @@ class KafkaMetricsConfig(props: VerifiableProperties) { * Comma-separated list of reporter types. These classes should be on the * classpath and will be instantiated at run-time. */ - val reporters: Seq[String] = CoreUtils.parseCsvList(props.getString(MetricConfigs.KAFKA_METRICS_REPORTER_CLASSES_CONFIG, - MetricConfigs.KAFKA_METRIC_REPORTER_CLASSES_DEFAULT)) + val reporters: Seq[String] = Csv.parseCsvList(props.getString(MetricConfigs.KAFKA_METRICS_REPORTER_CLASSES_CONFIG, + MetricConfigs.KAFKA_METRIC_REPORTER_CLASSES_DEFAULT)).asScala /** * The metrics polling interval (in seconds). diff --git a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala index 1a10c6f65957..5f3fdc81887e 100644 --- a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala +++ b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala @@ -33,6 +33,7 @@ import org.apache.kafka.queue.{EventQueue, KafkaEventQueue} import org.apache.kafka.server.{ControllerRequestCompletionHandler, NodeToControllerChannelManager} import java.util.{Comparator, OptionalLong} +import scala.collection.mutable import scala.jdk.CollectionConverters._ /** @@ -58,7 +59,8 @@ class BrokerLifecycleManager( val time: Time, val threadNamePrefix: String, val isZkBroker: Boolean, - val logDirs: Set[Uuid] + val logDirs: Set[Uuid], + val shutdownHook: () => Unit = () => {} ) extends Logging { private def logPrefix(): String = { @@ -149,10 +151,11 @@ class BrokerLifecycleManager( private var readyToUnfence = false /** - * List of accumulated offline directories. + * Map of accumulated offline directories. The value is true if the directory couldn't be communicated + * to the Controller. * This variable can only be read or written from the event queue thread. */ - private var offlineDirs = Set[Uuid]() + private var offlineDirs = mutable.Map[Uuid, Boolean]() /** * True if we sent a event queue to the active controller requesting controlled @@ -253,8 +256,12 @@ class BrokerLifecycleManager( * Propagate directory failures to the controller. * @param directory The ID for the directory that failed. */ - def propagateDirectoryFailure(directory: Uuid): Unit = { + def propagateDirectoryFailure(directory: Uuid, timeout: Long): Unit = { eventQueue.append(new OfflineDirEvent(directory)) + // If we can't communicate the offline directory to the controller, we should shut down. + eventQueue.scheduleDeferred("offlineDirFailure", + new DeadlineFunction(time.nanoseconds() + MILLISECONDS.toNanos(timeout)), + new OfflineDirBrokerFailureEvent(directory)) } def handleKraftJBODMetadataVersionUpdate(): Unit = { @@ -327,9 +334,9 @@ class BrokerLifecycleManager( private class OfflineDirEvent(val dir: Uuid) extends EventQueue.Event { override def run(): Unit = { if (offlineDirs.isEmpty) { - offlineDirs = Set(dir) + offlineDirs = mutable.Map(dir -> false) } else { - offlineDirs = offlineDirs + dir + offlineDirs += (dir -> false) } if (registered) { scheduleNextCommunicationImmediately() @@ -337,6 +344,15 @@ class BrokerLifecycleManager( } } + private class OfflineDirBrokerFailureEvent(offlineDir: Uuid) extends EventQueue.Event { + override def run(): Unit = { + if (!offlineDirs.getOrElse(offlineDir, false)) { + error(s"Shutting down because couldn't communicate offline log dir $offlineDir with controllers") + shutdownHook() + } + } + } + private class StartupEvent(highestMetadataOffsetProvider: () => Long, channelManager: NodeToControllerChannelManager, clusterId: String, @@ -456,11 +472,11 @@ class BrokerLifecycleManager( setCurrentMetadataOffset(metadataOffset). setWantFence(!readyToUnfence). setWantShutDown(_state == BrokerState.PENDING_CONTROLLED_SHUTDOWN). - setOfflineLogDirs(offlineDirs.toSeq.asJava) + setOfflineLogDirs(offlineDirs.keys.toSeq.asJava) if (isTraceEnabled) { trace(s"Sending broker heartbeat $data") } - val handler = new BrokerHeartbeatResponseHandler() + val handler = new BrokerHeartbeatResponseHandler(offlineDirs.keys) _channelManager.sendRequest(new BrokerHeartbeatRequest.Builder(data), handler) communicationInFlight = true } @@ -468,18 +484,19 @@ class BrokerLifecycleManager( // the response handler is not invoked from the event handler thread, // so it is not safe to update state here, instead, schedule an event // to continue handling the response on the event handler thread - private class BrokerHeartbeatResponseHandler extends ControllerRequestCompletionHandler { + private class BrokerHeartbeatResponseHandler(currentOfflineDirs: Iterable[Uuid]) extends ControllerRequestCompletionHandler { override def onComplete(response: ClientResponse): Unit = { - eventQueue.prepend(new BrokerHeartbeatResponseEvent(response, false)) + eventQueue.prepend(new BrokerHeartbeatResponseEvent(response, false, currentOfflineDirs)) } override def onTimeout(): Unit = { info("Unable to send a heartbeat because the RPC got timed out before it could be sent.") - eventQueue.prepend(new BrokerHeartbeatResponseEvent(null, true)) + eventQueue.prepend(new BrokerHeartbeatResponseEvent(null, true, currentOfflineDirs)) } } - private class BrokerHeartbeatResponseEvent(response: ClientResponse, timedOut: Boolean) extends EventQueue.Event { + private class BrokerHeartbeatResponseEvent(response: ClientResponse, timedOut: Boolean, + currentOfflineDirs: Iterable[Uuid]) extends EventQueue.Event { override def run(): Unit = { communicationInFlight = false if (timedOut) { @@ -507,6 +524,7 @@ class BrokerLifecycleManager( if (errorCode == Errors.NONE) { val responseData = message.data() failedAttempts = 0 + currentOfflineDirs.foreach(cur => offlineDirs.put(cur, true)) _state match { case BrokerState.STARTING => if (responseData.isCaughtUp) { diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 94da0fe58375..ef81bee016ba 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -57,7 +57,7 @@ import java.util import java.util.Optional import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.locks.{Condition, ReentrantLock} -import java.util.concurrent.{CompletableFuture, ExecutionException, TimeUnit, TimeoutException} +import java.util.concurrent.{CompletableFuture, ExecutionException, TimeoutException, TimeUnit} import scala.collection.Map import scala.compat.java8.OptionConverters.RichOptionForJava8 import scala.jdk.CollectionConverters._ @@ -210,7 +210,8 @@ class BrokerServer( time, s"broker-${config.nodeId}-", isZkBroker = false, - logDirs = logManager.directoryIdsSet) + logDirs = logManager.directoryIdsSet, + () => new Thread(() => shutdown(), "kafka-shutdown-thread").start()) // Enable delegation token cache for all SCRAM mechanisms to simplify dynamic update. // This keeps the cache up-to-date if new SCRAM mechanisms are enabled dynamically. @@ -304,7 +305,7 @@ class BrokerServer( assignmentsManager.onAssignment(partition, directoryId, reason, callback) override def handleFailure(directoryId: Uuid): Unit = - lifecycleManager.propagateDirectoryFailure(directoryId) + lifecycleManager.propagateDirectoryFailure(directoryId, config.logDirFailureTimeoutMs) } this._replicaManager = new ReplicaManager( diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 9a228e014365..4c4217404b37 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -21,7 +21,6 @@ import java.{lang, util} import java.util.concurrent.TimeUnit import java.util.{Collections, Properties} import kafka.cluster.EndPoint -import kafka.utils.CoreUtils.parseCsvList import kafka.utils.{CoreUtils, Logging} import kafka.utils.Implicits._ import org.apache.kafka.clients.CommonClientConfigs @@ -345,6 +344,7 @@ object KafkaConfig { .define(ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_CONFIG, BOOLEAN, ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_DEFAULT, LOW, ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_DOC) .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC) + .define(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG, LONG, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DEFAULT, atLeast(1), LOW, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DOC) /** ********* Replication configuration ***********/ .define(ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_CONFIG, INT, ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_DEFAULT, MEDIUM, ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_DOC) @@ -895,7 +895,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami /** ********* Log Configuration ***********/ val autoCreateTopicsEnable = getBoolean(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG) val numPartitions = getInt(ServerLogConfigs.NUM_PARTITIONS_CONFIG) - val logDirs = CoreUtils.parseCsvList(Option(getString(ServerLogConfigs.LOG_DIRS_CONFIG)).getOrElse(getString(ServerLogConfigs.LOG_DIR_CONFIG))) + val logDirs: Seq[String] = Csv.parseCsvList(Option(getString(ServerLogConfigs.LOG_DIRS_CONFIG)).getOrElse(getString(ServerLogConfigs.LOG_DIR_CONFIG))).asScala def logSegmentBytes = getInt(ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG) def logFlushIntervalMessages = getLong(ServerLogConfigs.LOG_FLUSH_INTERVAL_MESSAGES_CONFIG) val logCleanerThreads = getInt(CleanerConfig.LOG_CLEANER_THREADS_PROP) @@ -975,6 +975,8 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami def logMessageDownConversionEnable: Boolean = getBoolean(ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_CONFIG) + def logDirFailureTimeoutMs: Long = getLong(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG) + /** ********* Replication configuration ***********/ val controllerSocketTimeoutMs: Int = getInt(ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_CONFIG) val defaultReplicationFactor: Int = getInt(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG) @@ -1292,7 +1294,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami // check controller listener names (they won't appear in listeners when process.roles=broker) // as well as listeners for occurrences of SSL or SASL_* if (controllerListenerNames.exists(isSslOrSasl) || - parseCsvList(getString(SocketServerConfigs.LISTENERS_CONFIG)).exists(listenerValue => isSslOrSasl(EndPoint.parseListenerName(listenerValue)))) { + Csv.parseCsvList(getString(SocketServerConfigs.LISTENERS_CONFIG)).asScala.exists(listenerValue => isSslOrSasl(EndPoint.parseListenerName(listenerValue)))) { mapValue // don't add default mappings since we found something that is SSL or SASL_* } else { // add the PLAINTEXT mappings for all controller listener names that are not explicitly PLAINTEXT diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 35499430d667..e69fd60385f9 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -2440,7 +2440,9 @@ class ReplicaManager(val config: KafkaConfig, def handleLogDirFailure(dir: String, notifyController: Boolean = true): Unit = { if (!logManager.isLogDirOnline(dir)) return - warn(s"Stopping serving replicas in dir $dir") + // retrieve the UUID here because logManager.handleLogDirFailure handler removes it + val uuid = logManager.directoryId(dir) + warn(s"Stopping serving replicas in dir $dir with uuid $uuid because the log directory has failed.") replicaStateChangeLock synchronized { val newOfflinePartitions = onlinePartitionsIterator.filter { partition => partition.log.exists { _.parentDir == dir } @@ -2465,8 +2467,6 @@ class ReplicaManager(val config: KafkaConfig, warn(s"Broker $localBrokerId stopped fetcher for partitions ${newOfflinePartitions.mkString(",")} and stopped moving logs " + s"for partitions ${partitionsWithOfflineFutureReplica.mkString(",")} because they are in the failed log directory $dir.") } - // retrieve the UUID here because logManager.handleLogDirFailure handler removes it - val uuid = logManager.directoryId(dir) logManager.handleLogDirFailure(dir) if (dir == new File(config.metadataLogDir).getAbsolutePath && (config.processRoles.nonEmpty || config.migrationEnabled)) { fatal(s"Shutdown broker because the metadata log dir $dir has failed") diff --git a/core/src/main/scala/kafka/utils/CoreUtils.scala b/core/src/main/scala/kafka/utils/CoreUtils.scala index 0b445ed1a3cd..8da7a4e7cc14 100755 --- a/core/src/main/scala/kafka/utils/CoreUtils.scala +++ b/core/src/main/scala/kafka/utils/CoreUtils.scala @@ -32,6 +32,7 @@ import org.apache.commons.validator.routines.InetAddressValidator import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.Utils +import org.apache.kafka.server.util.Csv import org.slf4j.event.Level import java.util @@ -109,17 +110,6 @@ object CoreUtils { } } - /** - * Parse a comma separated string into a sequence of strings. - * Whitespace surrounding the comma will be removed. - */ - def parseCsvList(csvList: String): Seq[String] = { - if (csvList == null || csvList.isEmpty) - Seq.empty[String] - else - csvList.split("\\s*,\\s*").filter(v => !v.equals("")) - } - /** * Create an instance of the class with the given class name */ @@ -219,8 +209,8 @@ object CoreUtils { } val endPoints = try { - val listenerList = parseCsvList(listeners) - listenerList.map(EndPoint.createEndPoint(_, Some(securityProtocolMap))) + val listenerList = Csv.parseCsvList(listeners) + listenerList.asScala.map(EndPoint.createEndPoint(_, Some(securityProtocolMap))) } catch { case e: Exception => throw new IllegalArgumentException(s"Error creating broker listeners from '$listeners': ${e.getMessage}", e) diff --git a/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala b/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala index 7752f0dae224..34f9d139a03c 100644 --- a/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala @@ -232,11 +232,11 @@ class BrokerLifecycleManagerTest { poll(ctx, manager, prepareResponse[BrokerHeartbeatRequest](ctx, new BrokerHeartbeatResponse(new BrokerHeartbeatResponseData()))) .data().offlineLogDirs().asScala.map(_.toString).toSet assertEquals(Set.empty, nextHeartbeatDirs()) - manager.propagateDirectoryFailure(Uuid.fromString("h3sC4Yk-Q9-fd0ntJTocCA")) + manager.propagateDirectoryFailure(Uuid.fromString("h3sC4Yk-Q9-fd0ntJTocCA"), Integer.MAX_VALUE) assertEquals(Set("h3sC4Yk-Q9-fd0ntJTocCA"), nextHeartbeatDirs()) - manager.propagateDirectoryFailure(Uuid.fromString("ej8Q9_d2Ri6FXNiTxKFiow")) + manager.propagateDirectoryFailure(Uuid.fromString("ej8Q9_d2Ri6FXNiTxKFiow"), Integer.MAX_VALUE) assertEquals(Set("h3sC4Yk-Q9-fd0ntJTocCA", "ej8Q9_d2Ri6FXNiTxKFiow"), nextHeartbeatDirs()) - manager.propagateDirectoryFailure(Uuid.fromString("1iF76HVNRPqC7Y4r6647eg")) + manager.propagateDirectoryFailure(Uuid.fromString("1iF76HVNRPqC7Y4r6647eg"), Integer.MAX_VALUE) assertEquals(Set("h3sC4Yk-Q9-fd0ntJTocCA", "ej8Q9_d2Ri6FXNiTxKFiow", "1iF76HVNRPqC7Y4r6647eg"), nextHeartbeatDirs()) manager.close() } diff --git a/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala b/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala index 53bd2b0e0da4..9a6af049f8de 100644 --- a/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala @@ -21,13 +21,14 @@ import java.util.Collections import java.util.concurrent.{ExecutionException, TimeUnit} import kafka.api.IntegrationTestHarness import kafka.controller.{OfflineReplica, PartitionAndReplica} -import kafka.utils.TestUtils.{Checkpoint, LogDirFailureType, Roll, waitUntilTrue} +import kafka.utils.TestUtils.{waitUntilTrue, Checkpoint, LogDirFailureType, Roll} import kafka.utils.{CoreUtils, Exit, TestUtils} import org.apache.kafka.clients.consumer.Consumer import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.{KafkaStorageException, NotLeaderOrFollowerException} import org.apache.kafka.common.utils.Utils +import org.apache.kafka.metadata.BrokerState import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} @@ -52,6 +53,8 @@ class LogDirFailureTest extends IntegrationTestHarness { this.serverConfig.setProperty(ReplicationConfigs.REPLICA_HIGH_WATERMARK_CHECKPOINT_INTERVAL_MS_CONFIG, "60000") this.serverConfig.setProperty(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, "1") + this.serverConfig.setProperty(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG, "5000") + this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") @BeforeEach override def setUp(testInfo: TestInfo): Unit = { @@ -66,6 +69,31 @@ class LogDirFailureTest extends IntegrationTestHarness { testProduceErrorsFromLogDirFailureOnLeader(Roll) } + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testLogDirNotificationTimeout(quorum: String): Unit = { + // Disable retries to allow exception to bubble up for validation + this.producerConfig.setProperty(ProducerConfig.RETRIES_CONFIG, "0") + this.producerConfig.setProperty(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "false") + val producer = createProducer() + + val partition = new TopicPartition(topic, 0) + + val leaderServerId = producer.partitionsFor(topic).asScala.find(_.partition() == 0).get.leader().id() + val leaderServer = brokers.find(_.config.brokerId == leaderServerId).get + + // shut down the controller to simulate the case where the broker is not able to send the log dir notification + controllerServer.shutdown() + controllerServer.awaitShutdown() + + TestUtils.causeLogDirFailure(Checkpoint, leaderServer, partition) + + TestUtils.waitUntilTrue(() => leaderServer.brokerState == BrokerState.SHUTTING_DOWN, + s"Expected broker to be in NOT_RUNNING state but was ${leaderServer.brokerState}", 15000) + // wait for actual shutdown (by default max 5 minutes for graceful shutdown) + leaderServer.awaitShutdown() + } + @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testIOExceptionDuringLogRoll(quorum: String): Unit = { diff --git a/core/src/test/scala/unit/kafka/utils/CoreUtilsTest.scala b/core/src/test/scala/unit/kafka/utils/CoreUtilsTest.scala index e96da4cd7d3a..8acaa9586104 100755 --- a/core/src/test/scala/unit/kafka/utils/CoreUtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/CoreUtilsTest.scala @@ -89,19 +89,6 @@ class CoreUtilsTest extends Logging { assertEquals(Integer.MAX_VALUE, Utils.abs(Integer.MAX_VALUE)) } - @Test - def testCsvList(): Unit = { - val emptyString:String = "" - val nullString:String = null - val emptyList = CoreUtils.parseCsvList(emptyString) - val emptyListFromNullString = CoreUtils.parseCsvList(nullString) - val emptyStringList = Seq.empty[String] - assertTrue(emptyList!=null) - assertTrue(emptyListFromNullString!=null) - assertTrue(emptyStringList.equals(emptyListFromNullString)) - assertTrue(emptyStringList.equals(emptyList)) - } - @Test def testInLock(): Unit = { val lock = new ReentrantLock() diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 29732cc1b80d..f7b6cef051d9 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -103,10 +103,10 @@ versions += [ gradle: "8.7", grgit: "4.1.1", httpclient: "4.5.14", - jackson: "2.16.1", + jackson: "2.16.2", jacoco: "0.8.10", javassist: "3.29.2-GA", - jetty: "9.4.53.v20231009", + jetty: "9.4.54.v20240208", jersey: "2.39.1", jline: "3.25.1", jmh: "1.37", @@ -146,7 +146,7 @@ versions += [ lz4: "1.8.0", mavenArtifact: "3.9.6", metrics: "2.2.0", - netty: "4.1.100.Final", + netty: "4.1.110.Final", opentelemetryProto: "1.0.0-alpha", pcollections: "4.0.1", powermock: "2.0.9", @@ -165,7 +165,7 @@ versions += [ spotbugs: "4.8.0", zinc: "1.9.2", zookeeper: "3.8.4", - zstd: "1.5.5-11" + zstd: "1.5.6-3" ] libs += [ 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 6fe7928dc750..5e4e899faa68 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 @@ -423,12 +423,11 @@ public CompletableFuture heartbeat( ); } - // Using a read operation is okay here as we ignore the last committed offset in the snapshot registry. - // This means we will read whatever is in the latest snapshot, which is how the old coordinator behaves. - return runtime.scheduleReadOperation( + return runtime.scheduleWriteOperation( "classic-group-heartbeat", topicPartitionFor(request.groupId()), - (coordinator, __) -> coordinator.classicGroupHeartbeat(context, request) + Duration.ofMillis(config.offsetCommitTimeoutMs), + coordinator -> coordinator.classicGroupHeartbeat(context, request) ).exceptionally(exception -> handleOperationException( "classic-group-heartbeat", request, diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java index c01605926ba7..081c9764944c 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java @@ -355,9 +355,10 @@ public CoordinatorResult classicGroupSync( * @param context The request context. * @param request The actual Heartbeat request. * - * @return The HeartbeatResponse. + * @return A Result containing the heartbeat response and + * a list of records to update the state machine. */ - public HeartbeatResponseData classicGroupHeartbeat( + public CoordinatorResult classicGroupHeartbeat( RequestContext context, HeartbeatRequestData request ) { 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 5520676d21b1..9dae56ef7a62 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 @@ -1273,6 +1273,7 @@ private void throwIfRebalanceInProgress( // member has already rejoined, so it needs to first finish revoking the partitions and the reconciliation, // and then the next rejoin will be triggered automatically if needed. if (group.groupEpoch() > member.memberEpoch() && !member.state().equals(MemberState.UNREVOKED_PARTITIONS)) { + scheduleConsumerGroupJoinTimeoutIfAbsent(group.groupId(), member.memberId(), member.rebalanceTimeoutMs()); throw Errors.REBALANCE_IN_PROGRESS.exception( String.format("A new rebalance is triggered in group %s and member %s should rejoin to catch up.", group.groupId(), member.memberId()) @@ -1753,6 +1754,7 @@ private CoordinatorResult classicGroupJoinToConsumerGro CompletableFuture appendFuture = new CompletableFuture<>(); appendFuture.whenComplete((__, t) -> { if (t == null) { + cancelConsumerGroupJoinTimeout(groupId, response.memberId()); scheduleConsumerGroupSessionTimeout(groupId, response.memberId(), sessionTimeoutMs); // The sync timeout ensures that the member send sync request within the rebalance timeout. scheduleConsumerGroupSyncTimeout(groupId, response.memberId(), request.rebalanceTimeoutMs()); @@ -2077,6 +2079,39 @@ private void scheduleConsumerGroupSessionTimeout( scheduleConsumerGroupSessionTimeout(groupId, memberId, consumerGroupSessionTimeoutMs); } + /** + * Fences a member from a consumer group. Returns an empty CoordinatorResult + * if the group or the member doesn't exist. + * + * @param groupId The group id. + * @param memberId The member id. + * @param reason The reason for fencing the member. + * + * @return The CoordinatorResult to be applied. + */ + private CoordinatorResult consumerGroupFenceMemberOperation( + String groupId, + String memberId, + String reason + ) { + try { + ConsumerGroup group = consumerGroup(groupId); + ConsumerGroupMember member = group.getOrMaybeCreateMember(memberId, false); + log.info("[GroupId {}] Member {} fenced from the group because {}.", + groupId, memberId, reason); + + return consumerGroupFenceMember(group, member, null); + } catch (GroupIdNotFoundException ex) { + log.debug("[GroupId {}] Could not fence {} because the group does not exist.", + groupId, memberId); + } catch (UnknownMemberIdException ex) { + log.debug("[GroupId {}] Could not fence {} because the member does not exist.", + groupId, memberId); + } + + return new CoordinatorResult<>(Collections.emptyList()); + } + /** * Schedules (or reschedules) the session timeout for the member. * @@ -2089,25 +2124,13 @@ private void scheduleConsumerGroupSessionTimeout( String memberId, int sessionTimeoutMs ) { - String key = consumerGroupSessionTimeoutKey(groupId, memberId); - timer.schedule(key, sessionTimeoutMs, TimeUnit.MILLISECONDS, true, () -> { - try { - ConsumerGroup group = consumerGroup(groupId); - ConsumerGroupMember member = group.getOrMaybeCreateMember(memberId, false); - log.info("[GroupId {}] Member {} fenced from the group because its session expired.", - groupId, memberId); - - return consumerGroupFenceMember(group, member, null); - } catch (GroupIdNotFoundException ex) { - log.debug("[GroupId {}] Could not fence {} because the group does not exist.", - groupId, memberId); - } catch (UnknownMemberIdException ex) { - log.debug("[GroupId {}] Could not fence {} because the member does not exist.", - groupId, memberId); - } - - return new CoordinatorResult<>(Collections.emptyList()); - }); + timer.schedule( + consumerGroupSessionTimeoutKey(groupId, memberId), + sessionTimeoutMs, + TimeUnit.MILLISECONDS, + true, + () -> consumerGroupFenceMemberOperation(groupId, memberId, "the member session expired.") + ); } /** @@ -2180,36 +2203,58 @@ private void cancelConsumerGroupRebalanceTimeout( } /** - * Schedules a sync timeout for the member. + * Schedules a join timeout for the member if there's not a join timeout. * * @param groupId The group id. * @param memberId The member id. * @param rebalanceTimeoutMs The rebalance timeout. */ - private void scheduleConsumerGroupSyncTimeout( + private void scheduleConsumerGroupJoinTimeoutIfAbsent( String groupId, String memberId, int rebalanceTimeoutMs ) { - String key = consumerGroupSyncKey(groupId, memberId); - timer.schedule(key, rebalanceTimeoutMs, TimeUnit.MILLISECONDS, true, () -> { - try { - ConsumerGroup group = consumerGroup(groupId); - ConsumerGroupMember member = group.getOrMaybeCreateMember(memberId, false); - log.info("[GroupId {}] Member {} fenced from the group because its session expired.", - groupId, memberId); + timer.scheduleIfAbsent( + consumerGroupJoinKey(groupId, memberId), + rebalanceTimeoutMs, + TimeUnit.MILLISECONDS, + true, + () -> consumerGroupFenceMemberOperation(groupId, memberId, "the classic member failed to join within the rebalance timeout.") + ); + } - return consumerGroupFenceMember(group, member, null); - } catch (GroupIdNotFoundException ex) { - log.debug("[GroupId {}] Could not fence {} because the group does not exist.", - groupId, memberId); - } catch (UnknownMemberIdException ex) { - log.debug("[GroupId {}] Could not fence {} because the member does not exist.", - groupId, memberId); - } + /** + * Cancels the join timeout of the member. + * + * @param groupId The group id. + * @param memberId The member id. + */ + private void cancelConsumerGroupJoinTimeout( + String groupId, + String memberId + ) { + timer.cancel(consumerGroupJoinKey(groupId, memberId)); + } - return new CoordinatorResult<>(Collections.emptyList()); - }); + /** + * Schedules a sync timeout for the member. + * + * @param groupId The group id. + * @param memberId The member id. + * @param rebalanceTimeoutMs The rebalance timeout. + */ + private void scheduleConsumerGroupSyncTimeout( + String groupId, + String memberId, + int rebalanceTimeoutMs + ) { + timer.schedule( + consumerGroupSyncKey(groupId, memberId), + rebalanceTimeoutMs, + TimeUnit.MILLISECONDS, + true, + () -> consumerGroupFenceMemberOperation(groupId, memberId, "the member failed to sync within timeout.") + ); } /** @@ -4072,19 +4117,7 @@ private CoordinatorResult classicGroupSyncToConsumerGro String groupId = request.groupId(); String memberId = request.memberId(); String instanceId = request.groupInstanceId(); - - ConsumerGroupMember member; - if (instanceId == null) { - member = group.getOrMaybeCreateMember(request.memberId(), false); - } else { - member = group.staticMember(instanceId); - if (member == null) { - throw new UnknownMemberIdException( - String.format("Member with instance id %s is not a member of group %s.", instanceId, groupId) - ); - } - throwIfInstanceIdIsFenced(member, groupId, memberId, instanceId); - } + ConsumerGroupMember member = validateConsumerGroupMember(group, memberId, instanceId); throwIfMemberDoesNotUseClassicProtocol(member); throwIfGenerationIdUnmatched(member.memberId(), member.memberEpoch(), request.generationId()); @@ -4209,23 +4242,56 @@ private void removePendingSyncMember( * @param context The request context. * @param request The actual Heartbeat request. * - * @return The Heartbeat response. + * @return The coordinator result that contains the heartbeat response. */ - public HeartbeatResponseData classicGroupHeartbeat( + public CoordinatorResult classicGroupHeartbeat( RequestContext context, HeartbeatRequestData request ) { - ClassicGroup group = getOrMaybeCreateClassicGroup(request.groupId(), false); + Group group = groups.get(request.groupId(), Long.MAX_VALUE); + if (group == null) { + throw new UnknownMemberIdException( + String.format("Group %s not found.", request.groupId()) + ); + } + + if (group.type() == CLASSIC) { + return classicGroupHeartbeatToClassicGroup((ClassicGroup) group, context, request); + } else { + return classicGroupHeartbeatToConsumerGroup((ConsumerGroup) group, context, request); + } + } + + /** + * Handle a classic group HeartbeatRequest to a classic group. + * + * @param group The ClassicGroup. + * @param context The request context. + * @param request The actual Heartbeat request. + * + * @return The coordinator result that contains the heartbeat response. + */ + private CoordinatorResult classicGroupHeartbeatToClassicGroup( + ClassicGroup group, + RequestContext context, + HeartbeatRequestData request + ) { validateClassicGroupHeartbeat(group, request.memberId(), request.groupInstanceId(), request.generationId()); switch (group.currentState()) { case EMPTY: - return new HeartbeatResponseData().setErrorCode(Errors.UNKNOWN_MEMBER_ID.code()); + return new CoordinatorResult<>( + Collections.emptyList(), + new HeartbeatResponseData().setErrorCode(Errors.UNKNOWN_MEMBER_ID.code()) + ); case PREPARING_REBALANCE: rescheduleClassicGroupMemberHeartbeat(group, group.member(request.memberId())); - return new HeartbeatResponseData().setErrorCode(Errors.REBALANCE_IN_PROGRESS.code()); + return new CoordinatorResult<>( + Collections.emptyList(), + new HeartbeatResponseData().setErrorCode(Errors.REBALANCE_IN_PROGRESS.code()) + ); case COMPLETING_REBALANCE: case STABLE: @@ -4233,7 +4299,10 @@ public HeartbeatResponseData classicGroupHeartbeat( // is in CompletingRebalance state. In this case, we should treat them as // normal heartbeat requests and reset the timer rescheduleClassicGroupMemberHeartbeat(group, group.member(request.memberId())); - return new HeartbeatResponseData(); + return new CoordinatorResult<>( + Collections.emptyList(), + new HeartbeatResponseData() + ); default: throw new IllegalStateException("Reached unexpected state " + @@ -4274,6 +4343,81 @@ private void validateClassicGroupHeartbeat( } } + /** + * Handle a classic group HeartbeatRequest to a consumer group. A response with + * REBALANCE_IN_PROGRESS is returned if 1) the member epoch is smaller than the + * group epoch, 2) the member is in UNREVOKED_PARTITIONS, or 3) the member is in + * UNRELEASED_PARTITIONS and all its partitions pending assignment are free. + * + * @param group The ConsumerGroup. + * @param context The request context. + * @param request The actual Heartbeat request. + * + * @return The coordinator result that contains the heartbeat response. + */ + private CoordinatorResult classicGroupHeartbeatToConsumerGroup( + ConsumerGroup group, + RequestContext context, + HeartbeatRequestData request + ) throws UnknownMemberIdException, FencedInstanceIdException, IllegalGenerationException { + String groupId = request.groupId(); + String memberId = request.memberId(); + String instanceId = request.groupInstanceId(); + ConsumerGroupMember member = validateConsumerGroupMember(group, memberId, instanceId); + + throwIfMemberDoesNotUseClassicProtocol(member); + throwIfGenerationIdUnmatched(memberId, member.memberEpoch(), request.generationId()); + + scheduleConsumerGroupSessionTimeout(groupId, memberId, member.classicProtocolSessionTimeout().get()); + + Errors error = Errors.NONE; + // The member should rejoin if any of the following conditions is met. + // 1) The group epoch is bumped so the member need to rejoin to catch up. + // 2) The member needs to revoke some partitions and rejoin to reconcile with the new epoch. + // 3) The member's partitions pending assignment are free, so it can rejoin to get the complete assignment. + if (member.memberEpoch() < group.groupEpoch() || + member.state() == MemberState.UNREVOKED_PARTITIONS || + (member.state() == MemberState.UNRELEASED_PARTITIONS && !group.waitingOnUnreleasedPartition(member))) { + error = Errors.REBALANCE_IN_PROGRESS; + scheduleConsumerGroupJoinTimeoutIfAbsent(groupId, memberId, member.rebalanceTimeoutMs()); + } + + return new CoordinatorResult<>( + Collections.emptyList(), + new HeartbeatResponseData().setErrorCode(error.code()) + ); + } + + /** + * Validates that (1) the instance id exists and is mapped to the member id + * if the group instance id is provided; and (2) the member id exists in the group. + * + * @param group The consumer group. + * @param memberId The member id. + * @param instanceId The instance id. + * + * @return The ConsumerGroupMember. + */ + private ConsumerGroupMember validateConsumerGroupMember( + ConsumerGroup group, + String memberId, + String instanceId + ) throws UnknownMemberIdException, FencedInstanceIdException { + ConsumerGroupMember member; + if (instanceId == null) { + member = group.getOrMaybeCreateMember(memberId, false); + } else { + member = group.staticMember(instanceId); + if (member == null) { + throw new UnknownMemberIdException( + String.format("Member with instance id %s is not a member of group %s.", instanceId, group.groupId()) + ); + } + throwIfInstanceIdIsFenced(member, group.groupId(), memberId, instanceId); + } + return member; + } + /** * Handle a classic LeaveGroupRequest. * @@ -4583,6 +4727,20 @@ static String classicGroupSyncKey(String groupId) { return "sync-" + groupId; } + /** + * Generate a consumer group join key for the timer. + * + * Package private for testing. + * + * @param groupId The group id. + * @param memberId The member id. + * + * @return the sync key. + */ + static String consumerGroupJoinKey(String groupId, String memberId) { + return "join-" + groupId + "-" + memberId; + } + /** * Generate a consumer group sync key for the timer. * diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java index 9e334bb413b7..dd3a6f2e7bfd 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java @@ -1300,4 +1300,27 @@ public boolean allMembersUseClassicProtocolExcept(String memberId) { return numClassicProtocolMembers() == members().size() - 1 && !getOrMaybeCreateMember(memberId, false).useClassicProtocol(); } + + /** + * Checks whether the member has any unreleased partition. + * + * @param member The member to check. + * @return A boolean indicating whether the member has partitions in the target + * assignment that hasn't been revoked by other members. + */ + public boolean waitingOnUnreleasedPartition(ConsumerGroupMember member) { + if (member.state() == MemberState.UNRELEASED_PARTITIONS) { + for (Map.Entry> entry : targetAssignment().get(member.memberId()).partitions().entrySet()) { + Uuid topicId = entry.getKey(); + Set assignedPartitions = member.assignedPartitions().getOrDefault(topicId, Collections.emptySet()); + + for (int partition : entry.getValue()) { + if (!assignedPartitions.contains(partition) && currentPartitionEpoch(topicId, partition) != -1) { + return true; + } + } + } + } + return false; + } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java index ed6f649d61fc..c0c194a90104 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java @@ -402,6 +402,19 @@ public void run() { timer.add(task); } + @Override + public void scheduleIfAbsent( + String key, + long delay, + TimeUnit unit, + boolean retry, + TimeoutOperation operation + ) { + if (!tasks.containsKey(key)) { + schedule(key, delay, unit, retry, 500, operation); + } + } + @Override public void cancel(String key) { TimerTask prevTask = tasks.remove(key); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorTimer.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorTimer.java index 4c5b3aa8fd68..ef5dfc6dc39a 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorTimer.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorTimer.java @@ -66,6 +66,18 @@ interface TimeoutOperation { */ void schedule(String key, long delay, TimeUnit unit, boolean retry, long retryBackoff, TimeoutOperation operation); + /** + * Add an operation to the timer if there's no operation with the same key. + * + * @param key The key to identify this operation. + * @param delay The delay to wait before expiring. + * @param unit The delay unit. + * @param retry A boolean indicating whether the operation should + * be retried on failure. + * @param operation The operation to perform upon expiration. + */ + void scheduleIfAbsent(String key, long delay, TimeUnit unit, boolean retry, TimeoutOperation operation); + /** * Remove an operation corresponding to a given key. * diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessor.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessor.java index 20337a467758..31fa52ea7d15 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessor.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessor.java @@ -112,9 +112,6 @@ public MultiThreadedEventProcessor( */ private class EventProcessorThread extends Thread { private final Logger log; - private long pollStartMs; - private long timeSinceLastPollMs; - private long lastPollMs; EventProcessorThread( String name 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 731fa5ca08a8..6cd96458c647 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 @@ -676,9 +676,10 @@ public void testHeartbeat() throws Exception { service.startup(() -> 1); - when(runtime.scheduleReadOperation( + when(runtime.scheduleWriteOperation( ArgumentMatchers.eq("classic-group-heartbeat"), ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)), + ArgumentMatchers.eq(Duration.ofMillis(5000)), ArgumentMatchers.any() )).thenReturn(CompletableFuture.completedFuture( new HeartbeatResponseData() @@ -708,9 +709,10 @@ public void testHeartbeatCoordinatorNotAvailableException() throws Exception { service.startup(() -> 1); - when(runtime.scheduleReadOperation( + when(runtime.scheduleWriteOperation( ArgumentMatchers.eq("classic-group-heartbeat"), ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)), + ArgumentMatchers.eq(Duration.ofMillis(5000)), ArgumentMatchers.any() )).thenReturn(FutureUtils.failedFuture( new CoordinatorLoadInProgressException(null) @@ -740,9 +742,10 @@ public void testHeartbeatCoordinatorException() throws Exception { service.startup(() -> 1); - when(runtime.scheduleReadOperation( + when(runtime.scheduleWriteOperation( ArgumentMatchers.eq("classic-group-heartbeat"), ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)), + ArgumentMatchers.eq(Duration.ofMillis(5000)), ArgumentMatchers.any() )).thenReturn(FutureUtils.failedFuture( new RebalanceInProgressException() 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 058cdf206db6..41cd5405abb9 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 @@ -116,6 +116,7 @@ import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment; import static org.apache.kafka.coordinator.group.GroupMetadataManager.appendGroupMetadataErrorToResponseError; import static org.apache.kafka.coordinator.group.GroupMetadataManager.classicGroupJoinKey; +import static org.apache.kafka.coordinator.group.GroupMetadataManager.consumerGroupJoinKey; import static org.apache.kafka.coordinator.group.GroupMetadataManager.consumerGroupRebalanceTimeoutKey; import static org.apache.kafka.coordinator.group.GroupMetadataManager.consumerGroupSessionTimeoutKey; import static org.apache.kafka.coordinator.group.GroupMetadataManager.EMPTY_RESULT; @@ -7452,7 +7453,7 @@ public void testStaticMemberHeartbeatLeaderWithInvalidMemberId() throws Exceptio .setMemberId(rebalanceResult.leaderId) .setGenerationId(rebalanceResult.generationId); - HeartbeatResponseData validHeartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest); + HeartbeatResponseData validHeartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest).response(); assertEquals(Errors.NONE.code(), validHeartbeatResponse.errorCode()); assertThrows(FencedInstanceIdException.class, () -> context.sendClassicGroupHeartbeat( @@ -7513,7 +7514,7 @@ public void testHeartbeatEmptyGroup() { .setMemberId("member-id") .setGenerationId(0); - HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest); + HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest).response(); assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), heartbeatResponse.errorCode()); } @@ -7560,7 +7561,7 @@ public void testHeartbeatDuringPreparingRebalance() throws Exception { .setGroupId("group-id") .setMemberId(memberId) .setGenerationId(0) - ); + ).response(); assertEquals(Errors.REBALANCE_IN_PROGRESS.code(), heartbeatResponse.errorCode()); } @@ -7586,7 +7587,7 @@ public void testHeartbeatDuringCompletingRebalance() throws Exception { .setGroupId("group-id") .setMemberId(leaderJoinResponse.memberId()) .setGenerationId(leaderJoinResponse.generationId()) - ); + ).response(); assertEquals(new HeartbeatResponseData(), heartbeatResponse); } @@ -7616,7 +7617,7 @@ public void testValidHeartbeat() throws Exception { .setGroupId("group-id") .setMemberId(leaderJoinResponse.memberId()) .setGenerationId(leaderJoinResponse.generationId()) - ); + ).response(); assertEquals(Errors.NONE.code(), heartbeatResponse.errorCode()); } @@ -7653,12 +7654,12 @@ public void testClassicGroupMemberHeartbeatMaintainsSession() throws Exception { .setMemberId(leaderJoinResponse.memberId()) .setGenerationId(leaderJoinResponse.generationId()); - HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest); + HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest).response(); assertEquals(Errors.NONE.code(), heartbeatResponse.errorCode()); GroupMetadataManagerTestContext.assertNoOrEmptyResult(context.sleep(2500)); - heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest); + heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest).response(); assertEquals(Errors.NONE.code(), heartbeatResponse.errorCode()); } @@ -7693,7 +7694,7 @@ public void testClassicGroupMemberSessionTimeoutDuringRebalance() throws Excepti .setMemberId(leaderJoinResponse.memberId()) .setGenerationId(leaderJoinResponse.generationId()); - HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest); + HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest).response(); assertEquals(Errors.REBALANCE_IN_PROGRESS.code(), heartbeatResponse.errorCode()); // Advance clock by first member's session timeout. @@ -7771,7 +7772,7 @@ public void testRebalanceCompletesBeforeMemberJoins() throws Exception { for (int i = 0; i < 2; i++) { GroupMetadataManagerTestContext.assertNoOrEmptyResult(context.sleep(2500)); - HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(firstMemberHeartbeatRequest); + HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(firstMemberHeartbeatRequest).response(); assertEquals(Errors.REBALANCE_IN_PROGRESS.code(), heartbeatResponse.errorCode()); } @@ -7813,7 +7814,7 @@ public void testRebalanceCompletesBeforeMemberJoins() throws Exception { firstMemberHeartbeatRequest .setMemberId(otherMemberId) .setGenerationId(2) - ); + ).response(); assertEquals(expectedError.code(), heartbeatResponse.errorCode()); } @@ -7855,7 +7856,7 @@ public void testRebalanceCompletesBeforeMemberJoins() throws Exception { firstMemberHeartbeatRequest .setMemberId(otherMemberId) .setGenerationId(3) - ); + ).response(); assertEquals(Errors.NONE.code(), heartbeatResponse.errorCode()); } @@ -7872,7 +7873,7 @@ public void testSyncGroupEmptyAssignment() throws Exception { .setGroupId("group-id") .setMemberId(leaderJoinResponse.memberId()) .setGenerationId(leaderJoinResponse.generationId()) - ); + ).response(); assertEquals(Errors.NONE.code(), heartbeatResponse.errorCode()); } @@ -7942,7 +7943,7 @@ public void testSecondMemberPartiallyJoinAndTimeout() throws Exception { for (int i = 0; i < 2; i++) { GroupMetadataManagerTestContext.assertNoOrEmptyResult(context.sleep(2500)); - HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest); + HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest).response(); assertEquals(Errors.NONE.code(), heartbeatResponse.errorCode()); } @@ -8203,7 +8204,7 @@ public void testHeartbeatDuringRebalanceCausesRebalanceInProgress() throws Excep .setMemberId(leaderJoinResponse.memberId()) .setGenerationId(leaderJoinResponse.generationId()); - HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest); + HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest).response(); assertEquals(Errors.REBALANCE_IN_PROGRESS.code(), heartbeatResponse.errorCode()); } @@ -11726,6 +11727,18 @@ public void testReconciliationInJoiningConsumerGroupWithEagerProtocol() throws E .setTopicPartitions(Collections.emptyList()) ); + // Member 1 heartbeats to be notified to rejoin. + assertEquals( + Errors.REBALANCE_IN_PROGRESS.code(), + context.sendClassicGroupHeartbeat( + new HeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId1) + .setGenerationId(joinResponse1.generationId()) + ).response().errorCode() + ); + context.assertJoinTimeout(groupId, memberId1, 500); + // Member 1 rejoins to transition from UNRELEASED_PARTITIONS to STABLE. GroupMetadataManagerTestContext.JoinResult joinResult2 = context.sendClassicGroupJoin(request); ConsumerGroupMember expectedMember2 = new ConsumerGroupMember.Builder(expectedMember1) @@ -11743,6 +11756,7 @@ public void testReconciliationInJoiningConsumerGroupWithEagerProtocol() throws E assertEquals(expectedMember2.state(), group.getOrMaybeCreateMember(memberId1, false).state()); joinResult2.appendFuture.complete(null); + context.assertNoJoinTimeout(groupId, memberId1); JoinGroupResponseData joinResponse2 = joinResult2.joinFuture.get(); assertEquals( new JoinGroupResponseData() @@ -11943,6 +11957,18 @@ public void testReconciliationInJoiningConsumerGroupWithCooperativeProtocol() th Collections.singletonList(new TopicPartition(fooTopicName, 0)) ); + // Member 1 heartbeats to be notified to rejoin. + assertEquals( + Errors.REBALANCE_IN_PROGRESS.code(), + context.sendClassicGroupHeartbeat( + new HeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId1) + .setGenerationId(joinResponse1.generationId()) + ).response().errorCode() + ); + context.assertJoinTimeout(groupId, memberId1, 500); + // Member 1 rejoins to transition from UNREVOKED_PARTITIONS to UNRELEASED_PARTITIONS. JoinGroupRequestData request2 = new GroupMetadataManagerTestContext.JoinGroupRequestBuilder() .withGroupId(groupId) @@ -11983,6 +12009,7 @@ public void testReconciliationInJoiningConsumerGroupWithCooperativeProtocol() th assertEquals(expectedMember2.state(), group.getOrMaybeCreateMember(memberId1, false).state()); joinResult2.appendFuture.complete(null); + context.assertNoJoinTimeout(groupId, memberId1); JoinGroupResponseData joinResponse2 = joinResult2.joinFuture.get(); assertEquals( new JoinGroupResponseData() @@ -12017,6 +12044,18 @@ public void testReconciliationInJoiningConsumerGroupWithCooperativeProtocol() th .setTopicPartitions(Collections.emptyList()) ); + // Member 1 heartbeats to be notified to rejoin. + assertEquals( + Errors.REBALANCE_IN_PROGRESS.code(), + context.sendClassicGroupHeartbeat( + new HeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId1) + .setGenerationId(joinResponse2.generationId()) + ).response().errorCode() + ); + context.assertJoinTimeout(groupId, memberId1, 500); + // Member 1 rejoins to transition from UNRELEASED_PARTITIONS to STABLE. JoinGroupRequestData request3 = new GroupMetadataManagerTestContext.JoinGroupRequestBuilder() .withGroupId(groupId) @@ -12056,6 +12095,7 @@ public void testReconciliationInJoiningConsumerGroupWithCooperativeProtocol() th assertEquals(expectedMember3.state(), group.getOrMaybeCreateMember(memberId1, false).state()); joinResult3.appendFuture.complete(null); + context.assertNoJoinTimeout(groupId, memberId1); JoinGroupResponseData joinResponse3 = joinResult3.joinFuture.get(); assertEquals( new JoinGroupResponseData() @@ -12143,7 +12183,6 @@ public void testClassicGroupSyncToConsumerGroupWithAllConsumerProtocolVersions() // Consumer group with two members. // Member 1 uses the classic protocol and member 2 uses the consumer protocol. GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withConsumerGroupMigrationPolicy(ConsumerGroupMigrationPolicy.DOWNGRADE) .withAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) @@ -12181,7 +12220,6 @@ public void testClassicGroupSyncToConsumerGroupWithUnknownMemberId() throws Exce // Consumer group with a member that doesn't use the classic protocol. GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withConsumerGroupMigrationPolicy(ConsumerGroupMigrationPolicy.DOWNGRADE) .withAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) .withMember(new ConsumerGroupMember.Builder(memberId) @@ -12231,7 +12269,6 @@ public void testClassicGroupSyncToConsumerGroupWithFencedInstanceId() throws Exc // Consumer group with a static member. GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withConsumerGroupMigrationPolicy(ConsumerGroupMigrationPolicy.DOWNGRADE) .withAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) .withMember(new ConsumerGroupMember.Builder(memberId) @@ -12261,7 +12298,7 @@ public void testClassicGroupSyncToConsumerGroupWithInconsistentGroupProtocol() t .setName("range") .setMetadata(Utils.toArray(ConsumerProtocol.serializeSubscription( new ConsumerPartitionAssignor.Subscription( - Arrays.asList("foo"), + Collections.singletonList("foo"), null, Collections.emptyList() ) @@ -12270,7 +12307,6 @@ public void testClassicGroupSyncToConsumerGroupWithInconsistentGroupProtocol() t // Consumer group with a member using the classic protocol. GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withConsumerGroupMigrationPolicy(ConsumerGroupMigrationPolicy.DOWNGRADE) .withAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) .withMember(new ConsumerGroupMember.Builder(memberId) @@ -12326,7 +12362,7 @@ public void testClassicGroupSyncToConsumerGroupWithIllegalGeneration() throws Ex .setName("range") .setMetadata(Utils.toArray(ConsumerProtocol.serializeSubscription( new ConsumerPartitionAssignor.Subscription( - Arrays.asList("foo"), + Collections.singletonList("foo"), null, Collections.emptyList() ) @@ -12335,7 +12371,6 @@ public void testClassicGroupSyncToConsumerGroupWithIllegalGeneration() throws Ex // Consumer group with a member using the classic protocol. GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withConsumerGroupMigrationPolicy(ConsumerGroupMigrationPolicy.DOWNGRADE) .withAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) .withMember(new ConsumerGroupMember.Builder(memberId) @@ -12369,7 +12404,7 @@ public void testClassicGroupSyncToConsumerGroupRebalanceInProgress() throws Exce .setName("range") .setMetadata(Utils.toArray(ConsumerProtocol.serializeSubscription( new ConsumerPartitionAssignor.Subscription( - Arrays.asList("foo"), + Collections.singletonList("foo"), null, Collections.emptyList() ) @@ -12379,10 +12414,10 @@ public void testClassicGroupSyncToConsumerGroupRebalanceInProgress() throws Exce // Consumer group with a member using the classic protocol. // The group epoch is greater than the member epoch. GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withConsumerGroupMigrationPolicy(ConsumerGroupMigrationPolicy.DOWNGRADE) .withAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) .withConsumerGroup(new ConsumerGroupBuilder(groupId, 11) .withMember(new ConsumerGroupMember.Builder(memberId) + .setRebalanceTimeoutMs(10000) .setClassicMemberMetadata( new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata() .setSessionTimeoutMs(5000) @@ -12401,6 +12436,369 @@ public void testClassicGroupSyncToConsumerGroupRebalanceInProgress() throws Exce .withProtocolName("range") .build()) ); + context.assertJoinTimeout(groupId, memberId, 10000); + } + + @Test + public void testClassicGroupHeartbeatToConsumerGroupMaintainsSession() throws Exception { + String groupId = "group-id"; + String memberId = Uuid.randomUuid().toString(); + int sessionTimeout = 5000; + + List protocols = Collections.singletonList( + new ConsumerGroupMemberMetadataValue.ClassicProtocol() + .setName("range") + .setMetadata(Utils.toArray(ConsumerProtocol.serializeSubscription( + new ConsumerPartitionAssignor.Subscription( + Collections.singletonList("foo"), + null, + Collections.emptyList() + ) + ))) + ); + + // Consumer group with a member using the classic protocol. + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) + .withMember(new ConsumerGroupMember.Builder(memberId) + .setClassicMemberMetadata( + new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata() + .setSessionTimeoutMs(sessionTimeout) + .setSupportedProtocols(protocols) + ) + .setMemberEpoch(10) + .build())) + .build(); + + // Heartbeat to schedule the session timeout. + HeartbeatRequestData request = new HeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setGenerationId(10); + context.sendClassicGroupHeartbeat(request); + context.assertSessionTimeout(groupId, memberId, sessionTimeout); + + // Advance clock by 1/2 of session timeout. + GroupMetadataManagerTestContext.assertNoOrEmptyResult(context.sleep(sessionTimeout / 2)); + + HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(request).response(); + assertEquals(Errors.NONE.code(), heartbeatResponse.errorCode()); + context.assertSessionTimeout(groupId, memberId, sessionTimeout); + + // Advance clock by 1/2 of session timeout. + GroupMetadataManagerTestContext.assertNoOrEmptyResult(context.sleep(sessionTimeout / 2)); + + heartbeatResponse = context.sendClassicGroupHeartbeat(request).response(); + assertEquals(Errors.NONE.code(), heartbeatResponse.errorCode()); + context.assertSessionTimeout(groupId, memberId, sessionTimeout); + } + + @Test + public void testClassicGroupHeartbeatToConsumerGroupRebalanceInProgress() throws Exception { + String groupId = "group-id"; + String memberId1 = Uuid.randomUuid().toString(); + String memberId2 = Uuid.randomUuid().toString(); + String memberId3 = Uuid.randomUuid().toString(); + Uuid fooTopicId = Uuid.randomUuid(); + Uuid barTopicId = Uuid.randomUuid(); + int sessionTimeout = 5000; + int rebalanceTimeout = 10000; + + List protocols = Collections.singletonList( + new ConsumerGroupMemberMetadataValue.ClassicProtocol() + .setName("range") + .setMetadata(Utils.toArray(ConsumerProtocol.serializeSubscription( + new ConsumerPartitionAssignor.Subscription( + Collections.singletonList("foo"), + null, + Collections.emptyList() + ) + ))) + ); + + // Member 1 has a member epoch smaller than the group epoch. + ConsumerGroupMember member1 = new ConsumerGroupMember.Builder(memberId1) + .setRebalanceTimeoutMs(rebalanceTimeout) + .setClassicMemberMetadata( + new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata() + .setSessionTimeoutMs(sessionTimeout) + .setSupportedProtocols(protocols) + ) + .setMemberEpoch(9) + .build(); + + // Member 2 has unrevoked partition. + ConsumerGroupMember member2 = new ConsumerGroupMember.Builder(memberId2) + .setState(MemberState.UNREVOKED_PARTITIONS) + .setRebalanceTimeoutMs(rebalanceTimeout) + .setPartitionsPendingRevocation(mkAssignment(mkTopicAssignment(fooTopicId, 0))) + .setClassicMemberMetadata( + new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata() + .setSessionTimeoutMs(sessionTimeout) + .setSupportedProtocols(protocols) + ) + .setMemberEpoch(10) + .build(); + + // Member 3 is in UNRELEASED_PARTITIONS and all the partitions in its target assignment are free. + ConsumerGroupMember member3 = new ConsumerGroupMember.Builder(memberId3) + .setState(MemberState.UNRELEASED_PARTITIONS) + .setRebalanceTimeoutMs(rebalanceTimeout) + .setAssignedPartitions(mkAssignment(mkTopicAssignment(barTopicId, 0))) + .setClassicMemberMetadata( + new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata() + .setSessionTimeoutMs(sessionTimeout) + .setSupportedProtocols(protocols) + ) + .setMemberEpoch(10) + .build(); + + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) + .withMember(member1) + .withMember(member2) + .withMember(member3) + .withAssignment(memberId3, mkAssignment(mkTopicAssignment(barTopicId, 0, 1, 2)))) + .build(); + + Arrays.asList(memberId1, memberId2, memberId3).forEach(memberId -> { + CoordinatorResult heartbeatResult = context.sendClassicGroupHeartbeat( + new HeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setGenerationId(memberId.equals(memberId1) ? 9 : 10) + ); + assertEquals(Collections.emptyList(), heartbeatResult.records()); + assertEquals(Errors.REBALANCE_IN_PROGRESS.code(), heartbeatResult.response().errorCode()); + context.assertSessionTimeout(groupId, memberId, sessionTimeout); + context.assertJoinTimeout(groupId, memberId, rebalanceTimeout); + }); + } + + @Test + public void testClassicGroupHeartbeatToConsumerWithUnknownMember() { + String groupId = "group-id"; + + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10)) + .build(); + + assertThrows(UnknownMemberIdException.class, () -> context.sendClassicGroupHeartbeat( + new HeartbeatRequestData() + .setGroupId(groupId) + .setMemberId("unknown-member-id") + .setGenerationId(10) + )); + + assertThrows(UnknownMemberIdException.class, () -> context.sendClassicGroupHeartbeat( + new HeartbeatRequestData() + .setGroupId(groupId) + .setMemberId("unknown-member-id") + .setGroupInstanceId("unknown-instance-id") + .setGenerationId(10) + )); + } + + @Test + public void testClassicGroupHeartbeatToConsumerWithFencedInstanceId() { + String groupId = "group-id"; + String memberId = "member-id"; + String instanceId = "instance-id"; + + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) + .withMember(new ConsumerGroupMember.Builder(memberId) + .setInstanceId(instanceId) + .setMemberEpoch(10) + .setClassicMemberMetadata( + new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata() + .setSessionTimeoutMs(5000) + .setSupportedProtocols(Collections.emptyList()) + ) + .build())) + .build(); + + assertThrows(FencedInstanceIdException.class, () -> context.sendClassicGroupHeartbeat( + new HeartbeatRequestData() + .setGroupId(groupId) + .setMemberId("unknown-member-id") + .setGroupInstanceId(instanceId) + .setGenerationId(10) + )); + } + + @Test + public void testClassicGroupHeartbeatToConsumerWithIllegalGenerationId() { + String groupId = "group-id"; + String memberId = "member-id"; + + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) + .withMember(new ConsumerGroupMember.Builder(memberId) + .setMemberEpoch(10) + .setClassicMemberMetadata( + new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata() + .setSessionTimeoutMs(5000) + .setSupportedProtocols(Collections.emptyList()) + ) + .build())) + .build(); + + assertThrows(IllegalGenerationException.class, () -> context.sendClassicGroupHeartbeat( + new HeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setGenerationId(9) + )); + } + + @Test + public void testClassicGroupHeartbeatToConsumerWithMemberNotUsingClassicProtocol() { + String groupId = "group-id"; + String memberId = "member-id"; + + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) + .withMember(new ConsumerGroupMember.Builder(memberId) + .setMemberEpoch(10) + .build())) + .build(); + + assertThrows(UnknownMemberIdException.class, () -> context.sendClassicGroupHeartbeat( + new HeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setGenerationId(10) + )); + } + + @Test + public void testConsumerGroupMemberUsingClassicProtocolFencedWhenSessionTimeout() { + String groupId = "group-id"; + String memberId = Uuid.randomUuid().toString(); + int sessionTimeout = 5000; + + List protocols = Collections.singletonList( + new ConsumerGroupMemberMetadataValue.ClassicProtocol() + .setName("range") + .setMetadata(Utils.toArray(ConsumerProtocol.serializeSubscription( + new ConsumerPartitionAssignor.Subscription( + Collections.singletonList("foo"), + null, + Collections.emptyList() + ) + ))) + ); + + // Consumer group with a member using the classic protocol. + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) + .withMember(new ConsumerGroupMember.Builder(memberId) + .setClassicMemberMetadata( + new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata() + .setSessionTimeoutMs(sessionTimeout) + .setSupportedProtocols(protocols) + ) + .setMemberEpoch(10) + .build())) + .build(); + + // Heartbeat to schedule the session timeout. + HeartbeatRequestData request = new HeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setGenerationId(10); + context.sendClassicGroupHeartbeat(request); + context.assertSessionTimeout(groupId, memberId, sessionTimeout); + + // Advance clock by session timeout + 1. + List> timeouts = context.sleep(sessionTimeout + 1); + + // The member is fenced from the group. + assertEquals(1, timeouts.size()); + MockCoordinatorTimer.ExpiredTimeout timeout = timeouts.get(0); + assertEquals(consumerGroupSessionTimeoutKey(groupId, memberId), timeout.key); + assertRecordsEquals( + Arrays.asList( + // The member is removed. + CoordinatorRecordHelpers.newCurrentAssignmentTombstoneRecord(groupId, memberId), + CoordinatorRecordHelpers.newTargetAssignmentTombstoneRecord(groupId, memberId), + CoordinatorRecordHelpers.newMemberSubscriptionTombstoneRecord(groupId, memberId), + + // The group epoch is bumped. + CoordinatorRecordHelpers.newGroupEpochRecord(groupId, 11) + ), + timeout.result.records() + ); + } + + @Test + public void testConsumerGroupMemberUsingClassicProtocolFencedWhenJoinTimeout() { + String groupId = "group-id"; + String memberId = Uuid.randomUuid().toString(); + int rebalanceTimeout = 500; + + List protocols = Collections.singletonList( + new ConsumerGroupMemberMetadataValue.ClassicProtocol() + .setName("range") + .setMetadata(Utils.toArray(ConsumerProtocol.serializeSubscription( + new ConsumerPartitionAssignor.Subscription( + Collections.singletonList("foo"), + null, + Collections.emptyList() + ) + ))) + ); + + // Consumer group with a member using the classic protocol whose member epoch is smaller than the group epoch. + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) + .withMember(new ConsumerGroupMember.Builder(memberId) + .setRebalanceTimeoutMs(rebalanceTimeout) + .setClassicMemberMetadata( + new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata() + .setSessionTimeoutMs(5000) + .setSupportedProtocols(protocols) + ) + .setMemberEpoch(9) + .build())) + .build(); + + // Heartbeat to schedule the join timeout. + HeartbeatRequestData request = new HeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setGenerationId(9); + assertEquals( + Errors.REBALANCE_IN_PROGRESS.code(), + context.sendClassicGroupHeartbeat(request).response().errorCode() + ); + context.assertSessionTimeout(groupId, memberId, 5000); + context.assertJoinTimeout(groupId, memberId, rebalanceTimeout); + + // Advance clock by rebalance timeout + 1. + List> timeouts = context.sleep(rebalanceTimeout + 1); + + // The member is fenced from the group. + assertEquals(1, timeouts.size()); + MockCoordinatorTimer.ExpiredTimeout timeout = timeouts.get(0); + assertEquals(consumerGroupJoinKey(groupId, memberId), timeout.key); + assertRecordsEquals( + Arrays.asList( + // The member is removed. + CoordinatorRecordHelpers.newCurrentAssignmentTombstoneRecord(groupId, memberId), + CoordinatorRecordHelpers.newTargetAssignmentTombstoneRecord(groupId, memberId), + CoordinatorRecordHelpers.newMemberSubscriptionTombstoneRecord(groupId, memberId), + + // The group epoch is bumped. + CoordinatorRecordHelpers.newGroupEpochRecord(groupId, 11) + ), + timeout.result.records() + ); } private static void checkJoinGroupResponse( diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java index ce57498b2120..bd7a30c54177 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java @@ -89,6 +89,7 @@ import static org.apache.kafka.coordinator.group.Assertions.assertSyncGroupResponseEquals; import static org.apache.kafka.coordinator.group.GroupMetadataManager.EMPTY_RESULT; import static org.apache.kafka.coordinator.group.GroupMetadataManager.classicGroupHeartbeatKey; +import static org.apache.kafka.coordinator.group.GroupMetadataManager.consumerGroupJoinKey; import static org.apache.kafka.coordinator.group.GroupMetadataManager.consumerGroupRebalanceTimeoutKey; import static org.apache.kafka.coordinator.group.GroupMetadataManager.consumerGroupSessionTimeoutKey; import static org.apache.kafka.coordinator.group.GroupMetadataManager.consumerGroupSyncKey; @@ -619,6 +620,27 @@ public void assertNoRebalanceTimeout( assertNull(timeout); } + public MockCoordinatorTimer.ScheduledTimeout assertJoinTimeout( + String groupId, + String memberId, + long delayMs + ) { + MockCoordinatorTimer.ScheduledTimeout timeout = + timer.timeout(consumerGroupJoinKey(groupId, memberId)); + assertNotNull(timeout); + assertEquals(time.milliseconds() + delayMs, timeout.deadlineMs); + return timeout; + } + + public void assertNoJoinTimeout( + String groupId, + String memberId + ) { + MockCoordinatorTimer.ScheduledTimeout timeout = + timer.timeout(consumerGroupJoinKey(groupId, memberId)); + assertNull(timeout); + } + public MockCoordinatorTimer.ScheduledTimeout assertSyncTimeout( String groupId, String memberId, @@ -1112,7 +1134,7 @@ public void verifySessionExpiration(ClassicGroup group, int timeoutMs) { assertEquals(0, group.size()); } - public HeartbeatResponseData sendClassicGroupHeartbeat( + public CoordinatorResult sendClassicGroupHeartbeat( HeartbeatRequestData request ) { RequestContext context = new RequestContext( @@ -1164,7 +1186,7 @@ public void verifyHeartbeat( if (expectedError == Errors.UNKNOWN_MEMBER_ID) { assertThrows(UnknownMemberIdException.class, () -> sendClassicGroupHeartbeat(request)); } else { - HeartbeatResponseData response = sendClassicGroupHeartbeat(request); + HeartbeatResponseData response = sendClassicGroupHeartbeat(request).response(); assertEquals(expectedError.code(), response.errorCode()); } } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/MockCoordinatorTimer.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/MockCoordinatorTimer.java index 439da4bbf473..b4942c06dc55 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/MockCoordinatorTimer.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/MockCoordinatorTimer.java @@ -130,6 +130,19 @@ public void schedule( schedule(key, delay, unit, retry, 500L, operation); } + @Override + public void scheduleIfAbsent( + String key, + long delay, + TimeUnit unit, + boolean retry, + TimeoutOperation operation + ) { + if (!timeoutMap.containsKey(key)) { + schedule(key, delay, unit, retry, 500L, operation); + } + } + /** * Cancels a timeout. */ diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java index 870eb9d7ec86..4ae14c25439b 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java @@ -368,6 +368,42 @@ public void testDeletingMemberRemovesPartitionEpoch() { assertEquals(-1, consumerGroup.currentPartitionEpoch(fooTopicId, 9)); } + @Test + public void testWaitingOnUnreleasedPartition() { + Uuid fooTopicId = Uuid.randomUuid(); + Uuid barTopicId = Uuid.randomUuid(); + Uuid zarTopicId = Uuid.randomUuid(); + String memberId1 = Uuid.randomUuid().toString(); + String memberId2 = Uuid.randomUuid().toString(); + + ConsumerGroup consumerGroup = createConsumerGroup("foo"); + consumerGroup.updateTargetAssignment(memberId1, new Assignment(mkAssignment( + mkTopicAssignment(fooTopicId, 1, 2, 3), + mkTopicAssignment(zarTopicId, 7, 8, 9) + ))); + + ConsumerGroupMember member1 = new ConsumerGroupMember.Builder(memberId1) + .setMemberEpoch(10) + .setState(MemberState.UNRELEASED_PARTITIONS) + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 1, 2, 3))) + .setPartitionsPendingRevocation(mkAssignment( + mkTopicAssignment(barTopicId, 4, 5, 6))) + .build(); + consumerGroup.updateMember(member1); + + assertFalse(consumerGroup.waitingOnUnreleasedPartition(member1)); + + ConsumerGroupMember member2 = new ConsumerGroupMember.Builder(memberId2) + .setMemberEpoch(10) + .setPartitionsPendingRevocation(mkAssignment( + mkTopicAssignment(zarTopicId, 7))) + .build(); + consumerGroup.updateMember(member2); + + assertTrue(consumerGroup.waitingOnUnreleasedPartition(member1)); + } + @Test public void testGroupState() { Uuid fooTopicId = Uuid.randomUuid(); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntimeTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntimeTest.java index d874ceae95f0..a8cc200b3592 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntimeTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntimeTest.java @@ -2505,6 +2505,57 @@ public void testNonRetryableTimer() throws InterruptedException { assertEquals(0, ctx.timer.size()); } + @Test + public void testTimerScheduleIfAbsent() throws InterruptedException { + MockTimer timer = new MockTimer(); + CoordinatorRuntime runtime = + new CoordinatorRuntime.Builder() + .withTime(timer.time()) + .withTimer(timer) + .withDefaultWriteTimeOut(DEFAULT_WRITE_TIMEOUT) + .withLoader(new MockCoordinatorLoader()) + .withEventProcessor(new DirectEventProcessor()) + .withPartitionWriter(new MockPartitionWriter()) + .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) + .withCoordinatorRuntimeMetrics(mock(GroupCoordinatorRuntimeMetrics.class)) + .withCoordinatorMetrics(mock(GroupCoordinatorMetrics.class)) + .withSerializer(new StringSerializer()) + .build(); + + // Loads the coordinator. + runtime.scheduleLoadOperation(TP, 10); + + // Check initial state. + CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); + assertEquals(0, ctx.timer.size()); + + // Timer #1. + AtomicInteger cnt = new AtomicInteger(0); + ctx.timer.scheduleIfAbsent("timer-1", 10, TimeUnit.MILLISECONDS, false, () -> { + cnt.incrementAndGet(); + throw new KafkaException("error"); + }); + + // The coordinator timer should have one pending task. + assertEquals(1, ctx.timer.size()); + + // Advance half of the time to fire the pending timer. + timer.advanceClock(10 / 2); + + // Reschedule timer #1. Since the timer already exists, the timeout shouldn't be refreshed. + ctx.timer.scheduleIfAbsent("timer-1", 10, TimeUnit.MILLISECONDS, false, () -> { + cnt.incrementAndGet(); + throw new KafkaException("error"); + }); + + // Advance the time to fire the pending timer. + timer.advanceClock(10 / 2 + 1); + + // The timer should have been called and the timer should have no pending tasks. + assertEquals(1, cnt.get()); + assertEquals(0, ctx.timer.size()); + } + @Test public void testStateChanges() throws Exception { MockTimer timer = new MockTimer(); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessorTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessorTest.java index 313e46dfc047..0f2801daec3b 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessorTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessorTest.java @@ -61,8 +61,9 @@ public DelayEventAccumulator(Time time, long takeDelayMs) { @Override public CoordinatorEvent take() { + CoordinatorEvent event = super.take(); time.sleep(takeDelayMs); - return super.take(); + return event; } } @@ -475,9 +476,9 @@ public void testRecordThreadIdleRatio() throws Exception { doAnswer(invocation -> { long threadIdleTime = idleTimeCaptured.getValue(); assertEquals(100, threadIdleTime); - synchronized (recordedIdleTimesMs) { - recordedIdleTimesMs.add(threadIdleTime); - } + + // No synchronization required as the test uses a single event processor thread. + recordedIdleTimesMs.add(threadIdleTime); return null; }).when(mockRuntimeMetrics).recordThreadIdleTime(idleTimeCaptured.capture()); diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java index 8b4fed8375f6..17d4e683978b 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java @@ -190,4 +190,9 @@ public class ServerLogConfigs { public static final long LOG_INITIAL_TASK_DELAY_MS_DEFAULT = 30 * 1000L; public static final String LOG_INITIAL_TASK_DELAY_MS_DOC = "The initial task delay in millisecond when initializing " + "tasks in LogManager. This should be used for testing only."; + + public static final String LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG = LOG_PREFIX + "dir.failure.timeout.ms"; + public static final Long LOG_DIR_FAILURE_TIMEOUT_MS_DEFAULT = 30000L; + public static final String LOG_DIR_FAILURE_TIMEOUT_MS_DOC = "If the broker is unable to successfully communicate to the controller that some log " + + "directory has failed for longer than this time, the broker will fail and shut down."; } diff --git a/server-common/src/main/java/org/apache/kafka/server/util/Csv.java b/server-common/src/main/java/org/apache/kafka/server/util/Csv.java index f164dd114146..61f143bdc3f1 100644 --- a/server-common/src/main/java/org/apache/kafka/server/util/Csv.java +++ b/server-common/src/main/java/org/apache/kafka/server/util/Csv.java @@ -16,8 +16,12 @@ */ package org.apache.kafka.server.util; +import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; public class Csv { @@ -38,4 +42,16 @@ public static Map parseCsvMap(String str) { } return map; } + + /** + * Parse a comma separated string into a sequence of strings. + * Whitespace surrounding the comma will be removed. + */ + public static List parseCsvList(String csvList) { + if (csvList == null || csvList.isEmpty()) { + return Collections.emptyList(); + } else { + return Stream.of(csvList.split("\\s*,\\s*")).filter(v -> !v.isEmpty()).collect(Collectors.toList()); + } + } } diff --git a/server-common/src/test/java/org/apache/kafka/server/util/CsvTest.java b/server-common/src/test/java/org/apache/kafka/server/util/CsvTest.java index 46d54b42c2bb..65334a607b23 100644 --- a/server-common/src/test/java/org/apache/kafka/server/util/CsvTest.java +++ b/server-common/src/test/java/org/apache/kafka/server/util/CsvTest.java @@ -18,21 +18,19 @@ import org.junit.jupiter.api.Test; +import java.util.Arrays; import java.util.Collections; +import java.util.List; import java.util.Map; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNotNull; public class CsvTest { @Test public void testCsvMap() { - String emptyString = ""; - Map emptyMap = Csv.parseCsvMap(emptyString); - Map emptyStringMap = Collections.emptyMap(); - assertNotNull(emptyMap); - assertEquals(emptyStringMap, emptyStringMap); + Map emptyMap = Csv.parseCsvMap(""); + assertEquals(Collections.emptyMap(), emptyMap); String kvPairsIpV6 = "a:b:c:v,a:b:c:v"; Map ipv6Map = Csv.parseCsvMap(kvPairsIpV6); @@ -60,4 +58,16 @@ public void testCsvMap() { assertEquals("value", entry.getValue()); } } + + @Test + public void testCsvList() { + List emptyList = Csv.parseCsvList(""); + assertEquals(Collections.emptyList(), emptyList); + + List emptyListFromNullString = Csv.parseCsvList(null); + assertEquals(Collections.emptyList(), emptyListFromNullString); + + List csvList = Csv.parseCsvList("a,b ,c, d,,e,"); + assertEquals(Arrays.asList("a", "b", "c", "d", "e"), csvList); + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/TaskInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/TaskInfo.java index 871181363f80..22217b3bb255 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/TaskInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/TaskInfo.java @@ -17,9 +17,7 @@ package org.apache.kafka.streams.processor.assignment; -import java.util.Map; import java.util.Set; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.streams.processor.TaskId; /** @@ -52,27 +50,7 @@ public interface TaskInfo { /** * - * @return The topic partitions in use by this task. + * @return the set of topic partitions in use for this task. */ Set topicPartitions(); - - /** - * - * @return the set of source topic partitions. This set will include both changelog and non-changelog - * topic partitions. - */ - Set sourceTopicPartitions(); - - /** - * - * @return the set of changelog topic partitions. This set will include both source and non-source - * topic partitions. - */ - Set changelogTopicPartitions(); - - /** - * - * @return the mapping of {@code TopicPartition} to set of rack ids that this partition resides on. - */ - Map> partitionToRackIds(); -} \ No newline at end of file +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java index 82a10615293e..d7c15f9fd0d6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java @@ -41,6 +41,7 @@ import org.apache.kafka.streams.processor.assignment.TaskInfo; import org.apache.kafka.streams.processor.assignment.ProcessId; import org.apache.kafka.streams.processor.assignment.TaskAssignor.TaskAssignment; +import org.apache.kafka.streams.processor.assignment.TaskTopicPartition; import org.apache.kafka.streams.processor.internals.assignment.ApplicationStateImpl; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.TopicsInfo; import org.apache.kafka.streams.processor.internals.TopologyMetadata.Subtopology; @@ -51,6 +52,7 @@ import org.apache.kafka.streams.processor.internals.assignment.AssignorError; import org.apache.kafka.streams.processor.internals.assignment.ClientState; import org.apache.kafka.streams.processor.internals.assignment.CopartitionedTopicsEnforcer; +import org.apache.kafka.streams.processor.internals.assignment.DefaultTaskTopicPartition; import org.apache.kafka.streams.processor.internals.assignment.FallbackPriorTaskAssignor; import org.apache.kafka.streams.processor.internals.assignment.RackAwareTaskAssignor; import org.apache.kafka.streams.processor.internals.assignment.RackUtils; @@ -513,50 +515,45 @@ private ApplicationState buildApplicationState(final TopologyMetadata topologyMe + "tasks for source topics vs changelog topics."); } - final Set sourceTopicPartitions = new HashSet<>(); - final Set nonSourceChangelogTopicPartitions = new HashSet<>(); - for (final Map.Entry> entry : sourcePartitionsForTask.entrySet()) { - final TaskId taskId = entry.getKey(); - final Set taskSourcePartitions = entry.getValue(); - final Set taskChangelogPartitions = changelogPartitionsForTask.get(taskId); - final Set taskNonSourceChangelogPartitions = new HashSet<>(taskChangelogPartitions); - taskNonSourceChangelogPartitions.removeAll(taskSourcePartitions); - - sourceTopicPartitions.addAll(taskSourcePartitions); - nonSourceChangelogTopicPartitions.addAll(taskNonSourceChangelogPartitions); - } + final Set logicalTaskIds = unmodifiableSet(sourcePartitionsForTask.keySet()); + final Set allTopicPartitions = new HashSet<>(); + final Map> topicPartitionsForTask = new HashMap<>(); + logicalTaskIds.forEach(taskId -> { + final Set topicPartitions = new HashSet<>(); + + for (final TopicPartition topicPartition : sourcePartitionsForTask.get(taskId)) { + final boolean isSource = true; + final boolean isChangelog = changelogPartitionsForTask.get(taskId).contains(topicPartition); + final DefaultTaskTopicPartition racklessTopicPartition = new DefaultTaskTopicPartition( + topicPartition, isSource, isChangelog, null); + allTopicPartitions.add(racklessTopicPartition); + topicPartitions.add(racklessTopicPartition); + } - final Map> racksForSourcePartitions = RackUtils.getRacksForTopicPartition( - cluster, internalTopicManager, sourceTopicPartitions, false); - final Map> racksForChangelogPartitions = RackUtils.getRacksForTopicPartition( - cluster, internalTopicManager, nonSourceChangelogTopicPartitions, true); + for (final TopicPartition topicPartition : changelogPartitionsForTask.get(taskId)) { + final boolean isSource = sourcePartitionsForTask.get(taskId).contains(topicPartition); + final boolean isChangelog = true; + final DefaultTaskTopicPartition racklessTopicPartition = new DefaultTaskTopicPartition( + topicPartition, isSource, isChangelog, null); + allTopicPartitions.add(racklessTopicPartition); + topicPartitions.add(racklessTopicPartition); + } + + topicPartitionsForTask.put(taskId, topicPartitions); + }); + + RackUtils.annotateTopicPartitionsWithRackInfo(cluster, internalTopicManager, allTopicPartitions); - final Set logicalTaskIds = unmodifiableSet(sourcePartitionsForTask.keySet()); final Set logicalTasks = logicalTaskIds.stream().map(taskId -> { final Set stateStoreNames = topologyMetadata .stateStoreNameToSourceTopicsForTopology(taskId.topologyName()) .keySet(); - final Set sourcePartitions = sourcePartitionsForTask.get(taskId); - final Set changelogPartitions = changelogPartitionsForTask.get(taskId); - final Map> racksForTaskPartition = new HashMap<>(); - sourcePartitions.forEach(topicPartition -> { - racksForTaskPartition.put(topicPartition, racksForSourcePartitions.get(topicPartition)); - }); - changelogPartitions.forEach(topicPartition -> { - if (racksForSourcePartitions.containsKey(topicPartition)) { - racksForTaskPartition.put(topicPartition, racksForSourcePartitions.get(topicPartition)); - } else { - racksForTaskPartition.put(topicPartition, racksForChangelogPartitions.get(topicPartition)); - } - }); - + final Set topicPartitions = topicPartitionsForTask.get(taskId); return new DefaultTaskInfo( taskId, !stateStoreNames.isEmpty(), - racksForTaskPartition, stateStoreNames, - sourcePartitions, - changelogPartitions + topicPartitions ); }).collect(Collectors.toSet()); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultTaskInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultTaskInfo.java index c0212db862af..14e53440f4bf 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultTaskInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultTaskInfo.java @@ -16,36 +16,28 @@ */ package org.apache.kafka.streams.processor.internals.assignment; -import static java.util.Collections.unmodifiableMap; import static java.util.Collections.unmodifiableSet; -import java.util.Map; import java.util.Set; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.assignment.TaskInfo; +import org.apache.kafka.streams.processor.assignment.TaskTopicPartition; public class DefaultTaskInfo implements TaskInfo { private final TaskId id; private final boolean isStateful; - private final Map> partitionToRackIds; private final Set stateStoreNames; - private final Set sourceTopicPartitions; - private final Set changelogTopicPartitions; + private final Set topicPartitions; public DefaultTaskInfo(final TaskId id, final boolean isStateful, - final Map> partitionToRackIds, final Set stateStoreNames, - final Set sourceTopicPartitions, - final Set changelogTopicPartitions) { + final Set topicPartitions) { this.id = id; - this.partitionToRackIds = unmodifiableMap(partitionToRackIds); this.isStateful = isStateful; this.stateStoreNames = unmodifiableSet(stateStoreNames); - this.sourceTopicPartitions = unmodifiableSet(sourceTopicPartitions); - this.changelogTopicPartitions = unmodifiableSet(changelogTopicPartitions); + this.topicPartitions = unmodifiableSet(topicPartitions); } @Override @@ -64,17 +56,7 @@ public Set stateStoreNames() { } @Override - public Set sourceTopicPartitions() { - return sourceTopicPartitions; - } - - @Override - public Set changelogTopicPartitions() { - return changelogTopicPartitions; - } - - @Override - public Map> partitionToRackIds() { - return partitionToRackIds; + public Set topicPartitions() { + return topicPartitions; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultTaskTopicPartition.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultTaskTopicPartition.java index 815aa1ff64c9..1c0a640d9c4c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultTaskTopicPartition.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultTaskTopicPartition.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.processor.internals.assignment; +import java.util.Objects; import java.util.Optional; import java.util.Set; import org.apache.kafka.common.TopicPartition; @@ -35,7 +36,9 @@ public class DefaultTaskTopicPartition implements TaskTopicPartition { private final TopicPartition topicPartition; private final boolean isSourceTopic; private final boolean isChangelogTopic; - private final Optional> rackIds; + + private Optional> rackIds; + public DefaultTaskTopicPartition(final TopicPartition topicPartition, final boolean isSourceTopic, @@ -66,4 +69,31 @@ public boolean isChangelog() { public Optional> rackIds() { return rackIds; } + + @Override + public int hashCode() { + int result = topicPartition.hashCode(); + result = 31 * result + Objects.hashCode(isSourceTopic); + result = 31 * result + Objects.hashCode(isChangelogTopic); + return result; + } + + @Override + public boolean equals(final Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + final TaskTopicPartition other = (TaskTopicPartition) obj; + return topicPartition.equals(other.topicPartition()) && + isSourceTopic == other.isSource() && + isChangelogTopic == other.isChangelog() && + rackIds.equals(other.rackIds()); + } + + public void annotateWithRackIds(final Set rackIds) { + this.rackIds = Optional.of(rackIds); + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/RackUtils.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/RackUtils.java index b3554c36b03b..f9961c555fa6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/RackUtils.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/RackUtils.java @@ -28,6 +28,7 @@ import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartitionInfo; +import org.apache.kafka.streams.processor.assignment.TaskTopicPartition; import org.apache.kafka.streams.processor.internals.InternalTopicManager; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,26 +39,37 @@ public final class RackUtils { private RackUtils() { } - public static Map> getRacksForTopicPartition(final Cluster cluster, - final InternalTopicManager internalTopicManager, - final Set topicPartitions, - final boolean isChangelog) { - final Set topicsToDescribe = new HashSet<>(); - if (isChangelog) { - topicsToDescribe.addAll(topicPartitions.stream().map(TopicPartition::topic).collect( - Collectors.toSet())); - } else { - topicsToDescribe.addAll(topicsWithMissingMetadata(cluster, topicPartitions)); - } + public static void annotateTopicPartitionsWithRackInfo(final Cluster cluster, + final InternalTopicManager internalTopicManager, + final Set topicPartitions) { + // First we add all the changelog topics to the set of topics to describe. + final Set topicsToDescribe = topicPartitions.stream() + .filter(DefaultTaskTopicPartition::isChangelog) + .map(topicPartition -> topicPartition.topicPartition().topic()) + .collect(Collectors.toSet()); - final Set topicsWithUpToDateMetadata = topicPartitions.stream() - .filter(partition -> !topicsToDescribe.contains(partition.topic())) + // Then we add the non changelog topics that we do not have full information about. + final Set nonChangelogTopics = topicPartitions.stream() + .filter(taskTopicPartition -> !taskTopicPartition.isChangelog()) + .map(TaskTopicPartition::topicPartition) .collect(Collectors.toSet()); - final Map> racksForTopicPartition = knownRacksForPartition( - cluster, topicsWithUpToDateMetadata); + topicsToDescribe.addAll(topicsWithMissingMetadata(cluster, nonChangelogTopics)); + // We can issue an RPC call to get up-to-date information about the topics that had rack + // information missing. final Map> freshTopicPartitionInfo = describeTopics(internalTopicManager, topicsToDescribe); + + // Finally we compute the list of topics that already have all rack information known. + final Set topicsWithUpToDateMetadata = topicPartitions.stream() + .map(TaskTopicPartition::topicPartition) + .filter(topicPartition -> !topicsToDescribe.contains(topicPartition.topic())) + .collect(Collectors.toSet()); + + // Lastly we compile the mapping of topic partition to rack ids by combining known data and + // information that we got from the earlier RPC call. + final Map> racksForTopicPartition = knownRacksForPartition( + cluster, topicsWithUpToDateMetadata); freshTopicPartitionInfo.forEach((topic, partitionInfos) -> { for (final TopicPartitionInfo partitionInfo : partitionInfos) { final int partition = partitionInfo.partition(); @@ -75,7 +87,14 @@ public static Map> getRacksForTopicPartition(final C } }); - return racksForTopicPartition; + for (final DefaultTaskTopicPartition topicPartition : topicPartitions) { + if (!racksForTopicPartition.containsKey(topicPartition.topicPartition())) { + continue; + } + + final Set racks = racksForTopicPartition.get(topicPartition.topicPartition()); + topicPartition.annotateWithRackIds(racks); + } } public static Set topicsWithMissingMetadata(final Cluster cluster, final Set topicPartitions) { 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 88b49cd21803..85a7fe9e6bea 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 @@ -90,6 +90,7 @@ public class MeteredKeyValueStore private Sensor prefixScanSensor; private Sensor flushSensor; private Sensor e2eLatencySensor; + protected Sensor iteratorDurationSensor; protected InternalProcessorContext context; private StreamsMetricsImpl streamsMetrics; private TaskId taskId; @@ -165,6 +166,7 @@ private void registerMetrics() { flushSensor = StateStoreMetrics.flushSensor(taskId.toString(), metricsScope, name(), streamsMetrics); deleteSensor = StateStoreMetrics.deleteSensor(taskId.toString(), metricsScope, name(), streamsMetrics); 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.get()); } @@ -486,7 +488,9 @@ public void close() { try { iter.close(); } finally { - sensor.record(time.nanoseconds() - startNs); + final long duration = time.nanoseconds() - startNs; + sensor.record(duration); + iteratorDurationSensor.record(duration); numOpenIterators.decrementAndGet(); } } @@ -532,7 +536,9 @@ public void close() { try { iter.close(); } finally { - sensor.record(time.nanoseconds() - startNs); + final long duration = time.nanoseconds() - startNs; + sensor.record(duration); + iteratorDurationSensor.record(duration); numOpenIterators.decrementAndGet(); } } 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 be695501cafd..4663ef5abbfe 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,6 +18,9 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; + +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.state.VersionedRecordIterator; import org.apache.kafka.streams.state.VersionedRecord; @@ -26,13 +29,21 @@ public class MeteredMultiVersionedKeyQueryIterator implements VersionedRecord private final VersionedRecordIterator iterator; private final Function, VersionedRecord> deserializeValue; private final AtomicInteger numOpenIterators; + private final Sensor sensor; + private final Time time; + private final long startNs; public MeteredMultiVersionedKeyQueryIterator(final VersionedRecordIterator iterator, + final Sensor sensor, + final Time time, final Function, VersionedRecord> deserializeValue, final AtomicInteger numOpenIterators) { this.iterator = iterator; this.deserializeValue = deserializeValue; this.numOpenIterators = numOpenIterators; + this.sensor = sensor; + this.time = time; + this.startNs = time.nanoseconds(); numOpenIterators.incrementAndGet(); } @@ -42,6 +53,7 @@ public void close() { try { iterator.close(); } finally { + sensor.record(time.nanoseconds() - startNs); numOpenIterators.decrementAndGet(); } } 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 4bcbb483a314..0cb9445a92c2 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 @@ -68,6 +68,7 @@ public class MeteredSessionStore private Sensor flushSensor; private Sensor removeSensor; private Sensor e2eLatencySensor; + private Sensor iteratorDurationSensor; private InternalProcessorContext context; private TaskId taskId; @@ -134,6 +135,7 @@ private void registerMetrics() { flushSensor = StateStoreMetrics.flushSensor(taskId.toString(), metricsScope, name(), streamsMetrics); removeSensor = StateStoreMetrics.removeSensor(taskId.toString(), metricsScope, name(), streamsMetrics); 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.get()); } @@ -250,6 +252,7 @@ public KeyValueIterator, V> fetch(final K key) { return new MeteredWindowedKeyValueIterator<>( wrapped().fetch(keyBytes(key)), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, @@ -263,6 +266,7 @@ public KeyValueIterator, V> backwardFetch(final K key) { return new MeteredWindowedKeyValueIterator<>( wrapped().backwardFetch(keyBytes(key)), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, @@ -277,6 +281,7 @@ public KeyValueIterator, V> fetch(final K keyFrom, return new MeteredWindowedKeyValueIterator<>( wrapped().fetch(keyBytes(keyFrom), keyBytes(keyTo)), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, @@ -290,6 +295,7 @@ public KeyValueIterator, V> backwardFetch(final K keyFrom, return new MeteredWindowedKeyValueIterator<>( wrapped().backwardFetch(keyBytes(keyFrom), keyBytes(keyTo)), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, @@ -310,6 +316,7 @@ public KeyValueIterator, V> findSessions(final K key, earliestSessionEndTime, latestSessionStartTime), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, @@ -330,6 +337,7 @@ public KeyValueIterator, V> backwardFindSessions(final K key, latestSessionStartTime ), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, @@ -352,6 +360,7 @@ public KeyValueIterator, V> findSessions(final K keyFrom, earliestSessionEndTime, latestSessionStartTime), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, @@ -365,6 +374,7 @@ public KeyValueIterator, V> findSessions(final long earliestSessionE return new MeteredWindowedKeyValueIterator<>( wrapped().findSessions(earliestSessionEndTime, latestSessionEndTime), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, @@ -387,6 +397,7 @@ public KeyValueIterator, V> backwardFindSessions(final K keyFrom, latestSessionStartTime ), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, @@ -458,6 +469,7 @@ private QueryResult runRangeQuery(final Query query, new MeteredWindowedKeyValueIterator<>( rawResult.getResult(), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, StoreQueryUtils.getDeserializeValue(serdes, wrapped()), 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 d0fcb0cf0ed4..3c3999838938 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 @@ -350,7 +350,9 @@ public void close() { try { iter.close(); } finally { - sensor.record(time.nanoseconds() - startNs); + final long duration = time.nanoseconds() - startNs; + sensor.record(duration); + iteratorDurationSensor.record(duration); numOpenIterators.decrementAndGet(); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStore.java index 0c929308d84b..836bdfb3c4a0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStore.java @@ -264,7 +264,13 @@ private QueryResult runMultiVersionedKeyQuery(final Query query, final final QueryResult> rawResult = wrapped().query(rawKeyQuery, positionBound, config); if (rawResult.isSuccess()) { final MeteredMultiVersionedKeyQueryIterator typedResult = - new MeteredMultiVersionedKeyQueryIterator(rawResult.getResult(), StoreQueryUtils.getDeserializeValue(plainValueSerdes), numOpenIterators); + new MeteredMultiVersionedKeyQueryIterator( + rawResult.getResult(), + iteratorDurationSensor, + time, + StoreQueryUtils.getDeserializeValue(plainValueSerdes), + numOpenIterators + ); final QueryResult> typedQueryResult = InternalQueryResultUtil.copyAndSubstituteDeserializedResult(rawResult, typedResult); result = (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 2d63e3ca7c5b..3f7289e89ac5 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 @@ -72,6 +72,7 @@ public class MeteredWindowStore private Sensor fetchSensor; private Sensor flushSensor; private Sensor e2eLatencySensor; + private Sensor iteratorDurationSensor; private InternalProcessorContext context; private TaskId taskId; @@ -153,6 +154,7 @@ private void registerMetrics() { fetchSensor = StateStoreMetrics.fetchSensor(taskId.toString(), metricsScope, name(), streamsMetrics); flushSensor = StateStoreMetrics.flushSensor(taskId.toString(), metricsScope, name(), streamsMetrics); 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.get()); } @@ -239,6 +241,7 @@ public WindowStoreIterator fetch(final K key, return new MeteredWindowStoreIterator<>( wrapped().fetch(keyBytes(key), timeFrom, timeTo), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::valueFrom, time, @@ -254,6 +257,7 @@ public WindowStoreIterator backwardFetch(final K key, return new MeteredWindowStoreIterator<>( wrapped().backwardFetch(keyBytes(key), timeFrom, timeTo), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::valueFrom, time, @@ -273,6 +277,7 @@ public KeyValueIterator, V> fetch(final K keyFrom, timeFrom, timeTo), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, @@ -292,6 +297,7 @@ public KeyValueIterator, V> backwardFetch(final K keyFrom, timeFrom, timeTo), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, @@ -305,6 +311,7 @@ public KeyValueIterator, V> fetchAll(final long timeFrom, return new MeteredWindowedKeyValueIterator<>( wrapped().fetchAll(timeFrom, timeTo), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, @@ -318,6 +325,7 @@ public KeyValueIterator, V> backwardFetchAll(final long timeFrom, return new MeteredWindowedKeyValueIterator<>( wrapped().backwardFetchAll(timeFrom, timeTo), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, @@ -330,6 +338,7 @@ public KeyValueIterator, V> all() { return new MeteredWindowedKeyValueIterator<>( wrapped().all(), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, @@ -343,6 +352,7 @@ public KeyValueIterator, V> backwardAll() { return new MeteredWindowedKeyValueIterator<>( wrapped().backwardAll(), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, @@ -420,6 +430,7 @@ private QueryResult runRangeQuery(final Query query, new MeteredWindowedKeyValueIterator<>( rawResult.getResult(), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, getDeserializeValue(serdes, wrapped()), @@ -471,6 +482,7 @@ private QueryResult runKeyQuery(final Query query, final MeteredWindowStoreIterator typedResult = new MeteredWindowStoreIterator<>( rawResult.getResult(), fetchSensor, + iteratorDurationSensor, streamsMetrics, getDeserializeValue(serdes, wrapped()), time, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreIterator.java index 7f5cf99c0257..1294cfc1f450 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreIterator.java @@ -28,7 +28,8 @@ class MeteredWindowStoreIterator implements WindowStoreIterator { private final WindowStoreIterator iter; - private final Sensor sensor; + private final Sensor operationSensor; + private final Sensor iteratorSensor; private final StreamsMetrics metrics; private final Function valueFrom; private final long startNs; @@ -36,13 +37,15 @@ class MeteredWindowStoreIterator implements WindowStoreIterator { private final AtomicInteger numOpenIterators; MeteredWindowStoreIterator(final WindowStoreIterator iter, - final Sensor sensor, + final Sensor operationSensor, + final Sensor iteratorSensor, final StreamsMetrics metrics, final Function valueFrom, final Time time, final AtomicInteger numOpenIterators) { this.iter = iter; - this.sensor = sensor; + this.operationSensor = operationSensor; + this.iteratorSensor = iteratorSensor; this.metrics = metrics; this.valueFrom = valueFrom; this.startNs = time.nanoseconds(); @@ -67,7 +70,9 @@ public void close() { try { iter.close(); } finally { - sensor.record(time.nanoseconds() - startNs); + final long duration = time.nanoseconds() - startNs; + operationSensor.record(duration); + iteratorSensor.record(duration); numOpenIterators.decrementAndGet(); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowedKeyValueIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowedKeyValueIterator.java index a9354c863c81..e69b27c2c8ed 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowedKeyValueIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowedKeyValueIterator.java @@ -30,7 +30,8 @@ class MeteredWindowedKeyValueIterator implements KeyValueIterator, V> { private final KeyValueIterator, byte[]> iter; - private final Sensor sensor; + private final Sensor operationSensor; + private final Sensor iteratorSensor; private final StreamsMetrics metrics; private final Function deserializeKey; private final Function deserializeValue; @@ -39,14 +40,16 @@ class MeteredWindowedKeyValueIterator implements KeyValueIterator, byte[]> iter, - final Sensor sensor, + final Sensor operationSensor, + final Sensor iteratorSensor, final StreamsMetrics metrics, final Function deserializeKey, final Function deserializeValue, final Time time, final AtomicInteger numOpenIterators) { this.iter = iter; - this.sensor = sensor; + this.operationSensor = operationSensor; + this.iteratorSensor = iteratorSensor; this.metrics = metrics; this.deserializeKey = deserializeKey; this.deserializeValue = deserializeValue; @@ -77,7 +80,9 @@ public void close() { try { iter.close(); } finally { - sensor.record(time.nanoseconds() - startNs); + final long duration = time.nanoseconds() - startNs; + operationSensor.record(duration); + iteratorSensor.record(duration); numOpenIterators.decrementAndGet(); } } 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 eea7a9644221..aa9d9c3238da 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 @@ -149,6 +149,14 @@ private StateStoreMetrics() {} private static final String NUM_OPEN_ITERATORS_DESCRIPTION = "The current number of iterators on the store that have been created, but not yet closed"; + private static final String ITERATOR_DURATION = "iterator-duration"; + private static final String ITERATOR_DURATION_DESCRIPTION = + "time spent between creating an iterator and closing it, in nanoseconds"; + private static final String ITERATOR_DURATION_AVG_DESCRIPTION = + AVG_DESCRIPTION_PREFIX + ITERATOR_DURATION_DESCRIPTION; + private static final String ITERATOR_DURATION_MAX_DESCRIPTION = + MAX_DESCRIPTION_PREFIX + ITERATOR_DURATION_DESCRIPTION; + public static Sensor putSensor(final String taskId, final String storeType, final String storeName, @@ -409,6 +417,23 @@ public static Sensor e2ELatencySensor(final String taskId, return sensor; } + public static Sensor iteratorDurationSensor(final String taskId, + final String storeType, + final String storeName, + final StreamsMetricsImpl streamsMetrics) { + final Sensor sensor = streamsMetrics.storeLevelSensor(taskId, storeName, ITERATOR_DURATION, RecordingLevel.DEBUG); + final Map tagMap = streamsMetrics.storeLevelTagMap(taskId, storeType, storeName); + addAvgAndMaxToSensor( + sensor, + STATE_STORE_LEVEL_GROUP, + tagMap, + ITERATOR_DURATION, + ITERATOR_DURATION_AVG_DESCRIPTION, + ITERATOR_DURATION_MAX_DESCRIPTION + ); + return sensor; + } + public static void addNumOpenIteratorsGauge(final String taskId, final String storeType, final String storeName, 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 00151c679810..d2227bc69b10 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 @@ -30,7 +30,6 @@ import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.processor.ProcessorContext; @@ -51,6 +50,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import static org.apache.kafka.common.utils.Utils.mkEntry; @@ -98,10 +98,12 @@ public class MeteredKeyValueStoreTest { private MeteredKeyValueStore metered; private final Metrics metrics = new Metrics(); private Map tags; + private MockTime mockTime; @Before public void before() { - final Time mockTime = new MockTime(); + final MockTime mockTime = new MockTime(); + this.mockTime = mockTime; metered = new MeteredKeyValueStore<>( inner, STORE_TYPE, @@ -458,6 +460,36 @@ public void shouldTrackOpenIteratorsMetric() { assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(0)); } + @Test + public void shouldTimeIteratorDuration() { + when(inner.all()).thenReturn(KeyValueIterators.emptyIterator()); + init(); + + final KafkaMetric iteratorDurationAvgMetric = metric("iterator-duration-avg"); + final KafkaMetric iteratorDurationMaxMetric = metric("iterator-duration-max"); + assertThat(iteratorDurationAvgMetric, not(nullValue())); + assertThat(iteratorDurationMaxMetric, not(nullValue())); + + assertThat((Double) iteratorDurationAvgMetric.metricValue(), equalTo(Double.NaN)); + assertThat((Double) iteratorDurationMaxMetric.metricValue(), equalTo(Double.NaN)); + + try (final KeyValueIterator iterator = metered.all()) { + // nothing to do, just close immediately + mockTime.sleep(2); + } + + assertThat((double) iteratorDurationAvgMetric.metricValue(), equalTo(2.0 * TimeUnit.MILLISECONDS.toNanos(1))); + assertThat((double) iteratorDurationMaxMetric.metricValue(), equalTo(2.0 * TimeUnit.MILLISECONDS.toNanos(1))); + + try (final KeyValueIterator iterator = metered.all()) { + // nothing to do, just close immediately + mockTime.sleep(3); + } + + assertThat((double) iteratorDurationAvgMetric.metricValue(), equalTo(2.5 * TimeUnit.MILLISECONDS.toNanos(1))); + assertThat((double) iteratorDurationMaxMetric.metricValue(), equalTo(3.0 * TimeUnit.MILLISECONDS.toNanos(1))); + } + private KafkaMetric metric(final MetricName metricName) { return this.metrics.metric(metricName); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java index b7c99032f892..3c5a923e6d42 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java @@ -53,6 +53,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import static org.apache.kafka.common.utils.Utils.mkEntry; @@ -95,6 +96,7 @@ public class MeteredSessionStoreTest { private final String threadId = Thread.currentThread().getName(); private final TaskId taskId = new TaskId(0, 0, "My-Topology"); private final Metrics metrics = new Metrics(); + private MockTime mockTime; private MeteredSessionStore store; @Mock private SessionStore innerStore; @@ -105,7 +107,7 @@ public class MeteredSessionStoreTest { @Before public void before() { - final Time mockTime = new MockTime(); + mockTime = new MockTime(); store = new MeteredSessionStore<>( innerStore, STORE_TYPE, @@ -622,6 +624,36 @@ public void shouldTrackOpenIteratorsMetric() { assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(0)); } + @Test + public void shouldTimeIteratorDuration() { + when(innerStore.backwardFetch(KEY_BYTES)).thenReturn(KeyValueIterators.emptyIterator()); + init(); + + final KafkaMetric iteratorDurationAvgMetric = metric("iterator-duration-avg"); + final KafkaMetric iteratorDurationMaxMetric = metric("iterator-duration-max"); + assertThat(iteratorDurationAvgMetric, not(nullValue())); + assertThat(iteratorDurationMaxMetric, not(nullValue())); + + assertThat((Double) iteratorDurationAvgMetric.metricValue(), equalTo(Double.NaN)); + assertThat((Double) iteratorDurationMaxMetric.metricValue(), equalTo(Double.NaN)); + + try (final KeyValueIterator, String> iterator = store.backwardFetch(KEY)) { + // nothing to do, just close immediately + mockTime.sleep(2); + } + + assertThat((double) iteratorDurationAvgMetric.metricValue(), equalTo(2.0 * TimeUnit.MILLISECONDS.toNanos(1))); + assertThat((double) iteratorDurationMaxMetric.metricValue(), equalTo(2.0 * TimeUnit.MILLISECONDS.toNanos(1))); + + try (final KeyValueIterator, String> iterator = store.backwardFetch(KEY)) { + // nothing to do, just close immediately + mockTime.sleep(3); + } + + assertThat((double) iteratorDurationAvgMetric.metricValue(), equalTo(2.5 * TimeUnit.MILLISECONDS.toNanos(1))); + assertThat((double) iteratorDurationMaxMetric.metricValue(), equalTo(3.0 * TimeUnit.MILLISECONDS.toNanos(1))); + } + private KafkaMetric metric(final String name) { return this.metrics.metric(new MetricName(name, STORE_LEVEL_GROUP, "", this.tags)); } 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 f629acfc9a8a..7e37440a075b 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 @@ -29,7 +29,6 @@ import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.StreamsException; @@ -53,6 +52,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; @@ -95,6 +95,7 @@ public class MeteredTimestampedKeyValueStoreTest { private KeyValueStore inner; @Mock private InternalProcessorContext context; + private MockTime mockTime; private final static Map CONFIGS = mkMap(mkEntry(StreamsConfig.InternalConfig.TOPIC_PREFIX_ALTERNATIVE, APPLICATION_ID)); @@ -107,7 +108,7 @@ public class MeteredTimestampedKeyValueStoreTest { @Before public void before() { - final Time mockTime = new MockTime(); + mockTime = new MockTime(); metered = new MeteredTimestampedKeyValueStore<>( inner, "scope", @@ -456,4 +457,34 @@ public void shouldTrackOpenIteratorsMetric() { assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(0)); } + + @Test + public void shouldTimeIteratorDuration() { + when(inner.all()).thenReturn(KeyValueIterators.emptyIterator()); + init(); + + final KafkaMetric iteratorDurationAvgMetric = metric("iterator-duration-avg"); + final KafkaMetric iteratorDurationMaxMetric = metric("iterator-duration-max"); + assertThat(iteratorDurationAvgMetric, not(nullValue())); + assertThat(iteratorDurationMaxMetric, not(nullValue())); + + assertThat((Double) iteratorDurationAvgMetric.metricValue(), equalTo(Double.NaN)); + assertThat((Double) iteratorDurationMaxMetric.metricValue(), equalTo(Double.NaN)); + + try (final KeyValueIterator> iterator = metered.all()) { + // nothing to do, just close immediately + mockTime.sleep(2); + } + + assertThat((double) iteratorDurationAvgMetric.metricValue(), equalTo(2.0 * TimeUnit.MILLISECONDS.toNanos(1))); + assertThat((double) iteratorDurationMaxMetric.metricValue(), equalTo(2.0 * TimeUnit.MILLISECONDS.toNanos(1))); + + try (final KeyValueIterator> iterator = metered.all()) { + // nothing to do, just close immediately + mockTime.sleep(3); + } + + assertThat((double) iteratorDurationAvgMetric.metricValue(), equalTo(2.5 * TimeUnit.MILLISECONDS.toNanos(1))); + assertThat((double) iteratorDurationMaxMetric.metricValue(), equalTo(3.0 * TimeUnit.MILLISECONDS.toNanos(1))); + } } 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 99f76839041c..10b8e2ec6fb1 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 @@ -39,6 +39,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.KafkaMetric; @@ -390,6 +391,41 @@ public void shouldTrackOpenIteratorsMetric() { assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(0)); } + @Test + public void shouldTimeIteratorDuration() { + final MultiVersionedKeyQuery query = MultiVersionedKeyQuery.withKey(KEY); + final PositionBound bound = PositionBound.unbounded(); + final QueryConfig config = new QueryConfig(false); + when(inner.query(any(), any(), any())).thenReturn( + QueryResult.forResult(new LogicalSegmentIterator(Collections.emptyListIterator(), RAW_KEY, 0L, 0L, ResultOrder.ANY))); + + final KafkaMetric iteratorDurationAvgMetric = getMetric("iterator-duration-avg"); + final KafkaMetric iteratorDurationMaxMetric = getMetric("iterator-duration-max"); + assertThat(iteratorDurationAvgMetric, not(nullValue())); + assertThat(iteratorDurationMaxMetric, not(nullValue())); + + assertThat((Double) iteratorDurationAvgMetric.metricValue(), equalTo(Double.NaN)); + assertThat((Double) iteratorDurationMaxMetric.metricValue(), equalTo(Double.NaN)); + + final QueryResult> first = store.query(query, bound, config); + try (final VersionedRecordIterator iterator = first.getResult()) { + // nothing to do, just close immediately + mockTime.sleep(2); + } + + assertThat((double) iteratorDurationAvgMetric.metricValue(), equalTo(2.0 * TimeUnit.MILLISECONDS.toNanos(1))); + assertThat((double) iteratorDurationMaxMetric.metricValue(), equalTo(2.0 * TimeUnit.MILLISECONDS.toNanos(1))); + + final QueryResult> second = store.query(query, bound, config); + try (final VersionedRecordIterator iterator = second.getResult()) { + // nothing to do, just close immediately + mockTime.sleep(3); + } + + assertThat((double) iteratorDurationAvgMetric.metricValue(), equalTo(2.5 * TimeUnit.MILLISECONDS.toNanos(1))); + assertThat((double) iteratorDurationMaxMetric.metricValue(), equalTo(3.0 * TimeUnit.MILLISECONDS.toNanos(1))); + } + private KafkaMetric getMetric(final String name) { return metrics.metric(new MetricName(name, STORE_LEVEL_GROUP, "", tags)); } 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 3997a5e549ec..aa637035c775 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 @@ -52,6 +52,7 @@ import java.time.temporal.ChronoUnit; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import static java.time.Instant.ofEpochMilli; @@ -94,11 +95,12 @@ public class MeteredWindowStoreTest { private InternalMockProcessorContext context; @SuppressWarnings("unchecked") private final WindowStore innerStoreMock = mock(WindowStore.class); + private final MockTime mockTime = new MockTime(); private MeteredWindowStore store = new MeteredWindowStore<>( innerStoreMock, WINDOW_SIZE_MS, // any size STORE_TYPE, - new MockTime(), + mockTime, Serdes.String(), new SerdeThatDoesntHandleNull() ); @@ -463,6 +465,36 @@ public void shouldTrackOpenIteratorsMetric() { assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(0)); } + @Test + public void shouldTimeIteratorDuration() { + when(innerStoreMock.all()).thenReturn(KeyValueIterators.emptyIterator()); + store.init((StateStoreContext) context, store); + + final KafkaMetric iteratorDurationAvgMetric = metric("iterator-duration-avg"); + final KafkaMetric iteratorDurationMaxMetric = metric("iterator-duration-max"); + assertThat(iteratorDurationAvgMetric, not(nullValue())); + assertThat(iteratorDurationMaxMetric, not(nullValue())); + + assertThat((Double) iteratorDurationAvgMetric.metricValue(), equalTo(Double.NaN)); + assertThat((Double) iteratorDurationMaxMetric.metricValue(), equalTo(Double.NaN)); + + try (final KeyValueIterator, String> iterator = store.all()) { + // nothing to do, just close immediately + mockTime.sleep(2); + } + + assertThat((double) iteratorDurationAvgMetric.metricValue(), equalTo(2.0 * TimeUnit.MILLISECONDS.toNanos(1))); + assertThat((double) iteratorDurationMaxMetric.metricValue(), equalTo(2.0 * TimeUnit.MILLISECONDS.toNanos(1))); + + try (final KeyValueIterator, String> iterator = store.all()) { + // nothing to do, just close immediately + mockTime.sleep(3); + } + + assertThat((double) iteratorDurationAvgMetric.metricValue(), equalTo(2.5 * TimeUnit.MILLISECONDS.toNanos(1))); + assertThat((double) iteratorDurationMaxMetric.metricValue(), equalTo(3.0 * TimeUnit.MILLISECONDS.toNanos(1))); + } + private KafkaMetric metric(final String name) { return metrics.metric(new MetricName(name, STORE_LEVEL_GROUP, "", tags)); } diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java b/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java index aa3791951545..3d8aa93af6d6 100644 --- a/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java @@ -332,7 +332,7 @@ private long parseTimeoutMs() { private MessageFormatter buildFormatter() { MessageFormatter formatter = null; try { - Class messageFormatterClass = Class.forName(options.valueOf(messageFormatterOpt)); + Class messageFormatterClass = Class.forName(convertDeprecatedClass(options.valueOf(messageFormatterOpt))); formatter = (MessageFormatter) messageFormatterClass.getDeclaredConstructor().newInstance(); Properties formatterArgs = formatterArgs(); @@ -349,6 +349,25 @@ private MessageFormatter buildFormatter() { return formatter; } + private static String convertDeprecatedClass(String className) { + switch (className) { + case "kafka.tools.DefaultMessageFormatter": + System.err.println("WARNING: kafka.tools.DefaultMessageFormatter is deprecated and will be removed in the next major release. " + + "Please use org.apache.kafka.tools.consumer.DefaultMessageFormatter instead"); + return DefaultMessageFormatter.class.getName(); + case "kafka.tools.LoggingMessageFormatter": + System.err.println("WARNING: kafka.tools.LoggingMessageFormatter is deprecated and will be removed in the next major release. " + + "Please use org.apache.kafka.tools.consumer.LoggingMessageFormatter instead"); + return LoggingMessageFormatter.class.getName(); + case "kafka.tools.NoOpMessageFormatter": + System.err.println("WARNING: kafka.tools.NoOpMessageFormatter is deprecated and will be removed in the next major release. " + + "Please use org.apache.kafka.tools.consumer.NoOpMessageFormatter instead"); + return NoOpMessageFormatter.class.getName(); + default: + return className; + } + } + Properties consumerProps() { return consumerProps; } diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java index c7d3a82232b7..08029bc1d82b 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java @@ -645,4 +645,31 @@ public void testParseTimeoutMs() throws Exception { }; assertEquals(100, new ConsoleConsumerOptions(validTimeoutMs).timeoutMs()); } + + @Test + public void testParseDeprecatedFormatter() throws Exception { + String[] deprecatedDefaultMessageFormatter = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--partition", "0", + "--formatter", "kafka.tools.DefaultMessageFormatter", + }; + assertInstanceOf(DefaultMessageFormatter.class, new ConsoleConsumerOptions(deprecatedDefaultMessageFormatter).formatter()); + + String[] deprecatedLoggingMessageFormatter = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--partition", "0", + "--formatter", "kafka.tools.LoggingMessageFormatter", + }; + assertInstanceOf(LoggingMessageFormatter.class, new ConsoleConsumerOptions(deprecatedLoggingMessageFormatter).formatter()); + + String[] deprecatedNoOpMessageFormatter = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--partition", "0", + "--formatter", "kafka.tools.NoOpMessageFormatter", + }; + assertInstanceOf(NoOpMessageFormatter.class, new ConsoleConsumerOptions(deprecatedNoOpMessageFormatter).formatter()); + } }