From 8fa0d9723f6aa168cc1e050a6ea0a902e0c179fb Mon Sep 17 00:00:00 2001 From: Hong-Yi Chen <131829440+apalan60@users.noreply.github.com> Date: Mon, 21 Apr 2025 07:35:47 +0000 Subject: [PATCH 01/37] MINOR: Fix typo in ApiKeyVersionsProvider exception message (#19521) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This patch addresses issue #19516 and corrects a typo in `ApiKeyVersionsProvider`: when `toVersion` exceeds `latestVersion`, the `IllegalArgumentException` message was erroneously formatted with `fromVersion`. The format argument has been updated to use `toVersion` so that the error message reports the correct value. Reviewers: Ken Huang , PoAn Yang , Jhen-Yung Hsu , Chia-Ping Tsai --- .../kafka/common/utils/annotation/ApiKeyVersionsProvider.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/test/java/org/apache/kafka/common/utils/annotation/ApiKeyVersionsProvider.java b/clients/src/test/java/org/apache/kafka/common/utils/annotation/ApiKeyVersionsProvider.java index ea510414f0481..6c80d2b5df5f1 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/annotation/ApiKeyVersionsProvider.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/annotation/ApiKeyVersionsProvider.java @@ -52,7 +52,7 @@ public void accept(ApiKeyVersionsSource source) { if (toVersion > latestVersion) { throw new IllegalArgumentException(String.format("The toVersion %s is newer than the latest version %s", - fromVersion, latestVersion)); + toVersion, latestVersion)); } } From 7710d1c9511cc3e02ba2ecc8f48674f01ed05ae2 Mon Sep 17 00:00:00 2001 From: Mickael Maison Date: Mon, 21 Apr 2025 12:03:30 +0200 Subject: [PATCH 02/37] KAFKA-14487: Move LogManager static methods/fields to storage module (#19302) Move the static fields/methods Reviewers: Luke Chen --- build.gradle | 1 + checkstyle/import-control-storage.xml | 2 + .../src/main/scala/kafka/log/LogManager.scala | 73 +------- .../main/scala/kafka/raft/RaftManager.scala | 5 +- .../metadata/BrokerMetadataPublisher.scala | 3 +- .../scala/unit/kafka/log/LogManagerTest.scala | 166 ++--------------- .../unit/kafka/raft/RaftManagerTest.scala | 6 +- .../kafka/server/ServerShutdownTest.scala | 4 +- .../storage/internals/log/LogManager.java | 96 ++++++++++ .../storage/internals/log/LogManagerTest.java | 175 ++++++++++++++++++ ...FromLeaderWithCorruptedCheckpointTest.java | 4 +- 11 files changed, 301 insertions(+), 234 deletions(-) create mode 100644 storage/src/main/java/org/apache/kafka/storage/internals/log/LogManager.java create mode 100644 storage/src/test/java/org/apache/kafka/storage/internals/log/LogManagerTest.java diff --git a/build.gradle b/build.gradle index 2e35057165c53..50eec676569e9 100644 --- a/build.gradle +++ b/build.gradle @@ -2230,6 +2230,7 @@ project(':storage') { } dependencies { + implementation project(':metadata') implementation project(':storage:storage-api') implementation project(':server-common') implementation project(':clients') diff --git a/checkstyle/import-control-storage.xml b/checkstyle/import-control-storage.xml index 639cb6dc1d04c..6628de8d45345 100644 --- a/checkstyle/import-control-storage.xml +++ b/checkstyle/import-control-storage.xml @@ -94,6 +94,8 @@ + + diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index 506e46e6ce5d7..e555f1b0b467d 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -23,7 +23,6 @@ import java.nio.file.{Files, NoSuchFileException} import java.util.concurrent._ import java.util.concurrent.atomic.AtomicInteger import kafka.server.{KafkaConfig, KafkaRaftServer} -import kafka.server.metadata.BrokerMetadataPublisher.info import kafka.utils.threadsafe import kafka.utils.{CoreUtils, Logging, Pool} import org.apache.kafka.common.{DirectoryId, KafkaException, TopicPartition, Uuid} @@ -42,7 +41,7 @@ import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsem import java.util.{Collections, Optional, OptionalLong, Properties} import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.server.util.{FileLock, Scheduler} -import org.apache.kafka.storage.internals.log.{CleanerConfig, LogCleaner, LogConfig, LogDirFailureChannel, LogOffsetsListener, ProducerStateManagerConfig, RemoteIndexCache, UnifiedLog} +import org.apache.kafka.storage.internals.log.{CleanerConfig, LogCleaner, LogConfig, LogDirFailureChannel, LogManager => JLogManager, LogOffsetsListener, ProducerStateManagerConfig, RemoteIndexCache, UnifiedLog} import org.apache.kafka.storage.internals.checkpoint.{CleanShutdownFileHandler, OffsetCheckpointFile} import org.apache.kafka.storage.log.metrics.BrokerTopicStats @@ -80,8 +79,6 @@ class LogManager(logDirs: Seq[File], remoteStorageSystemEnable: Boolean, val initialTaskDelayMs: Long) extends Logging { - import LogManager._ - private val metricsGroup = new KafkaMetricsGroup(this.getClass) private val logCreationOrDeletionLock = new Object @@ -127,9 +124,9 @@ class LogManager(logDirs: Seq[File], def directoryIdsSet: Predef.Set[Uuid] = directoryIds.values.toSet @volatile private var recoveryPointCheckpoints = liveLogDirs.map(dir => - (dir, new OffsetCheckpointFile(new File(dir, RecoveryPointCheckpointFile), logDirFailureChannel))).toMap + (dir, new OffsetCheckpointFile(new File(dir, JLogManager.RECOVERY_POINT_CHECKPOINT_FILE), logDirFailureChannel))).toMap @volatile private var logStartOffsetCheckpoints = liveLogDirs.map(dir => - (dir, new OffsetCheckpointFile(new File(dir, LogStartOffsetCheckpointFile), logDirFailureChannel))).toMap + (dir, new OffsetCheckpointFile(new File(dir, JLogManager.LOG_START_OFFSET_CHECKPOINT_FILE), logDirFailureChannel))).toMap private val preferredLogDirs = new ConcurrentHashMap[TopicPartition, String]() @@ -261,7 +258,7 @@ class LogManager(logDirs: Seq[File], private def lockLogDirs(dirs: Seq[File]): Seq[FileLock] = { dirs.flatMap { dir => try { - val lock = new FileLock(new File(dir, LockFileName)) + val lock = new FileLock(new File(dir, JLogManager.LOCK_FILE_NAME)) if (!lock.tryLock()) throw new KafkaException("Failed to acquire lock on file .lock in " + lock.file.getParent + ". A Kafka instance in another process or thread is using this directory.") @@ -680,7 +677,7 @@ class LogManager(logDirs: Seq[File], try { jobs.foreachEntry { (dir, dirJobs) => - if (waitForAllToComplete(dirJobs, + if (JLogManager.waitForAllToComplete(dirJobs.toList.asJava, e => warn(s"There was an error in one of the threads during LogManager shutdown: ${e.getCause}"))) { val logs = logsInDir(localLogsByDir, dir) @@ -1520,25 +1517,6 @@ class LogManager(logDirs: Seq[File], } object LogManager { - val LockFileName = ".lock" - - /** - * Wait all jobs to complete - * @param jobs jobs - * @param callback this will be called to handle the exception caused by each Future#get - * @return true if all pass. Otherwise, false - */ - private[log] def waitForAllToComplete(jobs: Seq[Future[_]], callback: Throwable => Unit): Boolean = { - jobs.count(future => Try(future.get) match { - case Success(_) => false - case Failure(e) => - callback(e) - true - }) == 0 - } - - val RecoveryPointCheckpointFile = "recovery-point-offset-checkpoint" - val LogStartOffsetCheckpointFile = "log-start-offset-checkpoint" def apply(config: KafkaConfig, initialOfflineDirs: Seq[String], @@ -1575,45 +1553,4 @@ object LogManager { remoteStorageSystemEnable = config.remoteLogManagerConfig.isRemoteStorageSystemEnabled, initialTaskDelayMs = config.logInitialTaskDelayMs) } - - /** - * Returns true if the given log should not be on the current broker - * according to the metadata image. - * - * @param brokerId The ID of the current broker. - * @param newTopicsImage The new topics image after broker has been reloaded - * @param log The log object to check - * @return true if the log should not exist on the broker, false otherwise. - */ - def isStrayKraftReplica( - brokerId: Int, - newTopicsImage: TopicsImage, - log: UnifiedLog - ): Boolean = { - if (log.topicId.isEmpty) { - // Missing topic ID could result from storage failure or unclean shutdown after topic creation but before flushing - // data to the `partition.metadata` file. And before appending data to the log, the `partition.metadata` is always - // flushed to disk. So if the topic ID is missing, it mostly means no data was appended, and we can treat this as - // a stray log. - info(s"The topicId does not exist in $log, treat it as a stray log") - return true - } - - val topicId = log.topicId.get - val partitionId = log.topicPartition.partition() - Option(newTopicsImage.getPartition(topicId, partitionId)) match { - case Some(partition) => - if (!partition.replicas.contains(brokerId)) { - info(s"Found stray log dir $log: the current replica assignment ${partition.replicas.mkString("[", ", ", "]")} " + - s"does not contain the local brokerId $brokerId.") - true - } else { - false - } - - case None => - info(s"Found stray log dir $log: the topicId $topicId does not exist in the metadata image") - true - } - } } diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala b/core/src/main/scala/kafka/raft/RaftManager.scala index 84dfa5ebee001..0727c660fe406 100644 --- a/core/src/main/scala/kafka/raft/RaftManager.scala +++ b/core/src/main/scala/kafka/raft/RaftManager.scala @@ -24,7 +24,6 @@ import java.util.OptionalInt import java.util.concurrent.CompletableFuture import java.util.{Map => JMap} import java.util.{Collection => JCollection} -import kafka.log.LogManager import kafka.server.KafkaConfig import kafka.utils.CoreUtils import kafka.utils.Logging @@ -48,7 +47,7 @@ import org.apache.kafka.server.common.serialization.RecordSerde import org.apache.kafka.server.util.{FileLock, KafkaScheduler} import org.apache.kafka.server.fault.FaultHandler import org.apache.kafka.server.util.timer.SystemTimer -import org.apache.kafka.storage.internals.log.UnifiedLog +import org.apache.kafka.storage.internals.log.{LogManager, UnifiedLog} import scala.jdk.CollectionConverters._ import scala.jdk.OptionConverters._ @@ -62,7 +61,7 @@ object KafkaRaftManager { } private def lockDataDir(dataDir: File): FileLock = { - val lock = new FileLock(new File(dataDir, LogManager.LockFileName)) + val lock = new FileLock(new File(dataDir, LogManager.LOCK_FILE_NAME)) if (!lock.tryLock()) { throw new KafkaException( diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala index de8f16e1e5808..d95be34ff51de 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala @@ -34,6 +34,7 @@ import org.apache.kafka.image.{MetadataDelta, MetadataImage, TopicDelta} import org.apache.kafka.metadata.publisher.AclPublisher import org.apache.kafka.server.common.RequestLocal import org.apache.kafka.server.fault.FaultHandler +import org.apache.kafka.storage.internals.log.{LogManager => JLogManager} import java.util.concurrent.CompletableFuture import scala.collection.mutable @@ -300,7 +301,7 @@ class BrokerMetadataPublisher( // recovery-from-unclean-shutdown if required. logManager.startup( metadataCache.getAllTopics().asScala, - isStray = log => LogManager.isStrayKraftReplica(brokerId, newImage.topics(), log) + isStray = log => JLogManager.isStrayKraftReplica(brokerId, newImage.topics(), log) ) // Rename all future replicas which are in the same directory as the diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 5e721596ce057..67880e0ced54c 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -23,15 +23,14 @@ import org.apache.directory.api.util.FileUtils import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.errors.OffsetOutOfRangeException import org.apache.kafka.common.utils.{Time, Utils} -import org.apache.kafka.common.{DirectoryId, KafkaException, TopicIdPartition, TopicPartition, Uuid} +import org.apache.kafka.common.{DirectoryId, KafkaException, TopicPartition, Uuid} import org.apache.kafka.coordinator.transaction.TransactionLogConfig -import org.apache.kafka.image.{TopicImage, TopicsImage} -import org.apache.kafka.metadata.{ConfigRepository, LeaderRecoveryState, MockConfigRepository, PartitionRegistration} +import org.apache.kafka.metadata.{ConfigRepository, MockConfigRepository} import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.ArgumentMatchers.any -import org.mockito.{ArgumentCaptor, ArgumentMatchers, Mockito} +import org.mockito.{ArgumentCaptor, ArgumentMatchers} import org.mockito.Mockito.{doAnswer, doNothing, mock, never, spy, times, verify} import java.io._ @@ -39,12 +38,12 @@ import java.lang.{Long => JLong} import java.nio.file.Files import java.nio.file.attribute.PosixFilePermission import java.util -import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap, Future} +import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap} import java.util.{Collections, Optional, OptionalLong, Properties} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.storage.log.FetchIsolation import org.apache.kafka.server.util.{FileLock, KafkaScheduler, MockTime, Scheduler} -import org.apache.kafka.storage.internals.log.{CleanerConfig, FetchDataInfo, LogConfig, LogDirFailureChannel, LogMetricNames, LogOffsetsListener, LogStartOffsetIncrementReason, ProducerStateManagerConfig, RemoteIndexCache, UnifiedLog} +import org.apache.kafka.storage.internals.log.{CleanerConfig, FetchDataInfo, LogConfig, LogDirFailureChannel, LogMetricNames, LogManager => JLogManager, LogOffsetsListener, LogStartOffsetIncrementReason, ProducerStateManagerConfig, RemoteIndexCache, UnifiedLog} import org.apache.kafka.storage.internals.checkpoint.{CleanShutdownFileHandler, OffsetCheckpointFile} import org.apache.kafka.storage.log.metrics.BrokerTopicStats import org.junit.jupiter.api.function.Executable @@ -56,7 +55,6 @@ import scala.jdk.CollectionConverters._ import scala.util.{Failure, Try} class LogManagerTest { - import LogManagerTest._ val time = new MockTime() val maxRollInterval = 100 @@ -592,7 +590,7 @@ class LogManagerTest { } logManager.checkpointLogRecoveryOffsets() - val checkpoints = new OffsetCheckpointFile(new File(logDir, LogManager.RecoveryPointCheckpointFile), null).read() + val checkpoints = new OffsetCheckpointFile(new File(logDir, JLogManager.RECOVERY_POINT_CHECKPOINT_FILE), null).read() topicPartitions.zip(logs).foreach { case (tp, log) => assertEquals(checkpoints.get(tp), log.recoveryPoint, "Recovery point should equal checkpoint") @@ -672,7 +670,7 @@ class LogManagerTest { logManager.checkpointRecoveryOffsetsInDir(logDir) - val checkpoints = new OffsetCheckpointFile(new File(logDir, LogManager.RecoveryPointCheckpointFile), null).read() + val checkpoints = new OffsetCheckpointFile(new File(logDir, JLogManager.RECOVERY_POINT_CHECKPOINT_FILE), null).read() tps.zip(allLogs).foreach { case (tp, log) => assertEquals(checkpoints.get(tp), log.recoveryPoint, @@ -1094,36 +1092,6 @@ class LogManagerTest { verifyMetrics(1) } - @Test - def testWaitForAllToComplete(): Unit = { - var invokedCount = 0 - val success: Future[Boolean] = Mockito.mock(classOf[Future[Boolean]]) - Mockito.when(success.get()).thenAnswer { _ => - invokedCount += 1 - true - } - val failure: Future[Boolean] = Mockito.mock(classOf[Future[Boolean]]) - Mockito.when(failure.get()).thenAnswer{ _ => - invokedCount += 1 - throw new RuntimeException - } - - var failureCount = 0 - // all futures should be evaluated - assertFalse(LogManager.waitForAllToComplete(Seq(success, failure), _ => failureCount += 1)) - assertEquals(2, invokedCount) - assertEquals(1, failureCount) - assertFalse(LogManager.waitForAllToComplete(Seq(failure, success), _ => failureCount += 1)) - assertEquals(4, invokedCount) - assertEquals(2, failureCount) - assertTrue(LogManager.waitForAllToComplete(Seq(success, success), _ => failureCount += 1)) - assertEquals(6, invokedCount) - assertEquals(2, failureCount) - assertFalse(LogManager.waitForAllToComplete(Seq(failure, failure), _ => failureCount += 1)) - assertEquals(8, invokedCount) - assertEquals(4, failureCount) - } - @Test def testLoadDirectoryIds(): Unit = { val dirs: Seq[File] = Seq.fill(5)(TestUtils.tempDir()) @@ -1161,7 +1129,7 @@ class LogManagerTest { remoteStorageSystemEnable = true ) - val checkpointFile = new File(logDir, LogManager.LogStartOffsetCheckpointFile) + val checkpointFile = new File(logDir, JLogManager.LOG_START_OFFSET_CHECKPOINT_FILE) val checkpoint = new OffsetCheckpointFile(checkpointFile, null) val topicPartition = new TopicPartition("test", 0) val log = logManager.getOrCreateLog(topicPartition, topicId = Optional.empty) @@ -1192,7 +1160,7 @@ class LogManagerTest { @Test def testCheckpointLogStartOffsetForNormalTopic(): Unit = { - val checkpointFile = new File(logDir, LogManager.LogStartOffsetCheckpointFile) + val checkpointFile = new File(logDir, JLogManager.LOG_START_OFFSET_CHECKPOINT_FILE) val checkpoint = new OffsetCheckpointFile(checkpointFile, null) val topicPartition = new TopicPartition("test", 0) val log = logManager.getOrCreateLog(topicPartition, topicId = Optional.empty) @@ -1233,65 +1201,6 @@ class LogManagerTest { new File(dir, MetaPropertiesEnsemble.META_PROPERTIES_NAME).getAbsolutePath, false) } - val foo0 = new TopicIdPartition(Uuid.fromString("Sl08ZXU2QW6uF5hIoSzc8w"), new TopicPartition("foo", 0)) - val foo1 = new TopicIdPartition(Uuid.fromString("Sl08ZXU2QW6uF5hIoSzc8w"), new TopicPartition("foo", 1)) - val bar0 = new TopicIdPartition(Uuid.fromString("69O438ZkTSeqqclTtZO2KA"), new TopicPartition("bar", 0)) - val bar1 = new TopicIdPartition(Uuid.fromString("69O438ZkTSeqqclTtZO2KA"), new TopicPartition("bar", 1)) - val baz0 = new TopicIdPartition(Uuid.fromString("2Ik9_5-oRDOKpSXd2SuG5w"), new TopicPartition("baz", 0)) - val baz1 = new TopicIdPartition(Uuid.fromString("2Ik9_5-oRDOKpSXd2SuG5w"), new TopicPartition("baz", 1)) - val baz2 = new TopicIdPartition(Uuid.fromString("2Ik9_5-oRDOKpSXd2SuG5w"), new TopicPartition("baz", 2)) - val quux0 = new TopicIdPartition(Uuid.fromString("YS9owjv5TG2OlsvBM0Qw6g"), new TopicPartition("quux", 0)) - val recreatedFoo0 = new TopicIdPartition(Uuid.fromString("_dOOzPe3TfiWV21Lh7Vmqg"), new TopicPartition("foo", 0)) - val recreatedFoo1 = new TopicIdPartition(Uuid.fromString("_dOOzPe3TfiWV21Lh7Vmqg"), new TopicPartition("foo", 1)) - - @Test - def testIsStrayKraftReplicaWithEmptyImage(): Unit = { - val image: TopicsImage = topicsImage(Seq()) - val onDisk = Seq(foo0, foo1, bar0, bar1, quux0).map(mockLog) - assertTrue(onDisk.forall(log => LogManager.isStrayKraftReplica(0, image, log))) - } - - @Test - def testIsStrayKraftReplicaInImage(): Unit = { - val image: TopicsImage = topicsImage(Seq( - topicImage(Map( - foo0 -> Seq(0, 1, 2), - )), - topicImage(Map( - bar0 -> Seq(0, 1, 2), - bar1 -> Seq(0, 1, 2), - )) - )) - val onDisk = Seq(foo0, foo1, bar0, bar1, quux0).map(mockLog) - val expectedStrays = Set(foo1, quux0).map(_.topicPartition()) - - onDisk.foreach(log => assertEquals(expectedStrays.contains(log.topicPartition), LogManager.isStrayKraftReplica(0, image, log))) - } - - @Test - def testIsStrayKraftReplicaInImageWithRemoteReplicas(): Unit = { - val image: TopicsImage = topicsImage(Seq( - topicImage(Map( - foo0 -> Seq(0, 1, 2), - )), - topicImage(Map( - bar0 -> Seq(1, 2, 3), - bar1 -> Seq(2, 3, 0), - )) - )) - val onDisk = Seq(foo0, bar0, bar1).map(mockLog) - val expectedStrays = Set(bar0).map(_.topicPartition) - - onDisk.foreach(log => assertEquals(expectedStrays.contains(log.topicPartition), LogManager.isStrayKraftReplica(0, image, log))) - } - - @Test - def testIsStrayKraftMissingTopicId(): Unit = { - val log = Mockito.mock(classOf[UnifiedLog]) - Mockito.when(log.topicId).thenReturn(Optional.empty) - assertTrue(LogManager.isStrayKraftReplica(0, topicsImage(Seq()), log)) - } - /** * Test LogManager takes file lock by default and the lock is released after shutdown. */ @@ -1302,12 +1211,12 @@ class LogManagerTest { try { // ${tmpLogDir}.lock is acquired by tmpLogManager - val fileLock = new FileLock(new File(tmpLogDir, LogManager.LockFileName)) + val fileLock = new FileLock(new File(tmpLogDir, JLogManager.LOCK_FILE_NAME)) assertFalse(fileLock.tryLock()) } finally { // ${tmpLogDir}.lock is removed after shutdown tmpLogManager.shutdown() - val f = new File(tmpLogDir, LogManager.LockFileName) + val f = new File(tmpLogDir, JLogManager.LOCK_FILE_NAME) assertFalse(f.exists()) } } @@ -1376,56 +1285,3 @@ class LogManagerTest { } } } - -object LogManagerTest { - def mockLog( - topicIdPartition: TopicIdPartition - ): UnifiedLog = { - val log = Mockito.mock(classOf[UnifiedLog]) - Mockito.when(log.topicId).thenReturn(Optional.of(topicIdPartition.topicId())) - Mockito.when(log.topicPartition).thenReturn(topicIdPartition.topicPartition()) - log - } - - def topicImage( - partitions: Map[TopicIdPartition, Seq[Int]] - ): TopicImage = { - var topicName: String = null - var topicId: Uuid = null - partitions.keySet.foreach { - partition => if (topicId == null) { - topicId = partition.topicId() - } else if (!topicId.equals(partition.topicId())) { - throw new IllegalArgumentException("partition topic IDs did not match") - } - if (topicName == null) { - topicName = partition.topic() - } else if (!topicName.equals(partition.topic())) { - throw new IllegalArgumentException("partition topic names did not match") - } - } - if (topicId == null) { - throw new IllegalArgumentException("Invalid empty partitions map.") - } - val partitionRegistrations = partitions.map { case (partition, replicas) => - Int.box(partition.partition()) -> new PartitionRegistration.Builder(). - setReplicas(replicas.toArray). - setDirectories(DirectoryId.unassignedArray(replicas.size)). - setIsr(replicas.toArray). - setLeader(replicas.head). - setLeaderRecoveryState(LeaderRecoveryState.RECOVERED). - setLeaderEpoch(0). - setPartitionEpoch(0). - build() - } - new TopicImage(topicName, topicId, partitionRegistrations.asJava) - } - - def topicsImage( - topics: Seq[TopicImage] - ): TopicsImage = { - var retval = TopicsImage.EMPTY - topics.foreach { t => retval = retval.including(t) } - retval - } -} diff --git a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala index 4255648347cfc..3c816f635db78 100644 --- a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala +++ b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala @@ -22,7 +22,6 @@ import java.nio.channels.OverlappingFileLockException import java.nio.file.{Files, Path, StandardOpenOption} import java.util.Properties import java.util.concurrent.CompletableFuture -import kafka.log.LogManager import kafka.server.KafkaConfig import kafka.tools.TestRaftServer.ByteArraySerde import kafka.utils.TestUtils @@ -35,6 +34,7 @@ import org.apache.kafka.raft.{Endpoints, MetadataLogConfig, QuorumConfig} import org.apache.kafka.server.ProcessRole import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs, ServerLogConfigs} import org.apache.kafka.server.fault.FaultHandler +import org.apache.kafka.storage.internals.log.LogManager import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test import org.junit.jupiter.params.ParameterizedTest @@ -164,7 +164,7 @@ class RaftManagerTest { ) ) - val lockPath = metadataDir.getOrElse(logDir.head).resolve(LogManager.LockFileName) + val lockPath = metadataDir.getOrElse(logDir.head).resolve(LogManager.LOCK_FILE_NAME) assertTrue(fileLocked(lockPath)) raftManager.shutdown() @@ -188,7 +188,7 @@ class RaftManagerTest { ) ) - val lockPath = metadataDir.getOrElse(logDir.head).resolve(LogManager.LockFileName) + val lockPath = metadataDir.getOrElse(logDir.head).resolve(LogManager.LOCK_FILE_NAME) assertTrue(fileLocked(lockPath)) raftManager.shutdown() diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index a6b71d912b387..39327bbeaf858 100644 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -21,7 +21,6 @@ import kafka.utils.{CoreUtils, TestInfoUtils, TestUtils} import java.io.File import java.util.concurrent.CancellationException import kafka.integration.KafkaServerTestHarness -import kafka.log.LogManager import org.apache.kafka.clients.consumer.Consumer import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} import org.apache.kafka.common.security.auth.SecurityProtocol @@ -29,6 +28,7 @@ import org.apache.kafka.common.serialization.{IntegerDeserializer, IntegerSerial import org.apache.kafka.common.utils.Exit import org.apache.kafka.metadata.BrokerState import org.apache.kafka.server.config.{KRaftConfigs, ServerLogConfigs} +import org.apache.kafka.storage.internals.log.LogManager import org.junit.jupiter.api.{BeforeEach, TestInfo, Timeout} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.function.Executable @@ -105,7 +105,7 @@ class ServerShutdownTest extends KafkaServerTestHarness { // do a clean shutdown and check that offset checkpoint file exists shutdownBroker() for (logDir <- config.logDirs) { - val OffsetCheckpointFile = new File(logDir, LogManager.RecoveryPointCheckpointFile) + val OffsetCheckpointFile = new File(logDir, LogManager.RECOVERY_POINT_CHECKPOINT_FILE) assertTrue(OffsetCheckpointFile.exists) assertTrue(OffsetCheckpointFile.length() > 0) } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogManager.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogManager.java new file mode 100644 index 0000000000000..cc46e4fc98436 --- /dev/null +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogManager.java @@ -0,0 +1,96 @@ +/* + * 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.storage.internals.log; + +import org.apache.kafka.common.Uuid; +import org.apache.kafka.image.TopicsImage; +import org.apache.kafka.metadata.PartitionRegistration; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.Future; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +public class LogManager { + + private static final Logger LOG = LoggerFactory.getLogger(LogManager.class); + + public static final String LOCK_FILE_NAME = ".lock"; + public static final String RECOVERY_POINT_CHECKPOINT_FILE = "recovery-point-offset-checkpoint"; + public static final String LOG_START_OFFSET_CHECKPOINT_FILE = "log-start-offset-checkpoint"; + + /** + * Wait for all jobs to complete + * @param jobs The jobs + * @param callback This will be called to handle the exception caused by each Future#get + * @return true if all pass. Otherwise, false + */ + public static boolean waitForAllToComplete(List> jobs, Consumer callback) { + List> failed = new ArrayList<>(); + for (Future job : jobs) { + try { + job.get(); + } catch (Exception e) { + callback.accept(e); + failed.add(job); + } + } + return failed.isEmpty(); + } + + /** + * Returns true if the given log should not be on the current broker + * according to the metadata image. + * + * @param brokerId The ID of the current broker. + * @param newTopicsImage The new topics image after broker has been reloaded + * @param log The log object to check + * @return true if the log should not exist on the broker, false otherwise. + */ + public static boolean isStrayKraftReplica(int brokerId, TopicsImage newTopicsImage, UnifiedLog log) { + if (log.topicId().isEmpty()) { + // Missing topic ID could result from storage failure or unclean shutdown after topic creation but before flushing + // data to the `partition.metadata` file. And before appending data to the log, the `partition.metadata` is always + // flushed to disk. So if the topic ID is missing, it mostly means no data was appended, and we can treat this as + // a stray log. + LOG.info("The topicId does not exist in {}, treat it as a stray log.", log); + return true; + } + + Uuid topicId = log.topicId().get(); + int partitionId = log.topicPartition().partition(); + PartitionRegistration partition = newTopicsImage.getPartition(topicId, partitionId); + if (partition == null) { + LOG.info("Found stray log dir {}: the topicId {} does not exist in the metadata image.", log, topicId); + return true; + } else { + List replicas = Arrays.stream(partition.replicas).boxed().toList(); + if (!replicas.contains(brokerId)) { + LOG.info("Found stray log dir {}: the current replica assignment {} does not contain the local brokerId {}.", + log, replicas.stream().map(String::valueOf).collect(Collectors.joining(", ", "[", "]")), brokerId); + return true; + } else { + return false; + } + } + } +} diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/LogManagerTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/LogManagerTest.java new file mode 100644 index 0000000000000..a86ec3691cf72 --- /dev/null +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/LogManagerTest.java @@ -0,0 +1,175 @@ +/* + * 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.storage.internals.log; + +import org.apache.kafka.common.DirectoryId; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.image.TopicImage; +import org.apache.kafka.image.TopicsImage; +import org.apache.kafka.metadata.LeaderRecoveryState; +import org.apache.kafka.metadata.PartitionRegistration; + +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class LogManagerTest { + + private static final TopicIdPartition FOO_0 = new TopicIdPartition(Uuid.fromString("Sl08ZXU2QW6uF5hIoSzc8w"), new TopicPartition("foo", 0)); + private static final TopicIdPartition FOO_1 = new TopicIdPartition(Uuid.fromString("Sl08ZXU2QW6uF5hIoSzc8w"), new TopicPartition("foo", 1)); + private static final TopicIdPartition BAR_0 = new TopicIdPartition(Uuid.fromString("69O438ZkTSeqqclTtZO2KA"), new TopicPartition("bar", 0)); + private static final TopicIdPartition BAR_1 = new TopicIdPartition(Uuid.fromString("69O438ZkTSeqqclTtZO2KA"), new TopicPartition("bar", 1)); + private static final TopicIdPartition QUUX_0 = new TopicIdPartition(Uuid.fromString("YS9owjv5TG2OlsvBM0Qw6g"), new TopicPartition("quux", 0)); + + @SuppressWarnings("unchecked") + @Test + public void testWaitForAllToComplete() throws ExecutionException, InterruptedException { + AtomicInteger invokedCount = new AtomicInteger(0); + Future success = mock(Future.class); + when(success.get()).thenAnswer(a -> { + invokedCount.incrementAndGet(); + return true; + }); + Future failure = mock(Future.class); + when(failure.get()).thenAnswer(a -> { + invokedCount.incrementAndGet(); + throw new RuntimeException(); + }); + + AtomicInteger failureCount = new AtomicInteger(0); + // all futures should be evaluated + assertFalse(LogManager.waitForAllToComplete(List.of(success, failure), t -> failureCount.incrementAndGet())); + assertEquals(2, invokedCount.get()); + assertEquals(1, failureCount.get()); + assertFalse(LogManager.waitForAllToComplete(List.of(failure, success), t -> failureCount.incrementAndGet())); + assertEquals(4, invokedCount.get()); + assertEquals(2, failureCount.get()); + assertTrue(LogManager.waitForAllToComplete(List.of(success, success), t -> failureCount.incrementAndGet())); + assertEquals(6, invokedCount.get()); + assertEquals(2, failureCount.get()); + assertFalse(LogManager.waitForAllToComplete(List.of(failure, failure), t -> failureCount.incrementAndGet())); + assertEquals(8, invokedCount.get()); + assertEquals(4, failureCount.get()); + } + + @Test + public void testIsStrayKraftReplicaWithEmptyImage() { + TopicsImage image = topicsImage(List.of()); + List onDisk = Stream.of(FOO_0, FOO_1, BAR_0, BAR_1, QUUX_0).map(this::mockLog).toList(); + assertTrue(onDisk.stream().allMatch(log -> LogManager.isStrayKraftReplica(0, image, log))); + } + + @Test + public void testIsStrayKraftReplicaInImage() { + TopicsImage image = topicsImage(List.of( + topicImage(Map.of( + FOO_0, List.of(0, 1, 2))), + topicImage(Map.of( + BAR_0, List.of(0, 1, 2), + BAR_1, List.of(0, 1, 2))) + )); + List onDisk = Stream.of(FOO_0, FOO_1, BAR_0, BAR_1, QUUX_0).map(this::mockLog).toList(); + Set expectedStrays = Stream.of(FOO_1, QUUX_0).map(TopicIdPartition::topicPartition).collect(Collectors.toSet()); + + onDisk.forEach(log -> assertEquals(expectedStrays.contains(log.topicPartition()), LogManager.isStrayKraftReplica(0, image, log))); + } + + @Test + public void testIsStrayKraftReplicaInImageWithRemoteReplicas() { + TopicsImage image = topicsImage(List.of( + topicImage(Map.of( + FOO_0, List.of(0, 1, 2))), + topicImage(Map.of( + BAR_0, List.of(1, 2, 3), + BAR_1, List.of(2, 3, 0))) + )); + List onDisk = Stream.of(FOO_0, BAR_0, BAR_1).map(this::mockLog).toList(); + Set expectedStrays = Stream.of(BAR_0).map(TopicIdPartition::topicPartition).collect(Collectors.toSet()); + onDisk.forEach(log -> assertEquals(expectedStrays.contains(log.topicPartition()), LogManager.isStrayKraftReplica(0, image, log))); + } + + @Test + public void testIsStrayKraftMissingTopicId() { + UnifiedLog log = mock(UnifiedLog.class); + when(log.topicId()).thenReturn(Optional.empty()); + assertTrue(LogManager.isStrayKraftReplica(0, topicsImage(List.of()), log)); + } + + private TopicsImage topicsImage(List topics) { + TopicsImage retval = TopicsImage.EMPTY; + for (TopicImage topic : topics) { + retval = retval.including(topic); + } + return retval; + } + + private TopicImage topicImage(Map> partitions) { + String topicName = null; + Uuid topicId = null; + for (TopicIdPartition partition : partitions.keySet()) { + if (topicId == null) { + topicId = partition.topicId(); + } else if (!topicId.equals(partition.topicId())) { + throw new IllegalArgumentException("partition topic IDs did not match"); + } + if (topicName == null) { + topicName = partition.topic(); + } else if (!topicName.equals(partition.topic())) { + throw new IllegalArgumentException("partition topic names did not match"); + } + } + if (topicId == null) { + throw new IllegalArgumentException("Invalid empty partitions map."); + } + Map partitionRegistrations = partitions.entrySet().stream().collect( + Collectors.toMap( + entry -> entry.getKey().partition(), + entry -> new PartitionRegistration.Builder() + .setReplicas(entry.getValue().stream().mapToInt(Integer::intValue).toArray()) + .setDirectories(DirectoryId.unassignedArray(entry.getValue().size())) + .setIsr(entry.getValue().stream().mapToInt(Integer::intValue).toArray()) + .setLeader(entry.getValue().get(0)) + .setLeaderRecoveryState(LeaderRecoveryState.RECOVERED) + .setLeaderEpoch(0) + .setPartitionEpoch(0) + .build())); + return new TopicImage(topicName, topicId, partitionRegistrations); + } + + private UnifiedLog mockLog(TopicIdPartition topicIdPartition) { + UnifiedLog log = mock(UnifiedLog.class); + when(log.topicId()).thenReturn(Optional.of(topicIdPartition.topicId())); + when(log.topicPartition()).thenReturn(topicIdPartition.topicPartition()); + return log; + } +} diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/FetchFromLeaderWithCorruptedCheckpointTest.java b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/FetchFromLeaderWithCorruptedCheckpointTest.java index 73699bffac6d3..e82aeff7f8faf 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/FetchFromLeaderWithCorruptedCheckpointTest.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/FetchFromLeaderWithCorruptedCheckpointTest.java @@ -16,10 +16,10 @@ */ package org.apache.kafka.tiered.storage.integration; -import kafka.log.LogManager; import kafka.server.ReplicaManager; import org.apache.kafka.storage.internals.checkpoint.CleanShutdownFileHandler; +import org.apache.kafka.storage.internals.log.LogManager; import org.apache.kafka.tiered.storage.TieredStorageTestBuilder; import org.apache.kafka.tiered.storage.TieredStorageTestHarness; import org.apache.kafka.tiered.storage.specs.KeyValueSpec; @@ -50,7 +50,7 @@ protected void writeTestSpecifications(TieredStorageTestBuilder builder) { final Map> assignment = mkMap(mkEntry(p0, List.of(broker0, broker1))); final List checkpointFiles = List.of( ReplicaManager.HighWatermarkFilename(), - LogManager.RecoveryPointCheckpointFile(), + LogManager.RECOVERY_POINT_CHECKPOINT_FILE, CleanShutdownFileHandler.CLEAN_SHUTDOWN_FILE_NAME); builder.createTopic(topicA, partitionCount, replicationFactor, maxBatchCountPerSegment, assignment, From a04c2fed0474509e7818b8da35b60a524f1c0354 Mon Sep 17 00:00:00 2001 From: Jhen-Yung Hsu Date: Mon, 21 Apr 2025 20:19:42 +0800 Subject: [PATCH 03/37] KAFKA-19180 Fix the hanging testPendingTaskSize (#19526) The check for `scheduler.pendingTaskSize()` may fail if the thread pool is too slow to consume the runnable objects Reviewers: Ken Huang , PoAn Yang , Chia-Ping Tsai --- core/src/test/scala/unit/kafka/utils/SchedulerTest.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala index 24be1e921bc24..8518a469c7d78 100644 --- a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala +++ b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala @@ -220,10 +220,14 @@ class SchedulerTest { scheduler.scheduleOnce("task1", task1, 0) scheduler.scheduleOnce("task2", () => latch2.countDown(), 5) scheduler.scheduleOnce("task3", () => latch2.countDown(), 5) - assertEquals(2, scheduler.pendingTaskSize()) + retry(30000) { + assertEquals(2, scheduler.pendingTaskSize()) + } latch1.countDown() latch2.await() - assertEquals(0, scheduler.pendingTaskSize()) + retry(30000) { + assertEquals(0, scheduler.pendingTaskSize()) + } scheduler.shutdown() assertEquals(0, scheduler.pendingTaskSize()) } From ae771d73d119b5de94a6862422ad6bf3dcce292e Mon Sep 17 00:00:00 2001 From: Rich Chen <92559179+rich-c-shop@users.noreply.github.com> Date: Mon, 21 Apr 2025 09:01:55 -0400 Subject: [PATCH 04/37] KAFKA-8830 make Record Headers available in onAcknowledgement (#17099) Two sets of tests are added: 1. KafkaProducerTest - when send success, both record.headers() and onAcknowledgement headers are read only - when send failure, record.headers() is writable as before and onAcknowledgement headers is read only 2. ProducerInterceptorsTest - make both old and new onAcknowledgement method are called successfully Reviewers: Lianet Magrans , Omnia Ibrahim , Matthias J. Sax , Andrew Schofield , Chia-Ping Tsai --- .../kafka/clients/producer/KafkaProducer.java | 9 +- .../clients/producer/ProducerInterceptor.java | 30 +++++- .../internals/ProducerInterceptors.java | 23 +++-- .../header/internals/RecordHeaders.java | 4 + .../clients/producer/KafkaProducerTest.java | 54 ++++++++++- .../internals/ProducerInterceptorsTest.java | 94 +++++++++++++++---- 6 files changed, 185 insertions(+), 29 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 64a3af83e6571..16512c42d5fa5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -1546,6 +1546,7 @@ private class AppendCallbacks implements RecordAccumulator.AppendCallbacks { private final String recordLogString; private volatile int partition = RecordMetadata.UNKNOWN_PARTITION; private volatile TopicPartition topicPartition; + private final Headers headers; private AppendCallbacks(Callback userCallback, ProducerInterceptors interceptors, ProducerRecord record) { this.userCallback = userCallback; @@ -1554,6 +1555,12 @@ private AppendCallbacks(Callback userCallback, ProducerInterceptors interc // whole lifetime of the batch. // We don't want to have an NPE here, because the interceptors would not be notified (see .doSend). topic = record != null ? record.topic() : null; + if (record != null) { + headers = record.headers(); + } else { + headers = new RecordHeaders(); + ((RecordHeaders) headers).setReadOnly(); + } recordPartition = record != null ? record.partition() : null; recordLogString = log.isTraceEnabled() && record != null ? record.toString() : ""; } @@ -1563,7 +1570,7 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { if (metadata == null) { metadata = new RecordMetadata(topicPartition(), -1, -1, RecordBatch.NO_TIMESTAMP, -1, -1); } - this.interceptors.onAcknowledgement(metadata, exception); + this.interceptors.onAcknowledgement(metadata, exception, headers); if (this.userCallback != null) this.userCallback.onCompletion(metadata, exception); } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerInterceptor.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerInterceptor.java index 5bc4b2c2c8520..4a813dc96babc 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerInterceptor.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerInterceptor.java @@ -17,6 +17,7 @@ package org.apache.kafka.clients.producer; import org.apache.kafka.common.Configurable; +import org.apache.kafka.common.header.Headers; /** * A plugin interface that allows you to intercept (and possibly mutate) the records received by the producer before @@ -83,12 +84,37 @@ public interface ProducerInterceptor extends Configurable, AutoCloseable { * @param metadata The metadata for the record that was sent (i.e. the partition and offset). * If an error occurred, metadata will contain only valid topic and maybe * partition. If partition is not given in ProducerRecord and an error occurs - * before partition gets assigned, then partition will be set to RecordMetadata.NO_PARTITION. + * before partition gets assigned, then partition will be set to {@link RecordMetadata#UNKNOWN_PARTITION}. * The metadata may be null if the client passed null record to * {@link org.apache.kafka.clients.producer.KafkaProducer#send(ProducerRecord)}. * @param exception The exception thrown during processing of this record. Null if no error occurred. */ - void onAcknowledgement(RecordMetadata metadata, Exception exception); + default void onAcknowledgement(RecordMetadata metadata, Exception exception) {} + + /** + * This method is called when the record sent to the server has been acknowledged, or when sending the record fails before + * it gets sent to the server. + *

+ * This method is generally called just before the user callback is called, and in additional cases when KafkaProducer.send() + * throws an exception. + *

+ * Any exception thrown by this method will be ignored by the caller. + *

+ * This method will generally execute in the background I/O thread, so the implementation should be reasonably fast. + * Otherwise, sending of messages from other threads could be delayed. + * + * @param metadata The metadata for the record that was sent (i.e. the partition and offset). + * If an error occurred, metadata will contain only valid topic and maybe + * partition. If partition is not given in ProducerRecord and an error occurs + * before partition gets assigned, then partition will be set to {@link RecordMetadata#UNKNOWN_PARTITION}. + * The metadata may be null if the client passed null record to + * {@link org.apache.kafka.clients.producer.KafkaProducer#send(ProducerRecord)}. + * @param exception The exception thrown during processing of this record. Null if no error occurred. + * @param headers The headers for the record that was sent. It is read-only. + */ + default void onAcknowledgement(RecordMetadata metadata, Exception exception, Headers headers) { + onAcknowledgement(metadata, exception); + } /** * This is called when interceptor is closed diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerInterceptors.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerInterceptors.java index 9936eef760940..71d3839cedd12 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerInterceptors.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerInterceptors.java @@ -22,6 +22,8 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.internals.Plugin; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.record.RecordBatch; @@ -77,7 +79,7 @@ public ProducerRecord onSend(ProducerRecord record) { /** * This method is called when the record sent to the server has been acknowledged, or when sending the record fails before - * it gets sent to the server. This method calls {@link ProducerInterceptor#onAcknowledgement(RecordMetadata, Exception)} + * it gets sent to the server. This method calls {@link ProducerInterceptor#onAcknowledgement(RecordMetadata, Exception, Headers)} * method for each interceptor. * * This method does not throw exceptions. Exceptions thrown by any of interceptor methods are caught and ignored. @@ -85,11 +87,12 @@ public ProducerRecord onSend(ProducerRecord record) { * @param metadata The metadata for the record that was sent (i.e. the partition and offset). * If an error occurred, metadata will only contain valid topic and maybe partition. * @param exception The exception thrown during processing of this record. Null if no error occurred. + * @param headers The headers for the record that was sent */ - public void onAcknowledgement(RecordMetadata metadata, Exception exception) { + public void onAcknowledgement(RecordMetadata metadata, Exception exception, Headers headers) { for (Plugin> interceptorPlugin : this.interceptorPlugins) { try { - interceptorPlugin.get().onAcknowledgement(metadata, exception); + interceptorPlugin.get().onAcknowledgement(metadata, exception, headers); } catch (Exception e) { // do not propagate interceptor exceptions, just log log.warn("Error executing interceptor onAcknowledgement callback", e); @@ -99,7 +102,7 @@ public void onAcknowledgement(RecordMetadata metadata, Exception exception) { /** * This method is called when sending the record fails in {@link ProducerInterceptor#onSend - * (ProducerRecord)} method. This method calls {@link ProducerInterceptor#onAcknowledgement(RecordMetadata, Exception)} + * (ProducerRecord)} method. This method calls {@link ProducerInterceptor#onAcknowledgement(RecordMetadata, Exception, Headers)} * method for each interceptor * * @param record The record from client @@ -110,14 +113,22 @@ public void onAcknowledgement(RecordMetadata metadata, Exception exception) { public void onSendError(ProducerRecord record, TopicPartition interceptTopicPartition, Exception exception) { for (Plugin> interceptorPlugin : this.interceptorPlugins) { try { + Headers headers = record != null ? record.headers() : new RecordHeaders(); + if (headers instanceof RecordHeaders && !((RecordHeaders) headers).isReadOnly()) { + // make a copy of the headers to make sure we don't change the state of origin record's headers. + // original headers are still writable because client might want to mutate them before retrying. + RecordHeaders recordHeaders = (RecordHeaders) headers; + headers = new RecordHeaders(recordHeaders); + ((RecordHeaders) headers).setReadOnly(); + } if (record == null && interceptTopicPartition == null) { - interceptorPlugin.get().onAcknowledgement(null, exception); + interceptorPlugin.get().onAcknowledgement(null, exception, headers); } else { if (interceptTopicPartition == null) { interceptTopicPartition = extractTopicPartition(record); } interceptorPlugin.get().onAcknowledgement(new RecordMetadata(interceptTopicPartition, -1, -1, - RecordBatch.NO_TIMESTAMP, -1, -1), exception); + RecordBatch.NO_TIMESTAMP, -1, -1), exception, headers); } } catch (Exception e) { // do not propagate interceptor exceptions, just log diff --git a/clients/src/main/java/org/apache/kafka/common/header/internals/RecordHeaders.java b/clients/src/main/java/org/apache/kafka/common/header/internals/RecordHeaders.java index 52863c6c0b564..6311f6ca34825 100644 --- a/clients/src/main/java/org/apache/kafka/common/header/internals/RecordHeaders.java +++ b/clients/src/main/java/org/apache/kafka/common/header/internals/RecordHeaders.java @@ -108,6 +108,10 @@ public void setReadOnly() { this.isReadOnly = true; } + public boolean isReadOnly() { + return isReadOnly; + } + public Header[] toArray() { return headers.isEmpty() ? Record.EMPTY_HEADERS : headers.toArray(new Header[0]); } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java index fbb3484a03f7f..fb2f4f01282e3 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java @@ -48,7 +48,9 @@ import org.apache.kafka.common.errors.RecordTooLargeException; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; +import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.internals.RecordHeader; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.internals.ClusterResourceListeners; import org.apache.kafka.common.message.AddOffsetsToTxnResponseData; import org.apache.kafka.common.message.ApiVersionsResponseData; @@ -1084,13 +1086,14 @@ public void testTopicExpiryInMetadata() throws InterruptedException { @SuppressWarnings("unchecked") @Test - public void testHeaders() { + public void testHeadersSuccess() { doTestHeaders(Serializer.class); } private > void doTestHeaders(Class serializerClassToMock) { Map configs = new HashMap<>(); configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); + configs.put(ProducerConfig.INTERCEPTOR_CLASSES_CONFIG, ProducerInterceptorForHeaders.class.getName()); Serializer keySerializer = mock(serializerClassToMock); Serializer valueSerializer = mock(serializerClassToMock); @@ -1119,7 +1122,9 @@ private > void doTestHeaders(Class serializerCla producer.send(record, null); //ensure headers are closed and cannot be mutated post send - assertThrows(IllegalStateException.class, () -> record.headers().add(new RecordHeader("test", "test".getBytes()))); + RecordHeaders recordHeaders = (RecordHeaders) record.headers(); + assertTrue(recordHeaders.isReadOnly()); + assertThrows(IllegalStateException.class, () -> recordHeaders.add(new RecordHeader("test", "test".getBytes()))); //ensure existing headers are not changed, and last header for key is still original value assertArrayEquals(record.headers().lastHeader("test").value(), "header2".getBytes()); @@ -1130,6 +1135,28 @@ private > void doTestHeaders(Class serializerCla producer.close(Duration.ofMillis(0)); } + @Test + public void testHeadersFailure() { + Properties props = new Properties(); + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); + props.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, 5); + props.put(ProducerConfig.INTERCEPTOR_CLASSES_CONFIG, ProducerInterceptorForHeaders.class.getName()); + Serializer keySerializer = mock(StringSerializer.class); + Serializer valueSerializer = mock(StringSerializer.class); + + KafkaProducer producer = new KafkaProducer<>(props, keySerializer, valueSerializer); + ProducerRecord record = new ProducerRecord<>("topic", "key", "value"); + Future future = producer.send(record, (recordMetadata, exception) -> { }); + try { + TestUtils.assertFutureThrows(TimeoutException.class, future); + //ensure headers are writable if send failure + RecordHeaders recordHeaders = (RecordHeaders) record.headers(); + assertFalse(recordHeaders.isReadOnly()); + } finally { + producer.close(Duration.ofMillis(0)); + } + } + @Test public void closeShouldBeIdempotent() { Properties producerProps = new Properties(); @@ -2500,6 +2527,29 @@ public void configure(Map configs) { } } + public static class ProducerInterceptorForHeaders implements ProducerInterceptor { + + @Override + public ProducerRecord onSend(ProducerRecord record) { + return record; + } + + @Override + public void onAcknowledgement(RecordMetadata metadata, Exception exception, Headers headers) { + RecordHeaders recordHeaders = (RecordHeaders) headers; + // Ensure that the headers are read-only, no matter send success or send failure + assertTrue(recordHeaders.isReadOnly()); + } + + @Override + public void close() { + } + + @Override + public void configure(Map configs) { + } + } + public static class ProducerInterceptorForClientId implements ProducerInterceptor { @Override diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java index 853b27b255114..74f5db740628e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java @@ -22,6 +22,7 @@ import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.header.Headers; import org.junit.jupiter.api.Test; @@ -95,13 +96,68 @@ public void injectOnAcknowledgementError(boolean on) { } } + private class AppendNewProducerInterceptor implements ProducerInterceptor { + private final String appendStr; + private boolean throwExceptionOnSend = false; + private boolean throwExceptionOnAck = false; + + public AppendNewProducerInterceptor(String appendStr) { + this.appendStr = appendStr; + } + + @Override + public void configure(Map configs) { + } + + @Override + public ProducerRecord onSend(ProducerRecord record) { + onSendCount++; + if (throwExceptionOnSend) + throw new KafkaException("Injected exception in AppendNewProducerInterceptor.onSend"); + + return new ProducerRecord<>( + record.topic(), record.partition(), record.key(), record.value().concat(appendStr)); + } + + @Override + public void onAcknowledgement(RecordMetadata metadata, Exception exception, Headers headers) { + onAckCount++; + if (exception != null) { + onErrorAckCount++; + // the length check is just to call topic() method and let it throw an exception + // if RecordMetadata.TopicPartition is null + if (metadata != null && metadata.topic().length() >= 0) { + onErrorAckWithTopicSetCount++; + if (metadata.partition() >= 0) + onErrorAckWithTopicPartitionSetCount++; + } + } + if (throwExceptionOnAck) + throw new KafkaException("Injected exception in AppendNewProducerInterceptor.onAcknowledgement"); + } + + @Override + public void close() { + } + + // if 'on' is true, onSend will always throw an exception + public void injectOnSendError(boolean on) { + throwExceptionOnSend = on; + } + + // if 'on' is true, onAcknowledgement will always throw an exception + public void injectOnAcknowledgementError(boolean on) { + throwExceptionOnAck = on; + } + } + @Test public void testOnSendChain() { List> interceptorList = new ArrayList<>(); // we are testing two different interceptors by configuring the same interceptor differently, which is not // how it would be done in KafkaProducer, but ok for testing interceptor callbacks AppendProducerInterceptor interceptor1 = new AppendProducerInterceptor("One"); - AppendProducerInterceptor interceptor2 = new AppendProducerInterceptor("Two"); + AppendNewProducerInterceptor interceptor2 = new AppendNewProducerInterceptor("Two"); interceptorList.add(interceptor1); interceptorList.add(interceptor2); ProducerInterceptors interceptors = new ProducerInterceptors<>(interceptorList, null); @@ -139,23 +195,23 @@ public void testOnAcknowledgementChain() { // we are testing two different interceptors by configuring the same interceptor differently, which is not // how it would be done in KafkaProducer, but ok for testing interceptor callbacks AppendProducerInterceptor interceptor1 = new AppendProducerInterceptor("One"); - AppendProducerInterceptor interceptor2 = new AppendProducerInterceptor("Two"); + AppendNewProducerInterceptor interceptor2 = new AppendNewProducerInterceptor("Two"); interceptorList.add(interceptor1); interceptorList.add(interceptor2); ProducerInterceptors interceptors = new ProducerInterceptors<>(interceptorList, null); // verify onAck is called on all interceptors RecordMetadata meta = new RecordMetadata(tp, 0, 0, 0, 0, 0); - interceptors.onAcknowledgement(meta, null); + interceptors.onAcknowledgement(meta, null, null); assertEquals(2, onAckCount); // verify that onAcknowledgement exceptions do not propagate interceptor1.injectOnAcknowledgementError(true); - interceptors.onAcknowledgement(meta, null); + interceptors.onAcknowledgement(meta, null, null); assertEquals(4, onAckCount); interceptor2.injectOnAcknowledgementError(true); - interceptors.onAcknowledgement(meta, null); + interceptors.onAcknowledgement(meta, null, null); assertEquals(6, onAckCount); interceptors.close(); @@ -165,27 +221,29 @@ public void testOnAcknowledgementChain() { public void testOnAcknowledgementWithErrorChain() { List> interceptorList = new ArrayList<>(); AppendProducerInterceptor interceptor1 = new AppendProducerInterceptor("One"); + AppendNewProducerInterceptor interceptor2 = new AppendNewProducerInterceptor("Two"); interceptorList.add(interceptor1); + interceptorList.add(interceptor2); ProducerInterceptors interceptors = new ProducerInterceptors<>(interceptorList, null); // verify that metadata contains both topic and partition interceptors.onSendError(producerRecord, new TopicPartition(producerRecord.topic(), producerRecord.partition()), new KafkaException("Test")); - assertEquals(1, onErrorAckCount); - assertEquals(1, onErrorAckWithTopicPartitionSetCount); + assertEquals(2, onErrorAckCount); + assertEquals(2, onErrorAckWithTopicPartitionSetCount); // verify that metadata contains both topic and partition (because record already contains partition) interceptors.onSendError(producerRecord, null, new KafkaException("Test")); - assertEquals(2, onErrorAckCount); - assertEquals(2, onErrorAckWithTopicPartitionSetCount); + assertEquals(4, onErrorAckCount); + assertEquals(4, onErrorAckWithTopicPartitionSetCount); // if producer record does not contain partition, interceptor should get partition == -1 ProducerRecord record2 = new ProducerRecord<>("test2", null, 1, "value"); interceptors.onSendError(record2, null, new KafkaException("Test")); - assertEquals(3, onErrorAckCount); - assertEquals(3, onErrorAckWithTopicSetCount); - assertEquals(2, onErrorAckWithTopicPartitionSetCount); + assertEquals(6, onErrorAckCount); + assertEquals(6, onErrorAckWithTopicSetCount); + assertEquals(4, onErrorAckWithTopicPartitionSetCount); // if producer record does not contain partition, but topic/partition is passed to // onSendError, then interceptor should get valid partition @@ -193,15 +251,15 @@ public void testOnAcknowledgementWithErrorChain() { interceptors.onSendError(record2, new TopicPartition(record2.topic(), reassignedPartition), new KafkaException("Test")); - assertEquals(4, onErrorAckCount); - assertEquals(4, onErrorAckWithTopicSetCount); - assertEquals(3, onErrorAckWithTopicPartitionSetCount); + assertEquals(8, onErrorAckCount); + assertEquals(8, onErrorAckWithTopicSetCount); + assertEquals(6, onErrorAckWithTopicPartitionSetCount); // if both record and topic/partition are null, interceptor should not receive metadata interceptors.onSendError(null, null, new KafkaException("Test")); - assertEquals(5, onErrorAckCount); - assertEquals(4, onErrorAckWithTopicSetCount); - assertEquals(3, onErrorAckWithTopicPartitionSetCount); + assertEquals(10, onErrorAckCount); + assertEquals(8, onErrorAckWithTopicSetCount); + assertEquals(6, onErrorAckWithTopicPartitionSetCount); interceptors.close(); } From 18584b11ecbb364428da5d0316a23008ff6e8cd9 Mon Sep 17 00:00:00 2001 From: Xuan-Zhang Gong Date: Mon, 21 Apr 2025 21:22:56 +0800 Subject: [PATCH 05/37] MINOR: ignore null judgement in LogCleaner (#19524) about https://github.com/apache/kafka/pull/19387/files#r2052025917 Reviewers: PoAn Yang , Chia-Ping Tsai , TengYao Chi --- .../org/apache/kafka/storage/internals/log/LogCleaner.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogCleaner.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogCleaner.java index 2c0086c4700a1..939db06c83ea0 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogCleaner.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogCleaner.java @@ -556,15 +556,14 @@ private boolean cleanFilthiestLog() throws LogCleaningException { } else { // there's a log, clean it this.lastPreCleanStats = preCleanStats; - LogToClean cleanable = null; + LogToClean cleanable = ltc.get(); try { - cleanable = ltc.get(); cleanLog(cleanable); cleaned = true; } catch (ThreadShutdownException e) { throw e; } catch (Exception e) { - throw new LogCleaningException(cleanable != null ? cleanable.log() : null, e.getMessage(), e); + throw new LogCleaningException(cleanable.log(), e.getMessage(), e); } } From a8f49999ccf6a33b986bf895884f1a5e9708fb47 Mon Sep 17 00:00:00 2001 From: Abhinav Dixit Date: Tue, 22 Apr 2025 04:00:24 +0530 Subject: [PATCH 06/37] KAFKA-19019: Add support for remote storage fetch for share groups (#19437) This PR adds the support for remote storage fetch for share groups. There is a limitation in remote storage fetch for consumer groups that we can only perform remote fetch for a single topic partition in a fetch request. Since, the logic of share fetch requests is largely based on how consumer groups work, we are following similar logic in implementing remote storage fetch. However, this problem should be addressed as part of KAFKA-19133 which should help us perform fetch for multiple remote fetch topic partition in a single share fetch request. Reviewers: Jun Rao --- .../kafka/server/share/DelayedShareFetch.java | 466 ++++++++++++-- .../server/share/DelayedShareFetchTest.java | 573 +++++++++++++++++- 2 files changed, 977 insertions(+), 62 deletions(-) diff --git a/core/src/main/java/kafka/server/share/DelayedShareFetch.java b/core/src/main/java/kafka/server/share/DelayedShareFetch.java index d68ed06d3070d..5bd5f4ea6aa55 100644 --- a/core/src/main/java/kafka/server/share/DelayedShareFetch.java +++ b/core/src/main/java/kafka/server/share/DelayedShareFetch.java @@ -22,6 +22,10 @@ import kafka.server.ReplicaManager; import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.errors.KafkaStorageException; +import org.apache.kafka.common.errors.NotLeaderOrFollowerException; +import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; +import org.apache.kafka.common.message.ShareFetchResponseData; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.utils.Time; @@ -34,8 +38,12 @@ import org.apache.kafka.server.share.fetch.ShareFetchPartitionData; import org.apache.kafka.server.share.metrics.ShareGroupMetrics; import org.apache.kafka.server.storage.log.FetchIsolation; +import org.apache.kafka.server.storage.log.FetchPartitionData; +import org.apache.kafka.storage.internals.log.FetchDataInfo; import org.apache.kafka.storage.internals.log.LogOffsetMetadata; import org.apache.kafka.storage.internals.log.LogOffsetSnapshot; +import org.apache.kafka.storage.internals.log.RemoteLogReadResult; +import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo; import com.yammer.metrics.core.Meter; @@ -44,10 +52,16 @@ import java.util.ArrayList; import java.util.LinkedHashMap; +import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.OptionalInt; +import java.util.OptionalLong; import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.Lock; import java.util.function.BiConsumer; @@ -83,7 +97,9 @@ public class DelayedShareFetch extends DelayedOperation { // Tracks the start time to acquire any share partition for a fetch request. private long acquireStartTimeMs; private LinkedHashMap partitionsAcquired; - private LinkedHashMap partitionsAlreadyFetched; + private LinkedHashMap localPartitionsAlreadyFetched; + private Optional remoteFetchOpt; + private Optional remoteStorageFetchException; /** * This function constructs an instance of delayed share fetch operation for completing share fetch @@ -110,10 +126,24 @@ public DelayedShareFetch( sharePartitions, PartitionMaxBytesStrategy.type(PartitionMaxBytesStrategy.StrategyType.UNIFORM), shareGroupMetrics, - time + time, + Optional.empty() ); } + /** + * This function constructs an instance of delayed share fetch operation for completing share fetch + * requests instantaneously or with delay. The direct usage of this constructor is only from tests. + * + * @param shareFetch The share fetch parameters of the share fetch request. + * @param replicaManager The replica manager instance used to read from log/complete the request. + * @param exceptionHandler The handler to complete share fetch requests with exception. + * @param sharePartitions The share partitions referenced in the share fetch request. + * @param partitionMaxBytesStrategy The strategy to identify the max bytes for topic partitions in the share fetch request. + * @param shareGroupMetrics The share group metrics to record the metrics. + * @param time The system time. + * @param remoteFetchOpt Optional containing an in-flight remote fetch object or an empty optional. + */ DelayedShareFetch( ShareFetch shareFetch, ReplicaManager replicaManager, @@ -121,19 +151,22 @@ public DelayedShareFetch( LinkedHashMap sharePartitions, PartitionMaxBytesStrategy partitionMaxBytesStrategy, ShareGroupMetrics shareGroupMetrics, - Time time + Time time, + Optional remoteFetchOpt ) { super(shareFetch.fetchParams().maxWaitMs, Optional.empty()); this.shareFetch = shareFetch; this.replicaManager = replicaManager; this.partitionsAcquired = new LinkedHashMap<>(); - this.partitionsAlreadyFetched = new LinkedHashMap<>(); + this.localPartitionsAlreadyFetched = new LinkedHashMap<>(); this.exceptionHandler = exceptionHandler; this.sharePartitions = sharePartitions; this.partitionMaxBytesStrategy = partitionMaxBytesStrategy; this.shareGroupMetrics = shareGroupMetrics; this.time = time; this.acquireStartTimeMs = time.hiResClockMs(); + this.remoteFetchOpt = remoteFetchOpt; + this.remoteStorageFetchException = Optional.empty(); // Register metrics for DelayedShareFetch. KafkaMetricsGroup metricsGroup = new KafkaMetricsGroup("kafka.server", "DelayedShareFetchMetrics"); this.expiredRequestMeter = metricsGroup.newMeter(EXPIRES_PER_SEC, "requests", TimeUnit.SECONDS); @@ -152,58 +185,68 @@ public void onExpiration() { @Override public void onComplete() { // We are utilizing lock so that onComplete doesn't do a dirty read for instance variables - - // partitionsAcquired and partitionsAlreadyFetched, since these variables can get updated in a different tryComplete thread. + // partitionsAcquired and localPartitionsAlreadyFetched, since these variables can get updated in a different tryComplete thread. lock.lock(); log.trace("Completing the delayed share fetch request for group {}, member {}, " + "topic partitions {}", shareFetch.groupId(), shareFetch.memberId(), partitionsAcquired.keySet()); try { - LinkedHashMap topicPartitionData; - // tryComplete did not invoke forceComplete, so we need to check if we have any partitions to fetch. - if (partitionsAcquired.isEmpty()) { - topicPartitionData = acquirablePartitions(); - // The TopicPartitionsAcquireTimeMs metric signifies the tension when acquiring the locks - // for the share partition, hence if no partitions are yet acquired by tryComplete, - // we record the metric here. Do not check if the request has successfully acquired any - // partitions now or not, as then the upper bound of request timeout shall be recorded - // for the metric. - updateAcquireElapsedTimeMetric(); - } else { - // tryComplete invoked forceComplete, so we can use the data from tryComplete. - topicPartitionData = partitionsAcquired; - } - - if (topicPartitionData.isEmpty()) { - // No locks for share partitions could be acquired, so we complete the request with an empty response. - shareGroupMetrics.recordTopicPartitionsFetchRatio(shareFetch.groupId(), 0); - shareFetch.maybeComplete(Map.of()); - return; + if (remoteStorageFetchException.isPresent()) { + completeErroneousRemoteShareFetchRequest(); + } else if (remoteFetchOpt.isPresent()) { + completeRemoteStorageShareFetchRequest(); } else { - // Update metric to record acquired to requested partitions. - double requestTopicToAcquired = (double) topicPartitionData.size() / shareFetch.topicIdPartitions().size(); - shareGroupMetrics.recordTopicPartitionsFetchRatio(shareFetch.groupId(), (int) (requestTopicToAcquired * 100)); + completeLocalLogShareFetchRequest(); } - log.trace("Fetchable share partitions data: {} with groupId: {} fetch params: {}", - topicPartitionData, shareFetch.groupId(), shareFetch.fetchParams()); - - completeShareFetchRequest(topicPartitionData); } finally { lock.unlock(); } } - private void completeShareFetchRequest(LinkedHashMap topicPartitionData) { + private void completeLocalLogShareFetchRequest() { + LinkedHashMap topicPartitionData; + // tryComplete did not invoke forceComplete, so we need to check if we have any partitions to fetch. + if (partitionsAcquired.isEmpty()) { + topicPartitionData = acquirablePartitions(sharePartitions); + // The TopicPartitionsAcquireTimeMs metric signifies the tension when acquiring the locks + // for the share partition, hence if no partitions are yet acquired by tryComplete, + // we record the metric here. Do not check if the request has successfully acquired any + // partitions now or not, as then the upper bound of request timeout shall be recorded + // for the metric. + updateAcquireElapsedTimeMetric(); + } else { + // tryComplete invoked forceComplete, so we can use the data from tryComplete. + topicPartitionData = partitionsAcquired; + } + + if (topicPartitionData.isEmpty()) { + // No locks for share partitions could be acquired, so we complete the request with an empty response. + shareGroupMetrics.recordTopicPartitionsFetchRatio(shareFetch.groupId(), 0); + shareFetch.maybeComplete(Map.of()); + return; + } else { + // Update metric to record acquired to requested partitions. + double requestTopicToAcquired = (double) topicPartitionData.size() / shareFetch.topicIdPartitions().size(); + shareGroupMetrics.recordTopicPartitionsFetchRatio(shareFetch.groupId(), (int) (requestTopicToAcquired * 100)); + } + log.trace("Fetchable share partitions data: {} with groupId: {} fetch params: {}", + topicPartitionData, shareFetch.groupId(), shareFetch.fetchParams()); + + processAcquiredTopicPartitionsForLocalLogFetch(topicPartitionData); + } + + private void processAcquiredTopicPartitionsForLocalLogFetch(LinkedHashMap topicPartitionData) { try { LinkedHashMap responseData; - if (partitionsAlreadyFetched.isEmpty()) + if (localPartitionsAlreadyFetched.isEmpty()) responseData = readFromLog( topicPartitionData, partitionMaxBytesStrategy.maxBytes(shareFetch.fetchParams().maxBytes, topicPartitionData.keySet(), topicPartitionData.size())); else // There shouldn't be a case when we have a partitionsAlreadyFetched value here and this variable is getting // updated in a different tryComplete thread. - responseData = combineLogReadResponse(topicPartitionData, partitionsAlreadyFetched); + responseData = combineLogReadResponse(topicPartitionData, localPartitionsAlreadyFetched); List shareFetchPartitionDataList = new ArrayList<>(); responseData.forEach((topicIdPartition, logReadResult) -> @@ -225,15 +268,7 @@ private void completeShareFetchRequest(LinkedHashMap top log.error("Error processing delayed share fetch request", e); handleFetchException(shareFetch, topicPartitionData.keySet(), e); } finally { - // Releasing the lock to move ahead with the next request in queue. - releasePartitionLocks(topicPartitionData.keySet()); - // If we have a fetch request completed for a topic-partition, we release the locks for that partition, - // then we should check if there is a pending share fetch request for the topic-partition and complete it. - // We add the action to delayed actions queue to avoid an infinite call stack, which could happen if - // we directly call delayedShareFetchPurgatory.checkAndComplete - replicaManager.addToActionQueue(() -> topicPartitionData.keySet().forEach(topicIdPartition -> - replicaManager.completeDelayedShareFetchRequest( - new DelayedShareFetchGroupKey(shareFetch.groupId(), topicIdPartition.topicId(), topicIdPartition.partition())))); + releasePartitionLocksAndAddToActionQueue(topicPartitionData.keySet()); } } @@ -242,8 +277,12 @@ private void completeShareFetchRequest(LinkedHashMap top */ @Override public boolean tryComplete() { - LinkedHashMap topicPartitionData = acquirablePartitions(); + // Check to see if the remote fetch is in flight. If there is an in flight remote fetch we want to resolve it first. + if (remoteFetchOpt.isPresent()) { + return maybeCompletePendingRemoteFetch(); + } + LinkedHashMap topicPartitionData = acquirablePartitions(sharePartitions); try { if (!topicPartitionData.isEmpty()) { // Update the metric to record the time taken to acquire the locks for the share partitions. @@ -252,17 +291,17 @@ public boolean tryComplete() { // replicaManager.readFromLog to populate the offset metadata and update the fetch offset metadata for // those topic partitions. LinkedHashMap replicaManagerReadResponse = maybeReadFromLog(topicPartitionData); + // Store the remote fetch info and the topic partition for which we need to perform remote fetch. + Optional topicPartitionRemoteFetchInfoOpt = maybePrepareRemoteStorageFetchInfo(topicPartitionData, replicaManagerReadResponse); + + if (topicPartitionRemoteFetchInfoOpt.isPresent()) { + return maybeProcessRemoteFetch(topicPartitionData, topicPartitionRemoteFetchInfoOpt.get()); + } maybeUpdateFetchOffsetMetadata(topicPartitionData, replicaManagerReadResponse); if (anyPartitionHasLogReadError(replicaManagerReadResponse) || isMinBytesSatisfied(topicPartitionData, partitionMaxBytesStrategy.maxBytes(shareFetch.fetchParams().maxBytes, topicPartitionData.keySet(), topicPartitionData.size()))) { partitionsAcquired = topicPartitionData; - partitionsAlreadyFetched = replicaManagerReadResponse; - boolean completedByMe = forceComplete(); - // If invocation of forceComplete is not successful, then that means the request is already completed - // hence release the acquired locks. - if (!completedByMe) { - releasePartitionLocks(partitionsAcquired.keySet()); - } - return completedByMe; + localPartitionsAlreadyFetched = replicaManagerReadResponse; + return forceCompleteRequest(); } else { log.debug("minBytes is not satisfied for the share fetch request for group {}, member {}, " + "topic partitions {}", shareFetch.groupId(), shareFetch.memberId(), @@ -277,10 +316,18 @@ public boolean tryComplete() { return false; } catch (Exception e) { log.error("Error processing delayed share fetch request", e); - releasePartitionLocks(topicPartitionData.keySet()); - partitionsAcquired.clear(); - partitionsAlreadyFetched.clear(); - return forceComplete(); + // In case we have a remote fetch exception, we have already released locks for partitions which have potential + // local log read. We do not release locks for partitions which have a remote storage read because we need to + // complete the share fetch request in onComplete and if we release the locks early here, some other DelayedShareFetch + // request might get the locks for those partitions without this one getting complete. + if (remoteStorageFetchException.isEmpty()) { + releasePartitionLocks(topicPartitionData.keySet()); + partitionsAcquired.clear(); + localPartitionsAlreadyFetched.clear(); + return forceCompleteRequest(); + } else { + return forceCompleteRequest(); + } } } @@ -288,11 +335,13 @@ public boolean tryComplete() { * Prepare fetch request structure for partitions in the share fetch request for which we can acquire records. */ // Visible for testing - LinkedHashMap acquirablePartitions() { + LinkedHashMap acquirablePartitions( + LinkedHashMap sharePartitionsForAcquire + ) { // Initialize the topic partitions for which the fetch should be attempted. LinkedHashMap topicPartitionData = new LinkedHashMap<>(); - sharePartitions.forEach((topicIdPartition, sharePartition) -> { + sharePartitionsForAcquire.forEach((topicIdPartition, sharePartition) -> { // Add the share partition to the list of partitions to be fetched only if we can // acquire the fetch lock on it. if (sharePartition.maybeAcquireFetchLock()) { @@ -529,8 +578,307 @@ Lock lock() { return lock; } + // Visible for testing. + RemoteFetch remoteFetch() { + return remoteFetchOpt.orElse(null); + } + // Visible for testing. Meter expiredRequestMeter() { return expiredRequestMeter; } + + private Optional maybePrepareRemoteStorageFetchInfo( + LinkedHashMap topicPartitionData, + LinkedHashMap replicaManagerReadResponse + ) { + Optional topicPartitionRemoteFetchInfoOpt = Optional.empty(); + for (Map.Entry entry : replicaManagerReadResponse.entrySet()) { + TopicIdPartition topicIdPartition = entry.getKey(); + LogReadResult logReadResult = entry.getValue(); + if (logReadResult.info().delayedRemoteStorageFetch.isPresent()) { + // TODO: There is a limitation in remote storage fetch for consumer groups that we can only perform remote fetch for + // a single topic partition in a fetch request. Since, the logic of fetch is largely based on how consumer groups work, + // we are following the same logic. However, this problem should be addressed as part of KAFKA-19133 which should help us perform + // fetch for multiple remote fetch topic partition in a single share fetch request + topicPartitionRemoteFetchInfoOpt = Optional.of(new TopicPartitionRemoteFetchInfo(topicIdPartition, logReadResult)); + partitionsAcquired.put(topicIdPartition, topicPartitionData.get(topicIdPartition)); + break; + } + } + return topicPartitionRemoteFetchInfoOpt; + } + + private boolean maybeProcessRemoteFetch( + LinkedHashMap topicPartitionData, + TopicPartitionRemoteFetchInfo topicPartitionRemoteFetchInfo + ) { + Set nonRemoteFetchTopicPartitions = new LinkedHashSet<>(); + topicPartitionData.keySet().forEach(topicIdPartition -> { + // topic partitions for which fetch would not be happening in this share fetch request. + if (!topicPartitionRemoteFetchInfo.topicIdPartition().equals(topicIdPartition)) { + nonRemoteFetchTopicPartitions.add(topicIdPartition); + } + }); + // Release fetch lock for the topic partitions that were acquired but were not a part of remote fetch and add + // them to the delayed actions queue. + releasePartitionLocksAndAddToActionQueue(nonRemoteFetchTopicPartitions); + processRemoteFetchOrException(topicPartitionRemoteFetchInfo); + // Check if remote fetch can be completed. + return maybeCompletePendingRemoteFetch(); + } + + /** + * Throws an exception if a task for remote storage fetch could not be scheduled successfully else updates remoteFetchOpt. + * @param topicPartitionRemoteFetchInfo - The remote storage fetch information. + */ + private void processRemoteFetchOrException( + TopicPartitionRemoteFetchInfo topicPartitionRemoteFetchInfo + ) { + TopicIdPartition remoteFetchTopicIdPartition = topicPartitionRemoteFetchInfo.topicIdPartition(); + RemoteStorageFetchInfo remoteStorageFetchInfo = topicPartitionRemoteFetchInfo.logReadResult().info().delayedRemoteStorageFetch.get(); + + Future remoteFetchTask; + CompletableFuture remoteFetchResult = new CompletableFuture<>(); + try { + remoteFetchTask = replicaManager.remoteLogManager().get().asyncRead( + remoteStorageFetchInfo, + result -> { + remoteFetchResult.complete(result); + replicaManager.completeDelayedShareFetchRequest(new DelayedShareFetchGroupKey(shareFetch.groupId(), remoteFetchTopicIdPartition.topicId(), remoteFetchTopicIdPartition.partition())); + } + ); + } catch (Exception e) { + // Throw the error if any in scheduling the remote fetch task. + remoteStorageFetchException = Optional.of(e); + throw e; + } + remoteFetchOpt = Optional.of(new RemoteFetch(remoteFetchTopicIdPartition, topicPartitionRemoteFetchInfo.logReadResult(), remoteFetchTask, remoteFetchResult, remoteStorageFetchInfo)); + } + + /** + * This function checks if the remote fetch can be completed or not. It should always be called once you confirm remoteFetchOpt.isPresent(). + * The operation can be completed if: + * Case a: The partition is in an offline log directory on this broker + * Case b: This broker does not know the partition it tries to fetch + * Case c: This broker is no longer the leader of the partition it tries to fetch + * Case d: The remote storage read request completed (succeeded or failed) + * @return boolean representing whether the remote fetch is completed or not. + */ + private boolean maybeCompletePendingRemoteFetch() { + boolean canComplete = false; + + TopicIdPartition topicIdPartition = remoteFetchOpt.get().topicIdPartition(); + try { + replicaManager.getPartitionOrException(topicIdPartition.topicPartition()); + } catch (KafkaStorageException e) { // Case a + log.debug("TopicPartition {} is in an offline log directory, satisfy {} immediately", topicIdPartition, shareFetch.fetchParams()); + canComplete = true; + } catch (UnknownTopicOrPartitionException e) { // Case b + log.debug("Broker no longer knows of topicPartition {}, satisfy {} immediately", topicIdPartition, shareFetch.fetchParams()); + canComplete = true; + } catch (NotLeaderOrFollowerException e) { // Case c + log.debug("Broker is no longer the leader or follower of topicPartition {}, satisfy {} immediately", topicIdPartition, shareFetch.fetchParams()); + canComplete = true; + } + + if (canComplete || remoteFetchOpt.get().remoteFetchResult().isDone()) { // Case d + return forceCompleteRequest(); + } else + return false; + } + + /** + * This function completes a share fetch request for which we have identified erroneous remote storage fetch in tryComplete() + * It should only be called when we know that there is remote fetch in-flight/completed. + */ + private void completeErroneousRemoteShareFetchRequest() { + try { + handleFetchException(shareFetch, partitionsAcquired.keySet(), remoteStorageFetchException.get()); + } finally { + releasePartitionLocksAndAddToActionQueue(partitionsAcquired.keySet()); + } + + } + + private void releasePartitionLocksAndAddToActionQueue(Set topicIdPartitions) { + if (topicIdPartitions.isEmpty()) { + return; + } + // Releasing the lock to move ahead with the next request in queue. + releasePartitionLocks(topicIdPartitions); + // If we have a fetch request completed for a topic-partition, we release the locks for that partition, + // then we should check if there is a pending share fetch request for the topic-partition and complete it. + // We add the action to delayed actions queue to avoid an infinite call stack, which could happen if + // we directly call delayedShareFetchPurgatory.checkAndComplete + replicaManager.addToActionQueue(() -> topicIdPartitions.forEach(topicIdPartition -> + replicaManager.completeDelayedShareFetchRequest( + new DelayedShareFetchGroupKey(shareFetch.groupId(), topicIdPartition.topicId(), topicIdPartition.partition())))); + } + + /** + * This function completes a share fetch request for which we have identified remoteFetch during tryComplete() + * Note - This function should only be called when we know that there is remote fetch. + */ + private void completeRemoteStorageShareFetchRequest() { + LinkedHashMap acquiredNonRemoteFetchTopicPartitionData = new LinkedHashMap<>(); + try { + List shareFetchPartitionData = new ArrayList<>(); + int readableBytes = 0; + if (remoteFetchOpt.get().remoteFetchResult().isDone()) { + RemoteFetch remoteFetch = remoteFetchOpt.get(); + RemoteLogReadResult remoteLogReadResult = remoteFetch.remoteFetchResult().get(); + if (remoteLogReadResult.error.isPresent()) { + Throwable error = remoteLogReadResult.error.get(); + // If there is any error for the remote fetch topic partition, we populate the error accordingly. + shareFetchPartitionData.add( + new ShareFetchPartitionData( + remoteFetch.topicIdPartition(), + partitionsAcquired.get(remoteFetch.topicIdPartition()), + ReplicaManager.createLogReadResult(error).toFetchPartitionData(false) + ) + ); + } else { + FetchDataInfo info = remoteLogReadResult.fetchDataInfo.get(); + TopicIdPartition topicIdPartition = remoteFetch.topicIdPartition(); + LogReadResult logReadResult = remoteFetch.logReadResult(); + shareFetchPartitionData.add( + new ShareFetchPartitionData( + topicIdPartition, + partitionsAcquired.get(remoteFetch.topicIdPartition()), + new FetchPartitionData( + logReadResult.error(), + logReadResult.highWatermark(), + logReadResult.leaderLogStartOffset(), + info.records, + Optional.empty(), + logReadResult.lastStableOffset().isDefined() ? OptionalLong.of((Long) logReadResult.lastStableOffset().get()) : OptionalLong.empty(), + info.abortedTransactions, + logReadResult.preferredReadReplica().isDefined() ? OptionalInt.of((Integer) logReadResult.preferredReadReplica().get()) : OptionalInt.empty(), + false + ) + ) + ); + readableBytes += info.records.sizeInBytes(); + } + } else { + cancelRemoteFetchTask(); + } + + // If remote fetch bytes < shareFetch.fetchParams().maxBytes, then we will try for a local read. + if (readableBytes < shareFetch.fetchParams().maxBytes) { + // Get the local log read based topic partitions. + LinkedHashMap nonRemoteFetchSharePartitions = new LinkedHashMap<>(); + sharePartitions.forEach((topicIdPartition, sharePartition) -> { + if (!partitionsAcquired.containsKey(topicIdPartition)) { + nonRemoteFetchSharePartitions.put(topicIdPartition, sharePartition); + } + }); + acquiredNonRemoteFetchTopicPartitionData = acquirablePartitions(nonRemoteFetchSharePartitions); + if (!acquiredNonRemoteFetchTopicPartitionData.isEmpty()) { + log.trace("Fetchable local share partitions for a remote share fetch request data: {} with groupId: {} fetch params: {}", + acquiredNonRemoteFetchTopicPartitionData, shareFetch.groupId(), shareFetch.fetchParams()); + + LinkedHashMap responseData = readFromLog( + acquiredNonRemoteFetchTopicPartitionData, + partitionMaxBytesStrategy.maxBytes(shareFetch.fetchParams().maxBytes - readableBytes, acquiredNonRemoteFetchTopicPartitionData.keySet(), acquiredNonRemoteFetchTopicPartitionData.size())); + for (Map.Entry entry : responseData.entrySet()) { + if (entry.getValue().info().delayedRemoteStorageFetch.isEmpty()) { + shareFetchPartitionData.add( + new ShareFetchPartitionData( + entry.getKey(), + acquiredNonRemoteFetchTopicPartitionData.get(entry.getKey()), + entry.getValue().toFetchPartitionData(false) + ) + ); + } + } + } + } + + // Update metric to record acquired to requested partitions. + double acquiredRatio = (double) (partitionsAcquired.size() + acquiredNonRemoteFetchTopicPartitionData.size()) / shareFetch.topicIdPartitions().size(); + if (acquiredRatio > 0) + shareGroupMetrics.recordTopicPartitionsFetchRatio(shareFetch.groupId(), (int) (acquiredRatio * 100)); + + Map remoteFetchResponse = ShareFetchUtils.processFetchResponse( + shareFetch, shareFetchPartitionData, sharePartitions, replicaManager, exceptionHandler); + shareFetch.maybeComplete(remoteFetchResponse); + log.trace("Remote share fetch request completed successfully, response: {}", remoteFetchResponse); + } catch (InterruptedException | ExecutionException e) { + log.error("Exception occurred in completing remote fetch {} for delayed share fetch request {}", remoteFetchOpt.get(), e); + handleExceptionInCompletingRemoteStorageShareFetchRequest(acquiredNonRemoteFetchTopicPartitionData.keySet(), e); + } catch (Exception e) { + log.error("Unexpected error in processing delayed share fetch request", e); + handleExceptionInCompletingRemoteStorageShareFetchRequest(acquiredNonRemoteFetchTopicPartitionData.keySet(), e); + } finally { + Set topicIdPartitions = new LinkedHashSet<>(partitionsAcquired.keySet()); + topicIdPartitions.addAll(acquiredNonRemoteFetchTopicPartitionData.keySet()); + releasePartitionLocksAndAddToActionQueue(topicIdPartitions); + } + } + + private void handleExceptionInCompletingRemoteStorageShareFetchRequest( + Set acquiredNonRemoteFetchTopicPartitions, + Exception e + ) { + Set topicIdPartitions = new LinkedHashSet<>(partitionsAcquired.keySet()); + topicIdPartitions.addAll(acquiredNonRemoteFetchTopicPartitions); + handleFetchException(shareFetch, topicIdPartitions, e); + } + + /** + * Cancel the remote storage read task, if it has not been executed yet and avoid interrupting the task if it is + * already running as it may force closing opened/cached resources as transaction index. + * Note - This function should only be called when we know that there is remote fetch. + */ + private void cancelRemoteFetchTask() { + boolean cancelled = remoteFetchOpt.get().remoteFetchTask().cancel(false); + if (!cancelled) { + log.debug("Remote fetch task for RemoteStorageFetchInfo: {} could not be cancelled and its isDone value is {}", + remoteFetchOpt.get().remoteFetchInfo(), remoteFetchOpt.get().remoteFetchTask().isDone()); + } + } + + private boolean forceCompleteRequest() { + boolean completedByMe = forceComplete(); + // If the delayed operation is completed by me, the acquired locks are already released in onComplete(). + // Otherwise, we need to release the acquired locks. + if (!completedByMe) { + releasePartitionLocksAndAddToActionQueue(partitionsAcquired.keySet()); + } + return completedByMe; + } + + public record RemoteFetch( + TopicIdPartition topicIdPartition, + LogReadResult logReadResult, + Future remoteFetchTask, + CompletableFuture remoteFetchResult, + RemoteStorageFetchInfo remoteFetchInfo + ) { + @Override + public String toString() { + return "RemoteFetch(" + + "topicIdPartition=" + topicIdPartition + + ", logReadResult=" + logReadResult + + ", remoteFetchTask=" + remoteFetchTask + + ", remoteFetchResult=" + remoteFetchResult + + ", remoteFetchInfo=" + remoteFetchInfo + + ")"; + } + } + + public record TopicPartitionRemoteFetchInfo( + TopicIdPartition topicIdPartition, + LogReadResult logReadResult + ) { + @Override + public String toString() { + return "TopicPartitionRemoteFetchInfo(" + + "topicIdPartition=" + topicIdPartition + + ", logReadResult=" + logReadResult + + ")"; + } + } } diff --git a/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java b/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java index bb8b51b40e297..43ece70ca0ee6 100644 --- a/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java +++ b/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java @@ -25,10 +25,15 @@ import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.errors.KafkaStorageException; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.message.ShareFetchResponseData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Records; import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.server.log.remote.storage.RemoteLogManager; import org.apache.kafka.server.purgatory.DelayedOperationKey; import org.apache.kafka.server.purgatory.DelayedOperationPurgatory; import org.apache.kafka.server.share.SharePartitionKey; @@ -46,6 +51,8 @@ import org.apache.kafka.storage.internals.log.FetchDataInfo; import org.apache.kafka.storage.internals.log.LogOffsetMetadata; import org.apache.kafka.storage.internals.log.LogOffsetSnapshot; +import org.apache.kafka.storage.internals.log.RemoteLogReadResult; +import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo; import org.apache.kafka.storage.log.metrics.BrokerTopicStats; import org.junit.jupiter.api.AfterEach; @@ -61,10 +68,15 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Future; +import java.util.concurrent.RejectedExecutionException; import java.util.function.BiConsumer; +import java.util.function.Consumer; import java.util.stream.Collectors; +import scala.Option; import scala.Tuple2; +import scala.collection.Seq; import scala.jdk.javaapi.CollectionConverters; import static kafka.server.share.SharePartitionManagerTest.DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL; @@ -73,6 +85,7 @@ import static org.apache.kafka.server.share.fetch.ShareFetchTestUtils.createShareAcquiredRecords; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; @@ -95,6 +108,8 @@ public class DelayedShareFetchTest { private static final FetchParams FETCH_PARAMS = new FetchParams( FetchRequest.ORDINARY_CONSUMER_ID, -1, MAX_WAIT_MS, 1, 1024 * 1024, FetchIsolation.HIGH_WATERMARK, Optional.empty(), true); + private static final FetchDataInfo REMOTE_FETCH_INFO = new FetchDataInfo(new LogOffsetMetadata(0, 0, 0), + MemoryRecords.EMPTY, false, Optional.empty(), Optional.of(mock(RemoteStorageFetchInfo.class))); private static final BrokerTopicStats BROKER_TOPIC_STATS = new BrokerTopicStats(); private Timer mockTimer; @@ -487,7 +502,7 @@ public void testToCompleteAnAlreadyCompletedFuture() { delayedShareFetch.forceComplete(); assertTrue(delayedShareFetch.isCompleted()); // Verifying that the first forceComplete calls acquirablePartitions method in DelayedShareFetch. - Mockito.verify(delayedShareFetch, times(1)).acquirablePartitions(); + Mockito.verify(delayedShareFetch, times(1)).acquirablePartitions(sharePartitions); assertEquals(0, future.join().size()); assertTrue(delayedShareFetch.lock().tryLock()); delayedShareFetch.lock().unlock(); @@ -497,7 +512,7 @@ public void testToCompleteAnAlreadyCompletedFuture() { delayedShareFetch.forceComplete(); assertTrue(delayedShareFetch.isCompleted()); // Verifying that the second forceComplete does not call acquirablePartitions method in DelayedShareFetch. - Mockito.verify(delayedShareFetch, times(1)).acquirablePartitions(); + Mockito.verify(delayedShareFetch, times(1)).acquirablePartitions(sharePartitions); Mockito.verify(delayedShareFetch, times(0)).releasePartitionLocks(any()); assertTrue(delayedShareFetch.lock().tryLock()); // Assert both metrics shall be recorded only once. @@ -1155,6 +1170,525 @@ public void testOnCompleteExecutionOnTimeout() { assertEquals(1, delayedShareFetch.expiredRequestMeter().count()); } + @Test + public void testRemoteStorageFetchTryCompleteReturnsFalse() { + ReplicaManager replicaManager = mock(ReplicaManager.class); + TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); + TopicIdPartition tp1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 1)); + TopicIdPartition tp2 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 2)); + + SharePartition sp0 = mock(SharePartition.class); + SharePartition sp1 = mock(SharePartition.class); + SharePartition sp2 = mock(SharePartition.class); + + // All the topic partitions are acquirable. + when(sp0.maybeAcquireFetchLock()).thenReturn(true); + when(sp1.maybeAcquireFetchLock()).thenReturn(true); + when(sp2.maybeAcquireFetchLock()).thenReturn(true); + when(sp0.canAcquireRecords()).thenReturn(true); + when(sp1.canAcquireRecords()).thenReturn(true); + when(sp2.canAcquireRecords()).thenReturn(true); + + LinkedHashMap sharePartitions = new LinkedHashMap<>(); + sharePartitions.put(tp0, sp0); + sharePartitions.put(tp1, sp1); + sharePartitions.put(tp2, sp2); + + ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, "grp", Uuid.randomUuid().toString(), + new CompletableFuture<>(), List.of(tp0, tp1, tp2), BATCH_SIZE, MAX_FETCH_RECORDS, + BROKER_TOPIC_STATS); + + when(sp0.nextFetchOffset()).thenReturn(10L); + when(sp1.nextFetchOffset()).thenReturn(20L); + when(sp2.nextFetchOffset()).thenReturn(30L); + + // Fetch offset matches with the cached entry for sp0 but not for sp1 and sp2. Hence, a replica manager fetch will happen for sp1 and sp2. + when(sp0.fetchOffsetMetadata(anyLong())).thenReturn(Optional.of(new LogOffsetMetadata(10, 1, 0))); + when(sp1.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + when(sp2.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + + // Mocking local log read result for tp1 and remote storage read result for tp2. + doAnswer(invocation -> buildLocalAndRemoteFetchResult(Set.of(tp1), Set.of(tp2))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + + // Remote fetch related mocks. Remote fetch object does not complete within tryComplete in this mock. + RemoteLogManager remoteLogManager = mock(RemoteLogManager.class); + when(remoteLogManager.asyncRead(any(), any())).thenReturn(mock(Future.class)); + when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager)); + + DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() + .withShareFetchData(shareFetch) + .withSharePartitions(sharePartitions) + .withReplicaManager(replicaManager) + .withPartitionMaxBytesStrategy(mockPartitionMaxBytes(Set.of(tp0, tp1, tp2))) + .build()); + + assertFalse(delayedShareFetch.isCompleted()); + assertFalse(delayedShareFetch.tryComplete()); + assertFalse(delayedShareFetch.isCompleted()); + // Remote fetch object gets created for delayed share fetch object. + assertNotNull(delayedShareFetch.remoteFetch()); + // Verify the locks are released for local log read topic partitions tp0 and tp1. + Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp0, tp1)); + assertTrue(delayedShareFetch.lock().tryLock()); + delayedShareFetch.lock().unlock(); + } + + @Test + public void testRemoteStorageFetchTryCompleteThrowsException() { + ReplicaManager replicaManager = mock(ReplicaManager.class); + TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); + TopicIdPartition tp1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 1)); + + SharePartition sp0 = mock(SharePartition.class); + SharePartition sp1 = mock(SharePartition.class); + + // All the topic partitions are acquirable. + when(sp0.maybeAcquireFetchLock()).thenReturn(true); + when(sp1.maybeAcquireFetchLock()).thenReturn(true); + when(sp0.canAcquireRecords()).thenReturn(true); + when(sp1.canAcquireRecords()).thenReturn(true); + + LinkedHashMap sharePartitions = new LinkedHashMap<>(); + sharePartitions.put(tp0, sp0); + sharePartitions.put(tp1, sp1); + + CompletableFuture> future = new CompletableFuture<>(); + ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, "grp", Uuid.randomUuid().toString(), + future, List.of(tp0, tp1), BATCH_SIZE, MAX_FETCH_RECORDS, + BROKER_TOPIC_STATS); + + when(sp0.nextFetchOffset()).thenReturn(10L); + when(sp1.nextFetchOffset()).thenReturn(20L); + + // Fetch offset does not match with the cached entry for sp0 and sp1. Hence, a replica manager fetch will happen for sp0 and sp1. + when(sp0.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + when(sp1.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + + // Mocking local log read result for tp0 and remote storage read result for tp1. + doAnswer(invocation -> buildLocalAndRemoteFetchResult(Set.of(tp0), Set.of(tp1))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + + // Remote fetch related mocks. Exception will be thrown during the creation of remoteFetch object. + RemoteLogManager remoteLogManager = mock(RemoteLogManager.class); + when(remoteLogManager.asyncRead(any(), any())).thenThrow(new RejectedExecutionException("Exception thrown")); + when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager)); + + BiConsumer exceptionHandler = mockExceptionHandler(); + DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() + .withShareFetchData(shareFetch) + .withSharePartitions(sharePartitions) + .withReplicaManager(replicaManager) + .withExceptionHandler(exceptionHandler) + .withPartitionMaxBytesStrategy(mockPartitionMaxBytes(Set.of(tp0, tp1))) + .build()); + + assertFalse(delayedShareFetch.isCompleted()); + // tryComplete returns true and goes to forceComplete once the exception occurs. + assertTrue(delayedShareFetch.tryComplete()); + assertTrue(delayedShareFetch.isCompleted()); + // The future of shareFetch completes. + assertTrue(shareFetch.isCompleted()); + assertFalse(future.isCompletedExceptionally()); + assertEquals(Set.of(tp1), future.join().keySet()); + // Exception occurred and was handled. + Mockito.verify(exceptionHandler, times(1)).accept(any(), any()); + // Verify the locks are released for both local and remote read topic partitions tp0 and tp1 because of exception occurrence. + Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp0)); + Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp1)); + Mockito.verify(delayedShareFetch, times(1)).onComplete(); + assertTrue(delayedShareFetch.lock().tryLock()); + delayedShareFetch.lock().unlock(); + } + + @Test + public void testRemoteStorageFetchTryCompletionDueToBrokerBecomingOffline() { + ReplicaManager replicaManager = mock(ReplicaManager.class); + TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); + TopicIdPartition tp1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 1)); + TopicIdPartition tp2 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 2)); + + SharePartition sp0 = mock(SharePartition.class); + SharePartition sp1 = mock(SharePartition.class); + SharePartition sp2 = mock(SharePartition.class); + + // All the topic partitions are acquirable. + when(sp0.maybeAcquireFetchLock()).thenReturn(true); + when(sp1.maybeAcquireFetchLock()).thenReturn(true); + when(sp2.maybeAcquireFetchLock()).thenReturn(true); + when(sp0.canAcquireRecords()).thenReturn(true); + when(sp1.canAcquireRecords()).thenReturn(true); + when(sp2.canAcquireRecords()).thenReturn(true); + + LinkedHashMap sharePartitions = new LinkedHashMap<>(); + sharePartitions.put(tp0, sp0); + sharePartitions.put(tp1, sp1); + sharePartitions.put(tp2, sp2); + + CompletableFuture> future = new CompletableFuture<>(); + ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, "grp", Uuid.randomUuid().toString(), + future, List.of(tp0, tp1, tp2), BATCH_SIZE, MAX_FETCH_RECORDS, + BROKER_TOPIC_STATS); + + when(sp0.nextFetchOffset()).thenReturn(10L); + when(sp1.nextFetchOffset()).thenReturn(20L); + when(sp2.nextFetchOffset()).thenReturn(30L); + + // Fetch offset matches with the cached entry for sp0 but not for sp1 and sp2. Hence, a replica manager fetch will happen for sp1 and sp2 during tryComplete. + when(sp0.fetchOffsetMetadata(anyLong())).thenReturn(Optional.of(new LogOffsetMetadata(10, 1, 0))); + when(sp1.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + when(sp2.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + + when(sp0.acquire(any(), anyInt(), anyInt(), anyLong(), any(), any())).thenReturn( + createShareAcquiredRecords(new ShareFetchResponseData.AcquiredRecords().setFirstOffset(0).setLastOffset(3).setDeliveryCount((short) 1))); + when(sp1.acquire(any(), anyInt(), anyInt(), anyLong(), any(), any())).thenReturn( + createShareAcquiredRecords(new ShareFetchResponseData.AcquiredRecords().setFirstOffset(0).setLastOffset(3).setDeliveryCount((short) 1))); + + // Mocking local log read result for tp1 and remote storage read result for tp2 on first replicaManager readFromLog call(from tryComplete). + // Mocking local log read result for tp0 and tp1 on second replicaManager readFromLog call(from onComplete). + doAnswer(invocation -> buildLocalAndRemoteFetchResult(Set.of(tp1), Set.of(tp2)) + ).doAnswer(invocation -> buildLocalAndRemoteFetchResult(Set.of(tp0, tp1), Set.of()) + ).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + + // Remote fetch related mocks. Remote fetch object does not complete within tryComplete in this mock but the broker becomes unavailable. + Future remoteFetchTask = mock(Future.class); + doAnswer(invocation -> { + when(remoteFetchTask.isCancelled()).thenReturn(true); + return false; + }).when(remoteFetchTask).cancel(false); + + when(remoteFetchTask.cancel(false)).thenReturn(true); + RemoteLogManager remoteLogManager = mock(RemoteLogManager.class); + when(remoteLogManager.asyncRead(any(), any())).thenReturn(remoteFetchTask); + when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager)); + when(replicaManager.getPartitionOrException(tp2.topicPartition())).thenThrow(mock(KafkaStorageException.class)); + + DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() + .withShareFetchData(shareFetch) + .withSharePartitions(sharePartitions) + .withReplicaManager(replicaManager) + .withPartitionMaxBytesStrategy(mockPartitionMaxBytes(Set.of(tp0, tp1, tp2))) + .build()); + + assertFalse(delayedShareFetch.isCompleted()); + assertTrue(delayedShareFetch.tryComplete()); + + assertTrue(delayedShareFetch.isCompleted()); + // Pending remote fetch object gets created for delayed share fetch. + assertNotNull(delayedShareFetch.remoteFetch()); + assertTrue(delayedShareFetch.remoteFetch().remoteFetchTask().isCancelled()); + // Partition locks should be released for all 3 topic partitions + Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp0, tp1, tp2)); + assertTrue(shareFetch.isCompleted()); + // Share fetch response contained tp0 and tp1 (local fetch) but not tp2, since it errored out. + assertEquals(Set.of(tp0, tp1), future.join().keySet()); + assertTrue(delayedShareFetch.lock().tryLock()); + delayedShareFetch.lock().unlock(); + } + + @Test + public void testRemoteStorageFetchRequestCompletionOnFutureCompletionFailure() { + ReplicaManager replicaManager = mock(ReplicaManager.class); + TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); + TopicIdPartition tp1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 1)); + + SharePartition sp0 = mock(SharePartition.class); + SharePartition sp1 = mock(SharePartition.class); + + // sp0 is acquirable, sp1 is not acquirable. + when(sp0.maybeAcquireFetchLock()).thenReturn(true); + when(sp1.maybeAcquireFetchLock()).thenReturn(false); + when(sp0.canAcquireRecords()).thenReturn(true); + when(sp1.canAcquireRecords()).thenReturn(false); + + LinkedHashMap sharePartitions = new LinkedHashMap<>(); + sharePartitions.put(tp0, sp0); + sharePartitions.put(tp1, sp1); + + CompletableFuture> future = new CompletableFuture<>(); + ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, "grp", Uuid.randomUuid().toString(), + future, List.of(tp0, tp1), BATCH_SIZE, MAX_FETCH_RECORDS, + BROKER_TOPIC_STATS); + + when(sp0.nextFetchOffset()).thenReturn(10L); + // Fetch offset does not match with the cached entry for sp0. Hence, a replica manager fetch will happen for sp0. + when(sp0.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + + // Mocking remote storage read result for tp0. + doAnswer(invocation -> buildLocalAndRemoteFetchResult(Set.of(), Set.of(tp0))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + + // Remote fetch related mocks. Remote fetch object completes within tryComplete in this mock, hence request will move on to forceComplete. + RemoteLogReadResult remoteFetchResult = new RemoteLogReadResult( + Optional.empty(), + Optional.of(new TimeoutException("Error occurred while creating remote fetch result")) // Remote fetch result is returned with an error. + ); + RemoteLogManager remoteLogManager = mock(RemoteLogManager.class); + doAnswer(invocationOnMock -> { + // Make sure that the callback is called to populate remoteFetchResult for the mock behaviour. + Consumer callback = invocationOnMock.getArgument(1); + callback.accept(remoteFetchResult); + return CompletableFuture.completedFuture(remoteFetchResult); + }).when(remoteLogManager).asyncRead(any(), any()); + when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager)); + + DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() + .withShareFetchData(shareFetch) + .withSharePartitions(sharePartitions) + .withReplicaManager(replicaManager) + .withPartitionMaxBytesStrategy(mockPartitionMaxBytes(Set.of(tp0, tp1))) + .build()); + + when(sp0.acquire(any(), anyInt(), anyInt(), anyLong(), any(), any())).thenReturn( + createShareAcquiredRecords(new ShareFetchResponseData.AcquiredRecords().setFirstOffset(0).setLastOffset(3).setDeliveryCount((short) 1))); + + assertFalse(delayedShareFetch.isCompleted()); + assertTrue(delayedShareFetch.tryComplete()); + + assertTrue(delayedShareFetch.isCompleted()); + // Pending remote fetch object gets created for delayed share fetch. + assertNotNull(delayedShareFetch.remoteFetch()); + // Verify the locks are released for tp0. + Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp0)); + assertTrue(shareFetch.isCompleted()); + assertEquals(Set.of(tp0), future.join().keySet()); + assertEquals(Errors.REQUEST_TIMED_OUT.code(), future.join().get(tp0).errorCode()); + assertTrue(delayedShareFetch.lock().tryLock()); + delayedShareFetch.lock().unlock(); + } + + @Test + public void testRemoteStorageFetchRequestCompletionOnFutureCompletionSuccessfully() { + ReplicaManager replicaManager = mock(ReplicaManager.class); + TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); + + SharePartition sp0 = mock(SharePartition.class); + + // sp0 is acquirable. + when(sp0.maybeAcquireFetchLock()).thenReturn(true); + when(sp0.canAcquireRecords()).thenReturn(true); + + LinkedHashMap sharePartitions = new LinkedHashMap<>(); + sharePartitions.put(tp0, sp0); + + CompletableFuture> future = new CompletableFuture<>(); + ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, "grp", Uuid.randomUuid().toString(), + future, List.of(tp0), BATCH_SIZE, MAX_FETCH_RECORDS, + BROKER_TOPIC_STATS); + + when(sp0.nextFetchOffset()).thenReturn(10L); + // Fetch offset does not match with the cached entry for sp0. Hence, a replica manager fetch will happen for sp0. + when(sp0.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + + // Mocking remote storage read result for tp0. + doAnswer(invocation -> buildLocalAndRemoteFetchResult(Set.of(), Set.of(tp0))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + + // Remote fetch related mocks. Remote fetch object completes within tryComplete in this mock, hence request will move on to forceComplete. + RemoteLogReadResult remoteFetchResult = new RemoteLogReadResult( + Optional.of(REMOTE_FETCH_INFO), + Optional.empty() // Remote fetch result is returned successfully without error. + ); + RemoteLogManager remoteLogManager = mock(RemoteLogManager.class); + doAnswer(invocationOnMock -> { + // Make sure that the callback is called to populate remoteFetchResult for the mock behaviour. + Consumer callback = invocationOnMock.getArgument(1); + callback.accept(remoteFetchResult); + return CompletableFuture.completedFuture(remoteFetchResult); + }).when(remoteLogManager).asyncRead(any(), any()); + when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager)); + + DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() + .withShareFetchData(shareFetch) + .withSharePartitions(sharePartitions) + .withReplicaManager(replicaManager) + .withPartitionMaxBytesStrategy(mockPartitionMaxBytes(Set.of(tp0))) + .build()); + + when(sp0.acquire(any(), anyInt(), anyInt(), anyLong(), any(), any())).thenReturn( + createShareAcquiredRecords(new ShareFetchResponseData.AcquiredRecords().setFirstOffset(0).setLastOffset(3).setDeliveryCount((short) 1))); + + assertFalse(delayedShareFetch.isCompleted()); + assertTrue(delayedShareFetch.tryComplete()); + + assertTrue(delayedShareFetch.isCompleted()); + // Pending remote fetch object gets created for delayed share fetch. + assertNotNull(delayedShareFetch.remoteFetch()); + // Verify the locks are released for tp0. + Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp0)); + assertTrue(shareFetch.isCompleted()); + assertEquals(Set.of(tp0), future.join().keySet()); + assertEquals(Errors.NONE.code(), future.join().get(tp0).errorCode()); + assertTrue(delayedShareFetch.lock().tryLock()); + delayedShareFetch.lock().unlock(); + } + + @Test + public void testRemoteStorageFetchRequestCompletionAlongWithLocalLogRead() { + ReplicaManager replicaManager = mock(ReplicaManager.class); + TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); + TopicIdPartition tp1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 1)); + TopicIdPartition tp2 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 2)); + + SharePartition sp0 = mock(SharePartition.class); + SharePartition sp1 = mock(SharePartition.class); + SharePartition sp2 = mock(SharePartition.class); + + // All the topic partitions are acquirable. + when(sp0.maybeAcquireFetchLock()).thenReturn(true); + when(sp1.maybeAcquireFetchLock()).thenReturn(true); + when(sp2.maybeAcquireFetchLock()).thenReturn(true); + when(sp0.canAcquireRecords()).thenReturn(true); + when(sp1.canAcquireRecords()).thenReturn(true); + when(sp2.canAcquireRecords()).thenReturn(true); + + LinkedHashMap sharePartitions = new LinkedHashMap<>(); + sharePartitions.put(tp0, sp0); + sharePartitions.put(tp1, sp1); + sharePartitions.put(tp2, sp2); + + CompletableFuture> future = new CompletableFuture<>(); + ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, "grp", Uuid.randomUuid().toString(), + future, List.of(tp0, tp1, tp2), BATCH_SIZE, MAX_FETCH_RECORDS, + BROKER_TOPIC_STATS); + + when(sp0.nextFetchOffset()).thenReturn(10L); + when(sp1.nextFetchOffset()).thenReturn(20L); + when(sp2.nextFetchOffset()).thenReturn(30L); + + // Fetch offset does not match with the cached entry for sp0, sp1 and sp2. Hence, a replica manager fetch will happen for all of them in tryComplete. + when(sp0.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + when(sp1.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + when(sp2.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + + when(sp0.acquire(any(), anyInt(), anyInt(), anyLong(), any(), any())).thenReturn( + createShareAcquiredRecords(new ShareFetchResponseData.AcquiredRecords().setFirstOffset(0).setLastOffset(3).setDeliveryCount((short) 1))); + when(sp1.acquire(any(), anyInt(), anyInt(), anyLong(), any(), any())).thenReturn( + createShareAcquiredRecords(new ShareFetchResponseData.AcquiredRecords().setFirstOffset(0).setLastOffset(3).setDeliveryCount((short) 1))); + when(sp2.acquire(any(), anyInt(), anyInt(), anyLong(), any(), any())).thenReturn( + createShareAcquiredRecords(new ShareFetchResponseData.AcquiredRecords().setFirstOffset(0).setLastOffset(3).setDeliveryCount((short) 1))); + + // Mocking local log read result for tp0, tp1 and remote storage read result for tp2 on first replicaManager readFromLog call(from tryComplete). + // Mocking local log read result for tp0 and tp1 on second replicaManager readFromLog call(from onComplete). + doAnswer(invocation -> buildLocalAndRemoteFetchResult(Set.of(tp0, tp1), Set.of(tp2)) + ).doAnswer(invocation -> buildLocalAndRemoteFetchResult(Set.of(tp0, tp1), Set.of()) + ).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + + // Remote fetch related mocks. Remote fetch object completes within tryComplete in this mock, hence request will move on to forceComplete. + RemoteLogReadResult remoteFetchResult = new RemoteLogReadResult( + Optional.of(REMOTE_FETCH_INFO), + Optional.empty() // Remote fetch result is returned successfully without error. + ); + RemoteLogManager remoteLogManager = mock(RemoteLogManager.class); + doAnswer(invocationOnMock -> { + // Make sure that the callback is called to populate remoteFetchResult for the mock behaviour. + Consumer callback = invocationOnMock.getArgument(1); + callback.accept(remoteFetchResult); + return CompletableFuture.completedFuture(remoteFetchResult); + }).when(remoteLogManager).asyncRead(any(), any()); + when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager)); + + DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() + .withShareFetchData(shareFetch) + .withReplicaManager(replicaManager) + .withSharePartitions(sharePartitions) + .withPartitionMaxBytesStrategy(mockPartitionMaxBytes(Set.of(tp0, tp1, tp2))) + .build()); + + assertFalse(delayedShareFetch.isCompleted()); + assertTrue(delayedShareFetch.tryComplete()); + + assertTrue(delayedShareFetch.isCompleted()); + // Pending remote fetch object gets created for delayed share fetch. + assertNotNull(delayedShareFetch.remoteFetch()); + // the future of shareFetch completes. + assertTrue(shareFetch.isCompleted()); + assertEquals(Set.of(tp0, tp1, tp2), future.join().keySet()); + // Verify the locks are released for both local log and remote storage read topic partitions tp0, tp1 and tp2. + Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp0, tp1, tp2)); + assertEquals(Errors.NONE.code(), future.join().get(tp0).errorCode()); + assertEquals(Errors.NONE.code(), future.join().get(tp1).errorCode()); + assertEquals(Errors.NONE.code(), future.join().get(tp2).errorCode()); + assertTrue(delayedShareFetch.lock().tryLock()); + delayedShareFetch.lock().unlock(); + } + + @Test + public void testRemoteStorageFetchOnlyHappensForFirstTopicPartition() { + ReplicaManager replicaManager = mock(ReplicaManager.class); + TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); + TopicIdPartition tp1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 1)); + + SharePartition sp0 = mock(SharePartition.class); + SharePartition sp1 = mock(SharePartition.class); + + // sp0 and sp1 are acquirable. + when(sp0.maybeAcquireFetchLock()).thenReturn(true); + when(sp1.maybeAcquireFetchLock()).thenReturn(true); + when(sp0.canAcquireRecords()).thenReturn(true); + when(sp1.canAcquireRecords()).thenReturn(true); + + LinkedHashMap sharePartitions = new LinkedHashMap<>(); + sharePartitions.put(tp0, sp0); + sharePartitions.put(tp1, sp1); + + CompletableFuture> future = new CompletableFuture<>(); + ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, "grp", Uuid.randomUuid().toString(), + future, List.of(tp0, tp1), BATCH_SIZE, MAX_FETCH_RECORDS, + BROKER_TOPIC_STATS); + + when(sp0.nextFetchOffset()).thenReturn(10L); + when(sp1.nextFetchOffset()).thenReturn(10L); + // Fetch offset does not match with the cached entry for sp0 and sp1. Hence, a replica manager fetch will happen for both. + when(sp0.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + when(sp1.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + + LinkedHashSet remoteStorageFetchPartitions = new LinkedHashSet<>(); + remoteStorageFetchPartitions.add(tp0); + remoteStorageFetchPartitions.add(tp1); + + // Mocking remote storage read result for tp0 and tp1. + doAnswer(invocation -> buildLocalAndRemoteFetchResult(Set.of(), remoteStorageFetchPartitions)).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + + // Remote fetch related mocks. Remote fetch object completes within tryComplete in this mock, hence request will move on to forceComplete. + RemoteLogReadResult remoteFetchResult = new RemoteLogReadResult( + Optional.of(REMOTE_FETCH_INFO), + Optional.empty() // Remote fetch result is returned successfully without error. + ); + RemoteLogManager remoteLogManager = mock(RemoteLogManager.class); + doAnswer(invocationOnMock -> { + // Make sure that the callback is called to populate remoteFetchResult for the mock behaviour. + Consumer callback = invocationOnMock.getArgument(1); + callback.accept(remoteFetchResult); + return CompletableFuture.completedFuture(remoteFetchResult); + }).when(remoteLogManager).asyncRead(any(), any()); + when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager)); + + DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() + .withShareFetchData(shareFetch) + .withSharePartitions(sharePartitions) + .withReplicaManager(replicaManager) + .withPartitionMaxBytesStrategy(mockPartitionMaxBytes(Set.of(tp0, tp1))) + .build()); + + when(sp0.acquire(any(), anyInt(), anyInt(), anyLong(), any(), any())).thenReturn( + createShareAcquiredRecords(new ShareFetchResponseData.AcquiredRecords().setFirstOffset(0).setLastOffset(3).setDeliveryCount((short) 1))); + + assertFalse(delayedShareFetch.isCompleted()); + assertTrue(delayedShareFetch.tryComplete()); + + assertTrue(delayedShareFetch.isCompleted()); + // Pending remote fetch object gets created for delayed share fetch. + assertNotNull(delayedShareFetch.remoteFetch()); + // Verify the locks are released separately for tp1 (from tryComplete). + Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp1)); + // From onComplete, the locks will be released for both tp0 and tp1. tp0 because it was acquired from + // tryComplete and has remote fetch processed. tp1 will be reacquired in onComplete when we check for local log read. + Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp0, tp1)); + assertTrue(shareFetch.isCompleted()); + // Share fetch response only contains the first remote storage fetch topic partition - tp0. + assertEquals(Set.of(tp0), future.join().keySet()); + assertEquals(Errors.NONE.code(), future.join().get(tp0).errorCode()); + assertTrue(delayedShareFetch.lock().tryLock()); + delayedShareFetch.lock().unlock(); + } + static void mockTopicIdPartitionToReturnDataEqualToMinBytes(ReplicaManager replicaManager, TopicIdPartition topicIdPartition, int minBytes) { LogOffsetMetadata hwmOffsetMetadata = new LogOffsetMetadata(1, 1, minBytes); LogOffsetSnapshot endOffsetSnapshot = new LogOffsetSnapshot(1, mock(LogOffsetMetadata.class), @@ -1182,6 +1716,37 @@ private PartitionMaxBytesStrategy mockPartitionMaxBytes(Set pa return partitionMaxBytesStrategy; } + private Seq> buildLocalAndRemoteFetchResult( + Set localLogReadTopicIdPartitions, + Set remoteReadTopicIdPartitions) { + List> logReadResults = new ArrayList<>(); + localLogReadTopicIdPartitions.forEach(topicIdPartition -> logReadResults.add(new Tuple2<>(topicIdPartition, new LogReadResult( + new FetchDataInfo(new LogOffsetMetadata(0, 0, 0), MemoryRecords.EMPTY), + Option.empty(), + -1L, + -1L, + -1L, + -1L, + -1L, + Option.empty(), + Option.empty(), + Option.empty() + )))); + remoteReadTopicIdPartitions.forEach(topicIdPartition -> logReadResults.add(new Tuple2<>(topicIdPartition, new LogReadResult( + REMOTE_FETCH_INFO, + Option.empty(), + -1L, + -1L, + -1L, + -1L, + -1L, + Option.empty(), + Option.empty(), + Option.empty() + )))); + return CollectionConverters.asScala(logReadResults).toSeq(); + } + @SuppressWarnings("unchecked") private static BiConsumer mockExceptionHandler() { return mock(BiConsumer.class); @@ -1194,6 +1759,7 @@ static class DelayedShareFetchBuilder { private LinkedHashMap sharePartitions = mock(LinkedHashMap.class); private PartitionMaxBytesStrategy partitionMaxBytesStrategy = mock(PartitionMaxBytesStrategy.class); private Time time = new MockTime(); + private final Optional remoteFetch = Optional.empty(); private ShareGroupMetrics shareGroupMetrics = mock(ShareGroupMetrics.class); DelayedShareFetchBuilder withShareFetchData(ShareFetch shareFetch) { @@ -1243,7 +1809,8 @@ public DelayedShareFetch build() { sharePartitions, partitionMaxBytesStrategy, shareGroupMetrics, - time); + time, + remoteFetch); } } } From 4410d35cdc7251cca607918fd754d458fde27918 Mon Sep 17 00:00:00 2001 From: Xuan-Zhang Gong Date: Tue, 22 Apr 2025 10:16:54 +0800 Subject: [PATCH 07/37] KAFKA-19179: remove the dot from thread_dump_url (#19525) As the title. Ticket: https://issues.apache.org/jira/browse/KAFKA-19179 Reviewers: PoAn Yang , Jhen-Yung Hsu , TengYao Chi , Nick Guo , Ken Huang , Chia-Ping Tsai --- .github/scripts/junit.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/scripts/junit.py b/.github/scripts/junit.py index 95b5d4e4610de..550ea4935116d 100644 --- a/.github/scripts/junit.py +++ b/.github/scripts/junit.py @@ -361,7 +361,7 @@ def split_report_path(base_path: str, report_path: str) -> Tuple[str, str]: failure_messages.append(f"Gradle task had a failure exit code. Failing this script.") if thread_dump_url: - failure_messages.append(f"Thread dump available at {thread_dump_url}. Failing this script.") + failure_messages.append(f"Thread dump available at {thread_dump_url} and the script will now fail.") if junit_report_url: report_md = f"Download [JUnit HTML report]({junit_report_url})" From 070892dafcdbd8042560a7284ad78f4c169c4cad Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dejan=20Stojadinovi=C4=87?= Date: Tue, 22 Apr 2025 09:24:58 +0200 Subject: [PATCH 08/37] MINOR: redundant gradle expression is removed (#19200) Note: this is a #18018 offshoot. See this comment made by @Goooler: https://github.com/apache/kafka/pull/18018#discussion_r1897025851 ![image](https://github.com/user-attachments/assets/c41acba6-9f2a-44a5-b47f-60a4d7078100) Reviewers: Apoorv Mittal , David Arthur , Goooler --- build.gradle | 2 -- 1 file changed, 2 deletions(-) diff --git a/build.gradle b/build.gradle index 50eec676569e9..036682572e48b 100644 --- a/build.gradle +++ b/build.gradle @@ -369,7 +369,6 @@ subprojects { // Fix for avoiding inclusion of runtime dependencies marked as 'shadow' in MANIFEST Class-Path. // https://github.com/GradleUp/shadow/issues/324 - afterEvaluate { pom.withXml { xml -> def dependenciesNode = xml.asNode().get('dependencies') ?: xml.asNode().appendNode('dependencies') project.configurations.shadowed.allDependencies.each { @@ -380,7 +379,6 @@ subprojects { dependencyNode.appendNode('scope', 'runtime') } } - } } afterEvaluate { From 11904c74e0395047a2db980ec3c65ae7fc7a72d4 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Tue, 22 Apr 2025 05:04:22 -0400 Subject: [PATCH 09/37] KAFKA-19166: Fix RC tag in release script (#19518) The release script was pushing the RC tag off of a temporary branch that was never merged back into the release branch. This meant that our RC and release tags were detached from the rest of the repository. This patch changes the release script to merge the RC tag back into the release branch and pushes both the tag and the branch. Reviewers: Luke Chen --- release/git.py | 8 ++++++-- release/release.py | 4 +++- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/release/git.py b/release/git.py index 61de830ecea29..9cb106df2fdcb 100644 --- a/release/git.py +++ b/release/git.py @@ -128,8 +128,12 @@ def create_tag(tag, **kwargs): cmd(f"Creating git tag {tag}", ["git", "tag", "-a", tag, "-m", tag], **kwargs) -def push_tag(tag, remote=push_remote_name, **kwargs): +def push_ref(ref, remote=push_remote_name, **kwargs): __defaults(kwargs) - cmd("Pushing tag {tag} to {remote}", f"git push {remote} {tag}") + cmd(f"Pushing ref {ref} to {remote}", f"git push {remote} {ref}") +def merge_ref(ref, **kwargs): + __defaults(kwargs) + cmd(f"Merging ref {ref}", f"git merge {ref}") + diff --git a/release/release.py b/release/release.py index b3a428f651636..d0cba6f178216 100644 --- a/release/release.py +++ b/release/release.py @@ -297,6 +297,7 @@ def delete_gitrefs(): git.commit(f"Bump version to {release_version}") git.create_tag(rc_tag) git.switch_branch(starting_branch) +git.merge_ref(rc_tag) # Note that we don't use tempfile here because mkdtemp causes problems with being able to determine the absolute path to a file. # Instead we rely on a fixed path @@ -367,7 +368,8 @@ def delete_gitrefs(): print(templates.deploy_instructions()) confirm_or_fail("Have you successfully deployed the artifacts?") confirm_or_fail(f"Ok to push RC tag {rc_tag}?") -git.push_tag(rc_tag) +git.push_ref(rc_tag) +git.push_ref(starting_branch) # Move back to starting branch and clean out the temporary release branch (e.g. 1.0.0) we used to generate everything git.reset_hard_head() From 66147d5de7cf625fc8799a6252cb99aad706c698 Mon Sep 17 00:00:00 2001 From: Andrew Schofield Date: Tue, 22 Apr 2025 11:43:32 +0100 Subject: [PATCH 10/37] KAFKA-19057: Stabilize KIP-932 RPCs for AK 4.1 (#19378) This PR removes the unstable API flag for the KIP-932 RPCs. The 4 RPCs which were exposed for the early access release in AK 4.0 are stabilised at v1. This is because the RPCs have evolved over time and AK 4.0 clients are not compatible with AK 4.1 brokers. By stabilising at v1, the API version checks prevent incompatible communication and server-side exceptions when trying to parse the requests from the older clients. Reviewers: Apoorv Mittal --- .../clients/consumer/ShareConsumerTest.java | 9 +- .../DeleteShareGroupOffsetsHandler.java | 3 +- .../internals/DescribeShareGroupsHandler.java | 2 +- .../ListShareGroupOffsetsHandler.java | 2 +- .../AlterShareGroupOffsetsRequest.java | 6 +- .../DeleteShareGroupOffsetsRequest.java | 6 +- .../DeleteShareGroupStateRequest.java | 6 +- .../DescribeShareGroupOffsetsRequest.java | 6 +- .../InitializeShareGroupStateRequest.java | 6 +- .../requests/ReadShareGroupStateRequest.java | 6 +- .../ReadShareGroupStateSummaryRequest.java | 6 +- .../requests/ShareAcknowledgeRequest.java | 8 +- .../common/requests/ShareFetchRequest.java | 8 +- .../requests/ShareGroupDescribeRequest.java | 6 +- .../requests/ShareGroupHeartbeatRequest.java | 6 +- .../requests/WriteShareGroupStateRequest.java | 6 +- .../AlterShareGroupOffsetsRequest.json | 1 - .../AlterShareGroupOffsetsResponse.json | 1 + .../DeleteShareGroupOffsetsRequest.json | 1 - .../message/DeleteShareGroupStateRequest.json | 5 +- .../DescribeShareGroupOffsetsRequest.json | 1 - .../InitializeShareGroupStateRequest.json | 3 +- .../message/ReadShareGroupStateRequest.json | 5 +- .../message/ReadShareGroupStateResponse.json | 7 +- .../ReadShareGroupStateSummaryRequest.json | 5 +- .../message/ShareAcknowledgeRequest.json | 17 +-- .../message/ShareAcknowledgeResponse.json | 13 +- .../common/message/ShareFetchRequest.json | 15 +- .../common/message/ShareFetchResponse.json | 17 ++- .../message/ShareGroupDescribeRequest.json | 9 +- .../message/ShareGroupDescribeResponse.json | 9 +- .../message/ShareGroupHeartbeatRequest.json | 11 +- .../message/ShareGroupHeartbeatResponse.json | 9 +- .../message/WriteShareGroupStateRequest.json | 11 +- .../message/WriteShareGroupStateResponse.json | 1 + .../kafka/api/IntegrationTestHarness.scala | 3 +- .../GroupCoordinatorBaseRequestTest.scala | 6 +- .../unit/kafka/server/KafkaApisTest.scala | 140 +++++++----------- .../unit/kafka/server/RequestQuotaTest.scala | 24 +-- .../ShareFetchAcknowledgeRequestTest.scala | 80 +++------- .../ShareGroupDescribeRequestTest.scala | 10 +- .../ShareGroupHeartbeatRequestTest.scala | 84 ++++------- .../persister/PersisterStateManager.java | 3 +- .../services/kafka/config_property.py | 1 - tests/kafkatest/services/kafka/kafka.py | 1 - 45 files changed, 214 insertions(+), 371 deletions(-) diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java index e402a4344c109..d48f620880ef3 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java @@ -122,8 +122,7 @@ @ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "3"), @ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), @ClusterConfigProperty(key = "transaction.state.log.min.isr", value = "1"), - @ClusterConfigProperty(key = "transaction.state.log.replication.factor", value = "1"), - @ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + @ClusterConfigProperty(key = "transaction.state.log.replication.factor", value = "1") }, types = {Type.KRAFT} ) @@ -1859,8 +1858,7 @@ public void testShareAutoOffsetResetByDurationInvalidFormat() throws Exception { @ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "3"), @ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "3"), @ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "3"), - @ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "3"), - @ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + @ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "3") } ) @Timeout(90) @@ -2011,8 +2009,7 @@ public void testShareConsumerAfterCoordinatorMovement() throws Exception { @ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "3"), @ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "3"), @ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "3"), - @ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "3"), - @ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + @ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "3") } ) @Timeout(150) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteShareGroupOffsetsHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteShareGroupOffsetsHandler.java index c5911e4303e72..e37b0427355a4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteShareGroupOffsetsHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteShareGroupOffsetsHandler.java @@ -99,8 +99,7 @@ DeleteShareGroupOffsetsRequest.Builder buildBatchedRequest(int brokerId, Set ackTopic.partitions().add(ackPartition)); }); - return new ShareAcknowledgeRequest.Builder(data, true); + return new ShareAcknowledgeRequest.Builder(data); } public ShareAcknowledgeRequestData data() { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchRequest.java index ea8d93f2a91f3..987400a7ce0df 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchRequest.java @@ -37,11 +37,7 @@ public static class Builder extends AbstractRequest.Builder { private final ShareFetchRequestData data; public Builder(ShareFetchRequestData data) { - this(data, false); - } - - public Builder(ShareFetchRequestData data, boolean enableUnstableLastVersion) { - super(ApiKeys.SHARE_FETCH, enableUnstableLastVersion); + super(ApiKeys.SHARE_FETCH); this.data = data; } @@ -104,7 +100,7 @@ public static Builder forConsumer(String groupId, ShareRequestMetadata metadata, }); } - Builder builder = new Builder(data, true); + Builder builder = new Builder(data); // And finally, forget the topic-partitions that are no longer in the session if (!forget.isEmpty()) { data.setForgottenTopicsData(new ArrayList<>()); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeRequest.java index 14dd429b8a4f8..1ad411f86010f 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeRequest.java @@ -32,11 +32,7 @@ public static class Builder extends AbstractRequest.Builder "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareFetchRequest(request) @@ -4076,7 +4075,6 @@ class KafkaApisTest extends Logging { var request = buildRequest(shareFetchRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareFetchRequest(request) @@ -4176,7 +4174,6 @@ class KafkaApisTest extends Logging { var request = buildRequest(shareFetchRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareFetchRequest(request) @@ -4256,7 +4253,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareFetchRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareFetchRequest(request) @@ -4332,7 +4328,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareFetchRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareFetchRequest(request) @@ -4395,7 +4390,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareFetchRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareFetchRequest(request) @@ -4449,7 +4443,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareFetchRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareFetchRequest(request) @@ -4518,7 +4511,6 @@ class KafkaApisTest extends Logging { // First share fetch request is to establish the share session with the broker. kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareFetchRequest(request) @@ -4608,7 +4600,6 @@ class KafkaApisTest extends Logging { // First share fetch request is to establish the share session with the broker. kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareFetchRequest(request) @@ -4752,7 +4743,6 @@ class KafkaApisTest extends Logging { // First share fetch request is to establish the share session with the broker. kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareFetchRequest(request) @@ -5075,7 +5065,6 @@ class KafkaApisTest extends Logging { // First share fetch request is to establish the share session with the broker. kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareFetchRequest(request) @@ -5417,7 +5406,6 @@ class KafkaApisTest extends Logging { // First share fetch request is to establish the share session with the broker. kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) val fetchResult: Map[TopicIdPartition, ShareFetchResponseData.PartitionData] = @@ -5557,7 +5545,6 @@ class KafkaApisTest extends Logging { // First share fetch request is to establish the share session with the broker. kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) val fetchResult: Map[TopicIdPartition, ShareFetchResponseData.PartitionData] = @@ -5686,7 +5673,6 @@ class KafkaApisTest extends Logging { // First share fetch request is to establish the share session with the broker. kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) val fetchResult: Map[TopicIdPartition, ShareFetchResponseData.PartitionData] = @@ -5836,7 +5822,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareFetchRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) val fetchResult: Map[TopicIdPartition, ShareFetchResponseData.PartitionData] = @@ -6008,7 +5993,6 @@ class KafkaApisTest extends Logging { var request = buildRequest(shareFetchRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareFetchRequest(request) @@ -6092,7 +6076,6 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "false"), ) kafkaApis.handleShareFetchRequest(request) @@ -6143,7 +6126,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareFetchRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), authorizer = Option(authorizer), ) @@ -6207,7 +6189,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareFetchRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareFetchRequest(request) @@ -6274,7 +6255,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareAcknowledgeRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareAcknowledgeRequest(request) @@ -6321,7 +6301,6 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "false"), ) kafkaApis.handleShareAcknowledgeRequest(request) @@ -6372,7 +6351,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareAcknowledgeRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), authorizer = Option(authorizer), ) @@ -6424,7 +6402,6 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareAcknowledgeRequest(request) @@ -6475,7 +6452,6 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareAcknowledgeRequest(request) @@ -6524,7 +6500,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareAcknowledgeRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareAcknowledgeRequest(request) @@ -6599,7 +6574,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareAcknowledgeRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareAcknowledgeRequest(request) @@ -6662,7 +6636,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareAcknowledgeRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareAcknowledgeRequest(request) @@ -6729,7 +6702,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareAcknowledgeRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareAcknowledgeRequest(request) @@ -6797,7 +6769,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareAcknowledgeRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareAcknowledgeRequest(request) @@ -6879,7 +6850,6 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) val acknowledgeBatches = kafkaApis.getAcknowledgeBatchesFromShareFetchRequest(shareFetchRequest, topicNames, erroneous) @@ -6945,7 +6915,6 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) val acknowledgeBatches = kafkaApis.getAcknowledgeBatchesFromShareFetchRequest(shareFetchRequest, topicIdNames, erroneous) @@ -7018,7 +6987,6 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) val acknowledgeBatches = kafkaApis.getAcknowledgeBatchesFromShareAcknowledgeRequest(shareAcknowledgeRequest, topicNames, erroneous) @@ -7085,7 +7053,6 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) val acknowledgeBatches = kafkaApis.getAcknowledgeBatchesFromShareAcknowledgeRequest(shareAcknowledgeRequest, topicIdNames, erroneous) @@ -7157,7 +7124,6 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) val ackResult = kafkaApis.handleAcknowledgements( @@ -7236,7 +7202,6 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) val ackResult = kafkaApis.handleAcknowledgements( @@ -7316,7 +7281,6 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) val ackResult = kafkaApis.handleAcknowledgements( @@ -7390,7 +7354,6 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) val ackResult = kafkaApis.handleAcknowledgements( @@ -7487,7 +7450,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareAcknowledgeRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) val response = kafkaApis.processShareAcknowledgeResponse(responseAcknowledgeData, request) @@ -10772,7 +10734,7 @@ class KafkaApisTest extends Logging { def testShareGroupHeartbeatReturnsUnsupportedVersion(): Unit = { val shareGroupHeartbeatRequest = new ShareGroupHeartbeatRequestData().setGroupId("group") - val requestChannelRequest = buildRequest(new ShareGroupHeartbeatRequest.Builder(shareGroupHeartbeatRequest, true).build()) + val requestChannelRequest = buildRequest(new ShareGroupHeartbeatRequest.Builder(shareGroupHeartbeatRequest).build()) metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) kafkaApis = createKafkaApis() kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -10787,7 +10749,7 @@ class KafkaApisTest extends Logging { def testShareGroupHeartbeatRequest(): Unit = { val shareGroupHeartbeatRequest = new ShareGroupHeartbeatRequestData().setGroupId("group") - val requestChannelRequest = buildRequest(new ShareGroupHeartbeatRequest.Builder(shareGroupHeartbeatRequest, true).build()) + val requestChannelRequest = buildRequest(new ShareGroupHeartbeatRequest.Builder(shareGroupHeartbeatRequest).build()) val future = new CompletableFuture[ShareGroupHeartbeatResponseData]() when(groupCoordinator.shareGroupHeartbeat( @@ -10812,7 +10774,7 @@ class KafkaApisTest extends Logging { def testShareGroupHeartbeatRequestGroupAuthorizationFailed(): Unit = { val shareGroupHeartbeatRequest = new ShareGroupHeartbeatRequestData().setGroupId("group") - val requestChannelRequest = buildRequest(new ShareGroupHeartbeatRequest.Builder(shareGroupHeartbeatRequest, true).build()) + val requestChannelRequest = buildRequest(new ShareGroupHeartbeatRequest.Builder(shareGroupHeartbeatRequest).build()) val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) @@ -10872,7 +10834,7 @@ class KafkaApisTest extends Logging { def testShareGroupHeartbeatRequestFutureFailed(): Unit = { val shareGroupHeartbeatRequest = new ShareGroupHeartbeatRequestData().setGroupId("group") - val requestChannelRequest = buildRequest(new ShareGroupHeartbeatRequest.Builder(shareGroupHeartbeatRequest, true).build()) + val requestChannelRequest = buildRequest(new ShareGroupHeartbeatRequest.Builder(shareGroupHeartbeatRequest).build()) val future = new CompletableFuture[ShareGroupHeartbeatResponseData]() when(groupCoordinator.shareGroupHeartbeat( @@ -11078,7 +11040,7 @@ class KafkaApisTest extends Logging { ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", ) - val response = getReadShareGroupResponse( + val response = getReadShareGroupStateResponse( readRequestData, config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = true, @@ -11133,7 +11095,7 @@ class KafkaApisTest extends Logging { ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", ) - val response = getReadShareGroupResponse( + val response = getReadShareGroupStateResponse( readRequestData, config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = false, @@ -11181,7 +11143,7 @@ class KafkaApisTest extends Logging { ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", ) - val response = getReadShareGroupSummaryResponse( + val response = getReadShareGroupStateSummaryResponse( readSummaryRequestData, config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = true, @@ -11229,7 +11191,7 @@ class KafkaApisTest extends Logging { ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", ) - val response = getReadShareGroupSummaryResponse( + val response = getReadShareGroupStateSummaryResponse( readSummaryRequestData, config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = false, @@ -11253,7 +11215,7 @@ class KafkaApisTest extends Logging { )) ) - val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest, true).build()) + val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest).build()) metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) kafkaApis = createKafkaApis() kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -11270,7 +11232,7 @@ class KafkaApisTest extends Logging { )) ) - val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest, true).build) + val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest).build) val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) @@ -11298,7 +11260,7 @@ class KafkaApisTest extends Logging { util.List.of(new DescribeShareGroupOffsetsRequestGroup().setGroupId("group").setTopics(null)) ) - val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest, true).build) + val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest).build) val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) @@ -11349,7 +11311,7 @@ class KafkaApisTest extends Logging { val describeShareGroupOffsetsRequest = new DescribeShareGroupOffsetsRequestData() .setGroups(util.List.of(describeShareGroupOffsetsRequestGroup1, describeShareGroupOffsetsRequestGroup2)) - val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest, true).build) + val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest).build) val futureGroup1 = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] when(groupCoordinator.describeShareGroupOffsets( @@ -11465,7 +11427,7 @@ class KafkaApisTest extends Logging { val describeShareGroupOffsetsRequest = new DescribeShareGroupOffsetsRequestData() .setGroups(util.List.of(describeShareGroupOffsetsRequestGroup1, describeShareGroupOffsetsRequestGroup2)) - val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest, true).build) + val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest).build) // The group coordinator will only be asked for information about topics which are authorized val futureGroup1 = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] @@ -11634,7 +11596,7 @@ class KafkaApisTest extends Logging { val describeShareGroupOffsetsRequest = new DescribeShareGroupOffsetsRequestData() .setGroups(util.List.of(describeShareGroupOffsetsRequestGroup1, describeShareGroupOffsetsRequestGroup2)) - val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest, true).build) + val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest).build) // The group coordinator is being asked for information about all topics, not just those which are authorized val futureGroup1 = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] @@ -11796,7 +11758,7 @@ class KafkaApisTest extends Logging { val describeShareGroupOffsetsRequest = new DescribeShareGroupOffsetsRequestData() .setGroups(util.List.of(describeShareGroupOffsetsRequestGroup1, describeShareGroupOffsetsRequestGroup2)) - val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest, true).build) + val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest).build) val futureGroup1 = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] when(groupCoordinator.describeShareGroupAllOffsets( @@ -11889,7 +11851,7 @@ class KafkaApisTest extends Logging { val describeShareGroupOffsetsRequest = new DescribeShareGroupOffsetsRequestData - val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest, true).build) + val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest).build) val future = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] kafkaApis = createKafkaApis( @@ -11914,7 +11876,7 @@ class KafkaApisTest extends Logging { val describeShareGroupOffsetsRequest = new DescribeShareGroupOffsetsRequestData().setGroups(util.List.of(describeShareGroupOffsetsRequestGroup)) - val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest, true).build) + val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest).build) val future = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] when(groupCoordinator.describeShareGroupOffsets( @@ -11943,7 +11905,7 @@ class KafkaApisTest extends Logging { .setGroupId("group") .setTopics(util.List.of(new DeleteShareGroupOffsetsRequestTopic().setTopicName("topic-1").setPartitions(util.List.of(1)))) - val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequest, true).build()) + val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequest).build()) metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) kafkaApis = createKafkaApis() kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -11958,7 +11920,7 @@ class KafkaApisTest extends Logging { .setGroupId("group") .setTopics(util.List.of(new DeleteShareGroupOffsetsRequestTopic().setTopicName("topic-1").setPartitions(util.List.of(1)))) - val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequest, true).build) + val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequest).build) val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) @@ -12015,7 +11977,7 @@ class KafkaApisTest extends Logging { .setGroupId(groupId) .setTopics(util.List.of(deleteShareGroupOffsetsRequestTopic2)) - val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequestData, true).build) + val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequestData).build) val resultFuture = new CompletableFuture[DeleteShareGroupOffsetsResponseData] when(groupCoordinator.deleteShareGroupOffsets( @@ -12116,7 +12078,7 @@ class KafkaApisTest extends Logging { .setGroupId("group") .setTopics(util.List.of(deleteShareGroupOffsetsRequestTopic1, deleteShareGroupOffsetsRequestTopic2, deleteShareGroupOffsetsRequestTopic3)) - val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequestData, true).build) + val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequestData).build) val resultFuture = new CompletableFuture[DeleteShareGroupOffsetsResponseData] when(groupCoordinator.deleteShareGroupOffsets( @@ -12201,7 +12163,7 @@ class KafkaApisTest extends Logging { .setGroupId("group") .setTopics(util.List.of(deleteShareGroupOffsetsRequestTopic1, deleteShareGroupOffsetsRequestTopic2)) - val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequestData, true).build) + val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequestData).build) when(groupCoordinator.deleteShareGroupOffsets( requestChannelRequest.context, @@ -12243,7 +12205,7 @@ class KafkaApisTest extends Logging { .setGroupId("group") .setTopics(util.List.of(deleteShareGroupOffsetsRequestTopic1, deleteShareGroupOffsetsRequestTopic2)) - val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequestData, true).build) + val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequestData).build) val groupCoordinatorResponse: DeleteShareGroupOffsetsResponseData = new DeleteShareGroupOffsetsResponseData() .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code()) @@ -12274,7 +12236,7 @@ class KafkaApisTest extends Logging { val deleteShareGroupOffsetsRequest = new DeleteShareGroupOffsetsRequestData() .setGroupId("group") - val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequest, true).build) + val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequest).build) val resultFuture = new CompletableFuture[DeleteShareGroupOffsetsResponseData] kafkaApis = createKafkaApis( @@ -12328,7 +12290,7 @@ class KafkaApisTest extends Logging { ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", ) - val response = getWriteShareGroupResponse( + val response = getWriteShareGroupStateResponse( writeRequestData, config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = true, @@ -12383,7 +12345,7 @@ class KafkaApisTest extends Logging { when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) .thenReturn(Seq(AuthorizationResult.DENIED).asJava, Seq(AuthorizationResult.ALLOWED).asJava) - val response = getWriteShareGroupResponse( + val response = getWriteShareGroupStateResponse( writeRequestData, config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = false, @@ -12428,7 +12390,7 @@ class KafkaApisTest extends Logging { ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", ) - val response = getDeleteShareGroupResponse( + val response = getDeleteShareGroupStateResponse( deleteRequestData, config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = true, @@ -12473,7 +12435,7 @@ class KafkaApisTest extends Logging { ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", ) - val response = getDeleteShareGroupResponse( + val response = getDeleteShareGroupStateResponse( deleteRequestData, config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = false, @@ -12519,7 +12481,7 @@ class KafkaApisTest extends Logging { ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", ) - val response = getInitializeShareGroupResponse( + val response = getInitializeShareGroupStateResponse( initRequestData, config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = true, @@ -12565,7 +12527,7 @@ class KafkaApisTest extends Logging { ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", ) - val response = getInitializeShareGroupResponse( + val response = getInitializeShareGroupStateResponse( initRequestData, config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = false, @@ -12586,7 +12548,7 @@ class KafkaApisTest extends Logging { describedGroups: util.List[ShareGroupDescribeResponseData.DescribedGroup]): ShareGroupDescribeResponse = { val shareGroupDescribeRequestData = new ShareGroupDescribeRequestData() shareGroupDescribeRequestData.groupIds.addAll(groupIds) - val requestChannelRequest = buildRequest(new ShareGroupDescribeRequest.Builder(shareGroupDescribeRequestData, true).build()) + val requestChannelRequest = buildRequest(new ShareGroupDescribeRequest.Builder(shareGroupDescribeRequestData).build()) val future = new CompletableFuture[util.List[ShareGroupDescribeResponseData.DescribedGroup]]() when(groupCoordinator.shareGroupDescribe( @@ -12611,10 +12573,10 @@ class KafkaApisTest extends Logging { response } - def getReadShareGroupResponse(requestData: ReadShareGroupStateRequestData, configOverrides: Map[String, String] = Map.empty, - verifyNoErr: Boolean = true, authorizer: Authorizer = null, - readStateResult: util.List[ReadShareGroupStateResponseData.ReadStateResult]): ReadShareGroupStateResponse = { - val requestChannelRequest = buildRequest(new ReadShareGroupStateRequest.Builder(requestData, true).build()) + def getReadShareGroupStateResponse(requestData: ReadShareGroupStateRequestData, configOverrides: Map[String, String] = Map.empty, + verifyNoErr: Boolean = true, authorizer: Authorizer = null, + readStateResult: util.List[ReadShareGroupStateResponseData.ReadStateResult]): ReadShareGroupStateResponse = { + val requestChannelRequest = buildRequest(new ReadShareGroupStateRequest.Builder(requestData).build()) val future = new CompletableFuture[ReadShareGroupStateResponseData]() when(shareCoordinator.readState( @@ -12640,10 +12602,10 @@ class KafkaApisTest extends Logging { response } - def getReadShareGroupSummaryResponse(requestData: ReadShareGroupStateSummaryRequestData, configOverrides: Map[String, String] = Map.empty, - verifyNoErr: Boolean = true, authorizer: Authorizer = null, - readStateSummaryResult: util.List[ReadShareGroupStateSummaryResponseData.ReadStateSummaryResult]): ReadShareGroupStateSummaryResponse = { - val requestChannelRequest = buildRequest(new ReadShareGroupStateSummaryRequest.Builder(requestData, true).build()) + def getReadShareGroupStateSummaryResponse(requestData: ReadShareGroupStateSummaryRequestData, configOverrides: Map[String, String] = Map.empty, + verifyNoErr: Boolean = true, authorizer: Authorizer = null, + readStateSummaryResult: util.List[ReadShareGroupStateSummaryResponseData.ReadStateSummaryResult]): ReadShareGroupStateSummaryResponse = { + val requestChannelRequest = buildRequest(new ReadShareGroupStateSummaryRequest.Builder(requestData).build()) val future = new CompletableFuture[ReadShareGroupStateSummaryResponseData]() when(shareCoordinator.readStateSummary( @@ -12669,10 +12631,10 @@ class KafkaApisTest extends Logging { response } - def getWriteShareGroupResponse(requestData: WriteShareGroupStateRequestData, configOverrides: Map[String, String] = Map.empty, - verifyNoErr: Boolean = true, authorizer: Authorizer = null, - writeStateResult: util.List[WriteShareGroupStateResponseData.WriteStateResult]): WriteShareGroupStateResponse = { - val requestChannelRequest = buildRequest(new WriteShareGroupStateRequest.Builder(requestData, true).build()) + def getWriteShareGroupStateResponse(requestData: WriteShareGroupStateRequestData, configOverrides: Map[String, String] = Map.empty, + verifyNoErr: Boolean = true, authorizer: Authorizer = null, + writeStateResult: util.List[WriteShareGroupStateResponseData.WriteStateResult]): WriteShareGroupStateResponse = { + val requestChannelRequest = buildRequest(new WriteShareGroupStateRequest.Builder(requestData).build()) val future = new CompletableFuture[WriteShareGroupStateResponseData]() when(shareCoordinator.writeState( @@ -12698,10 +12660,10 @@ class KafkaApisTest extends Logging { response } - def getDeleteShareGroupResponse(requestData: DeleteShareGroupStateRequestData, configOverrides: Map[String, String] = Map.empty, - verifyNoErr: Boolean = true, authorizer: Authorizer = null, - deleteStateResult: util.List[DeleteShareGroupStateResponseData.DeleteStateResult]): DeleteShareGroupStateResponse = { - val requestChannelRequest = buildRequest(new DeleteShareGroupStateRequest.Builder(requestData, true).build()) + def getDeleteShareGroupStateResponse(requestData: DeleteShareGroupStateRequestData, configOverrides: Map[String, String] = Map.empty, + verifyNoErr: Boolean = true, authorizer: Authorizer = null, + deleteStateResult: util.List[DeleteShareGroupStateResponseData.DeleteStateResult]): DeleteShareGroupStateResponse = { + val requestChannelRequest = buildRequest(new DeleteShareGroupStateRequest.Builder(requestData).build()) val future = new CompletableFuture[DeleteShareGroupStateResponseData]() when(shareCoordinator.deleteState( @@ -12727,10 +12689,10 @@ class KafkaApisTest extends Logging { response } - def getInitializeShareGroupResponse(requestData: InitializeShareGroupStateRequestData, configOverrides: Map[String, String] = Map.empty, - verifyNoErr: Boolean = true, authorizer: Authorizer = null, - initStateResult: util.List[InitializeShareGroupStateResponseData.InitializeStateResult]): InitializeShareGroupStateResponse = { - val requestChannelRequest = buildRequest(new InitializeShareGroupStateRequest.Builder(requestData, true).build()) + def getInitializeShareGroupStateResponse(requestData: InitializeShareGroupStateRequestData, configOverrides: Map[String, String] = Map.empty, + verifyNoErr: Boolean = true, authorizer: Authorizer = null, + initStateResult: util.List[InitializeShareGroupStateResponseData.InitializeStateResult]): InitializeShareGroupStateResponse = { + val requestChannelRequest = buildRequest(new InitializeShareGroupStateRequest.Builder(requestData).build()) val future = new CompletableFuture[InitializeShareGroupStateResponseData]() when(shareCoordinator.initializeState( diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index 6f16f1b7a73ff..154afd34a6015 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -644,16 +644,16 @@ class RequestQuotaTest extends BaseRequestTest { new DescribeTopicPartitionsRequest.Builder(new DescribeTopicPartitionsRequestData()) case ApiKeys.SHARE_GROUP_HEARTBEAT => - new ShareGroupHeartbeatRequest.Builder(new ShareGroupHeartbeatRequestData(), true) + new ShareGroupHeartbeatRequest.Builder(new ShareGroupHeartbeatRequestData()) case ApiKeys.SHARE_GROUP_DESCRIBE => - new ShareGroupDescribeRequest.Builder(new ShareGroupDescribeRequestData(), true) + new ShareGroupDescribeRequest.Builder(new ShareGroupDescribeRequestData()) case ApiKeys.SHARE_FETCH => - new ShareFetchRequest.Builder(new ShareFetchRequestData(), true) + new ShareFetchRequest.Builder(new ShareFetchRequestData()) case ApiKeys.SHARE_ACKNOWLEDGE => - new ShareAcknowledgeRequest.Builder(new ShareAcknowledgeRequestData(), true) + new ShareAcknowledgeRequest.Builder(new ShareAcknowledgeRequestData()) case ApiKeys.ADD_RAFT_VOTER => new AddRaftVoterRequest.Builder(new AddRaftVoterRequestData()) @@ -665,19 +665,19 @@ class RequestQuotaTest extends BaseRequestTest { new UpdateRaftVoterRequest.Builder(new UpdateRaftVoterRequestData()) case ApiKeys.INITIALIZE_SHARE_GROUP_STATE => - new InitializeShareGroupStateRequest.Builder(new InitializeShareGroupStateRequestData(), true) + new InitializeShareGroupStateRequest.Builder(new InitializeShareGroupStateRequestData()) case ApiKeys.READ_SHARE_GROUP_STATE => - new ReadShareGroupStateRequest.Builder(new ReadShareGroupStateRequestData(), true) + new ReadShareGroupStateRequest.Builder(new ReadShareGroupStateRequestData()) case ApiKeys.WRITE_SHARE_GROUP_STATE => - new WriteShareGroupStateRequest.Builder(new WriteShareGroupStateRequestData(), true) + new WriteShareGroupStateRequest.Builder(new WriteShareGroupStateRequestData()) case ApiKeys.DELETE_SHARE_GROUP_STATE => - new DeleteShareGroupStateRequest.Builder(new DeleteShareGroupStateRequestData(), true) + new DeleteShareGroupStateRequest.Builder(new DeleteShareGroupStateRequestData()) case ApiKeys.READ_SHARE_GROUP_STATE_SUMMARY => - new ReadShareGroupStateSummaryRequest.Builder(new ReadShareGroupStateSummaryRequestData(), true) + new ReadShareGroupStateSummaryRequest.Builder(new ReadShareGroupStateSummaryRequestData()) case ApiKeys.STREAMS_GROUP_HEARTBEAT => new StreamsGroupHeartbeatRequest.Builder(new StreamsGroupHeartbeatRequestData(), true) @@ -686,13 +686,13 @@ class RequestQuotaTest extends BaseRequestTest { new StreamsGroupDescribeRequest.Builder(new StreamsGroupDescribeRequestData(), true) case ApiKeys.DESCRIBE_SHARE_GROUP_OFFSETS => - new DescribeShareGroupOffsetsRequest.Builder(new DescribeShareGroupOffsetsRequestData(), true) + new DescribeShareGroupOffsetsRequest.Builder(new DescribeShareGroupOffsetsRequestData()) case ApiKeys.ALTER_SHARE_GROUP_OFFSETS => - new AlterShareGroupOffsetsRequest.Builder(new AlterShareGroupOffsetsRequestData(), true) + new AlterShareGroupOffsetsRequest.Builder(new AlterShareGroupOffsetsRequestData()) case ApiKeys.DELETE_SHARE_GROUP_OFFSETS => - new DeleteShareGroupOffsetsRequest.Builder(new DeleteShareGroupOffsetsRequestData(), true) + new DeleteShareGroupOffsetsRequest.Builder(new DeleteShareGroupOffsetsRequestData()) case _ => throw new IllegalArgumentException("Unsupported API key " + apiKey) diff --git a/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala b/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala index 9fc3165bd7b5c..45afae3a0dc13 100644 --- a/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala @@ -44,11 +44,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo closeProducer } - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") - ) - ) + @ClusterTest def testShareFetchRequestIsInAccessibleWhenConfigsDisabled(): Unit = { val groupId: String = "group" val metadata: ShareRequestMetadata = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH) @@ -64,11 +60,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo assertEquals(0, shareFetchResponse.data.acquisitionLockTimeoutMs) } - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") - ) - ) + @ClusterTest def testShareAcknowledgeRequestIsInAccessibleWhenConfigsDisabled(): Unit = { val groupId: String = "group" val metadata: ShareRequestMetadata = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH) @@ -98,8 +90,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1") ), brokers = 2 ), @@ -153,8 +144,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1") ) ) ) @@ -220,8 +210,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1") ) ) ) @@ -322,8 +311,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1") ), brokers = 3 ), @@ -443,8 +431,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -563,7 +550,6 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true"), new ClusterConfigProperty(key = "group.share.record.lock.duration.ms", value = "15000") ) ), @@ -686,8 +672,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -801,8 +786,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -920,8 +904,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -1038,8 +1021,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -1163,7 +1145,6 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true"), new ClusterConfigProperty(key = "group.share.delivery.count.limit", value = "2") // Setting max delivery count config to 2 ) ), @@ -1326,8 +1307,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -1423,8 +1403,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -1524,8 +1503,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -1637,8 +1615,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -1759,8 +1736,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -1816,8 +1792,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -1867,8 +1842,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -1944,8 +1918,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -2026,8 +1999,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -2104,8 +2076,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -2187,8 +2158,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -2290,8 +2260,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1") ) ) ) @@ -2357,8 +2326,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1") ) ) ) diff --git a/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala b/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala index dfd478616da30..6c707c7bae13f 100644 --- a/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala @@ -29,7 +29,6 @@ import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.modern.share.ShareGroupConfig import org.apache.kafka.security.authorizer.AclEntry -import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Timeout @@ -42,15 +41,10 @@ import scala.jdk.CollectionConverters._ )) class ShareGroupDescribeRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG, value = "true") - ) - ) + @ClusterTest def testShareGroupDescribeIsInAccessibleWhenConfigsDisabled(): Unit = { val shareGroupDescribeRequest = new ShareGroupDescribeRequest.Builder( - new ShareGroupDescribeRequestData().setGroupIds(List("grp-1", "grp-2").asJava), - true + new ShareGroupDescribeRequestData().setGroupIds(List("grp-1", "grp-2").asJava) ).build(ApiKeys.SHARE_GROUP_DESCRIBE.latestVersion(isUnstableApiEnabled)) val shareGroupDescribeResponse = connectAndReceive[ShareGroupDescribeResponse](shareGroupDescribeRequest) diff --git a/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala b/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala index ea9308730c3c5..75b428ee0b41c 100644 --- a/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala @@ -42,7 +42,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { ) def testShareGroupHeartbeatIsInAccessibleWhenConfigsDisabled(): Unit = { val shareGroupHeartbeatRequest = new ShareGroupHeartbeatRequest.Builder( - new ShareGroupHeartbeatRequestData(), true + new ShareGroupHeartbeatRequestData() ).build() val shareGroupHeartbeatResponse = connectAndReceive(shareGroupHeartbeatRequest) @@ -76,8 +76,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { .setGroupId("grp") .setMemberId(Uuid.randomUuid.toString) .setMemberEpoch(0) - .setSubscribedTopicNames(List("foo").asJava), - true + .setSubscribedTopicNames(List("foo").asJava) ).build() // Send the request until receiving a successful response. There is a delay @@ -105,8 +104,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(shareGroupHeartbeatResponse.data.memberId) - .setMemberEpoch(shareGroupHeartbeatResponse.data.memberEpoch), - true + .setMemberEpoch(shareGroupHeartbeatResponse.data.memberEpoch) ).build() // This is the expected assignment. here @@ -132,8 +130,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(shareGroupHeartbeatResponse.data.memberId) - .setMemberEpoch(-1), - true + .setMemberEpoch(-1) ).build() shareGroupHeartbeatResponse = connectAndReceive(shareGroupHeartbeatRequest) @@ -171,8 +168,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { .setGroupId("grp") .setMemberId(Uuid.randomUuid.toString) .setMemberEpoch(0) - .setSubscribedTopicNames(List("foo").asJava), - true + .setSubscribedTopicNames(List("foo").asJava) ).build() // Send the request until receiving a successful response. There is a delay @@ -195,8 +191,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { .setGroupId("grp") .setMemberId(Uuid.randomUuid.toString) .setMemberEpoch(0) - .setSubscribedTopicNames(List("foo").asJava), - true + .setSubscribedTopicNames(List("foo").asJava) ).build() // Send the second member request until receiving a successful response. @@ -225,8 +220,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId1) - .setMemberEpoch(1), - true + .setMemberEpoch(1) ).build() // Heartbeats until the partitions are assigned for member 1. @@ -241,8 +235,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId1) - .setMemberEpoch(shareGroupHeartbeatResponse.data.memberEpoch()), - true + .setMemberEpoch(shareGroupHeartbeatResponse.data.memberEpoch()) ).build() false } @@ -257,8 +250,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId2) - .setMemberEpoch(2), - true + .setMemberEpoch(2) ).build() // Heartbeats until the partitions are assigned for member 2. @@ -288,8 +280,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId1) - .setMemberEpoch(3), - true + .setMemberEpoch(3) ).build() // Heartbeats until the response for no change of assignment occurs for member 1 with same epoch. @@ -333,8 +324,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { .setGroupId("grp") .setMemberId(Uuid.randomUuid.toString) .setMemberEpoch(0) - .setSubscribedTopicNames(List("foo").asJava), - true + .setSubscribedTopicNames(List("foo").asJava) ).build() // Send the request until receiving a successful response. There is a delay @@ -369,8 +359,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId) - .setMemberEpoch(1), - true + .setMemberEpoch(1) ).build() TestUtils.waitUntilTrue(() => { @@ -387,8 +376,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberEpoch(-1) - .setMemberId(memberId), - true + .setMemberId(memberId) ).build() // Send the member request until receiving a successful response. @@ -406,8 +394,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { .setGroupId("grp") .setMemberEpoch(0) .setMemberId(memberId) - .setSubscribedTopicNames(List("foo").asJava), - true + .setSubscribedTopicNames(List("foo").asJava) ).build() shareGroupHeartbeatResponse = connectAndReceive(shareGroupHeartbeatRequest) @@ -446,8 +433,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { .setGroupId("grp") .setMemberId(Uuid.randomUuid.toString) .setMemberEpoch(0) - .setSubscribedTopicNames(List("foo", "bar", "baz").asJava), - true + .setSubscribedTopicNames(List("foo", "bar", "baz").asJava) ).build() // Send the request until receiving a successful response. There is a delay // here because the group coordinator is loaded in the background. @@ -487,8 +473,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId) - .setMemberEpoch(1), - true + .setMemberEpoch(1) ).build() cluster.waitForTopic("foo", 2) @@ -526,8 +511,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId) - .setMemberEpoch(3), - true + .setMemberEpoch(3) ).build() TestUtils.waitUntilTrue(() => { @@ -558,8 +542,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId) - .setMemberEpoch(5), - true + .setMemberEpoch(5) ).build() TestUtils.waitUntilTrue(() => { @@ -592,8 +575,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId) - .setMemberEpoch(7), - true + .setMemberEpoch(7) ).build() TestUtils.waitUntilTrue(() => { @@ -640,8 +622,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { .setGroupId("grp") .setMemberId(Uuid.randomUuid.toString) .setMemberEpoch(0) - .setSubscribedTopicNames(List("foo").asJava), - true + .setSubscribedTopicNames(List("foo").asJava) ).build() // Send the request until receiving a successful response. There is a delay @@ -677,8 +658,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId) - .setMemberEpoch(memberEpoch), - true + .setMemberEpoch(memberEpoch) ).build() TestUtils.waitUntilTrue(() => { @@ -697,8 +677,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { .setGroupId("grp") .setMemberId(memberId) .setMemberEpoch(memberEpoch) - .setSubscribedTopicNames(List("foo", "bar").asJava), - true + .setSubscribedTopicNames(List("foo", "bar").asJava) ).build() val barId = TestUtils.createTopicWithAdminRaw( @@ -729,7 +708,6 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { .setGroupId("grp") .setMemberId(memberId) .setMemberEpoch(shareGroupHeartbeatResponse.data.memberEpoch), - true ).build() false } @@ -744,8 +722,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId) - .setMemberEpoch(memberEpoch), - true + .setMemberEpoch(memberEpoch) ).build() TestUtils.waitUntilTrue(() => { @@ -764,8 +741,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId) - .setMemberEpoch(memberEpoch), - true + .setMemberEpoch(memberEpoch) ).build() TestUtils.waitUntilTrue(() => { @@ -779,8 +755,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { .setGroupId("grp") .setMemberId(memberId) .setMemberEpoch(0) - .setSubscribedTopicNames(List("foo", "bar").asJava), - true + .setSubscribedTopicNames(List("foo", "bar").asJava) ).build() TestUtils.waitUntilTrue(() => { @@ -822,8 +797,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { .setGroupId("grp") .setMemberId(Uuid.randomUuid.toString) .setMemberEpoch(0) - .setSubscribedTopicNames(List("foo").asJava), - true + .setSubscribedTopicNames(List("foo").asJava) ).build() // Send the request until receiving a successful response. There is a delay // here because the group coordinator is loaded in the background. @@ -853,8 +827,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId) - .setMemberEpoch(1), - true + .setMemberEpoch(1) ).build() TestUtils.waitUntilTrue(() => { @@ -875,8 +848,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId) - .setMemberEpoch(2), - true + .setMemberEpoch(2) ).build() // Should receive no error and no assignment changes. diff --git a/server-common/src/main/java/org/apache/kafka/server/share/persister/PersisterStateManager.java b/server-common/src/main/java/org/apache/kafka/server/share/persister/PersisterStateManager.java index 1b2a03b0b8bd9..9e451b4b1cd05 100644 --- a/server-common/src/main/java/org/apache/kafka/server/share/persister/PersisterStateManager.java +++ b/server-common/src/main/java/org/apache/kafka/server/share/persister/PersisterStateManager.java @@ -1475,8 +1475,7 @@ private static AbstractRequest.Builder coalesceReadSu .map(entry -> new ReadShareGroupStateSummaryRequestData.ReadStateSummaryData() .setTopicId(entry.getKey()) .setPartitions(entry.getValue())) - .collect(Collectors.toList())), - true + .collect(Collectors.toList())) ); } diff --git a/tests/kafkatest/services/kafka/config_property.py b/tests/kafkatest/services/kafka/config_property.py index 049af5a10a9a7..1d0e3a3bd5e00 100644 --- a/tests/kafkatest/services/kafka/config_property.py +++ b/tests/kafkatest/services/kafka/config_property.py @@ -77,7 +77,6 @@ GROUP_COORDINATOR_REBALANCE_PROTOCOLS="group.coordinator.rebalance.protocols" CONSUMER_GROUP_MIGRATION_POLICY = "group.consumer.migration.policy" -UNSTABLE_API_VERSIONS_ENABLE = "unstable.api.versions.enable" SHARE_COORDINATOR_STATE_TOPIC_REPLICATION_FACTOR ="share.coordinator.state.topic.replication.factor" SHARE_COORDINATOR_STATE_TOPIC_MIN_ISR = "share.coordinator.state.topic.min.isr" SHARE_GROUP_ENABLE = "group.share.enable" diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index a55844548ba0a..da62a0da3261d 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -778,7 +778,6 @@ def prop_file(self, node): if self.use_share_groups is not None and self.use_share_groups is True: override_configs[config_property.SHARE_GROUP_ENABLE] = str(self.use_share_groups) - override_configs[config_property.UNSTABLE_API_VERSIONS_ENABLE] = str(self.use_share_groups) override_configs[config_property.GROUP_COORDINATOR_REBALANCE_PROTOCOLS] = 'classic,consumer,share' #update template configs with test override configs From 22c5794bc314679bfae4d5edcb1909460e4bab7d Mon Sep 17 00:00:00 2001 From: Chirag Wadhwa <122860692+chirag-wadhwa5@users.noreply.github.com> Date: Tue, 22 Apr 2025 19:29:35 +0530 Subject: [PATCH 11/37] KAFKA-19159: Removed time based evictions for share sessions (#19500) Currently the share session cache is desgined like the fetch session cache. If the cache is full and a new share session is trying to get get initialized, then the sessions which haven't been touched for more than 2minutes are evicted. This wouldn't be right for share sessions as the members also hold locks on the acquired records, and session eviction would mean theose locks will need to be dropped and the corresponding records re-delivered. This PR removes the time based eviction logic for share sessions. Refer: [KAFKA-19159](https://issues.apache.org/jira/browse/KAFKA-19159) Reviewers: Apoorv Mittal , Chia-Ping Tsai --- .../server/share/SharePartitionManager.java | 7 +- .../scala/kafka/server/BrokerServer.scala | 4 +- .../share/SharePartitionManagerTest.java | 119 ++---------------- .../unit/kafka/server/KafkaApisTest.scala | 36 +++--- .../server/share/session/LastUsedKey.java | 66 ---------- .../server/share/session/ShareSession.java | 39 ++---- .../share/session/ShareSessionCache.java | 62 ++------- .../share/session/ShareSessionCacheTest.java | 88 +++++-------- 8 files changed, 84 insertions(+), 337 deletions(-) delete mode 100644 server/src/main/java/org/apache/kafka/server/share/session/LastUsedKey.java diff --git a/core/src/main/java/kafka/server/share/SharePartitionManager.java b/core/src/main/java/kafka/server/share/SharePartitionManager.java index 1b50a70c18ca6..a53f846a01c24 100644 --- a/core/src/main/java/kafka/server/share/SharePartitionManager.java +++ b/core/src/main/java/kafka/server/share/SharePartitionManager.java @@ -448,8 +448,7 @@ public ShareFetchContext newContext(String groupId, List share ImplicitLinkedHashCollection<>(shareFetchData.size()); shareFetchData.forEach(topicIdPartition -> cachedSharePartitions.mustAdd(new CachedSharePartition(topicIdPartition, false))); - ShareSessionKey responseShareSessionKey = cache.maybeCreateSession(groupId, reqMetadata.memberId(), - time.milliseconds(), cachedSharePartitions); + ShareSessionKey responseShareSessionKey = cache.maybeCreateSession(groupId, reqMetadata.memberId(), cachedSharePartitions); if (responseShareSessionKey == null) { log.error("Could not create a share session for group {} member {}", groupId, reqMetadata.memberId()); throw Errors.SHARE_SESSION_NOT_FOUND.exception(); @@ -476,7 +475,7 @@ public ShareFetchContext newContext(String groupId, List share } Map> modifiedTopicIdPartitions = shareSession.update( shareFetchData, toForget); - cache.touch(shareSession, time.milliseconds()); + cache.updateNumPartitions(shareSession); shareSession.epoch = ShareRequestMetadata.nextEpoch(shareSession.epoch); log.debug("Created a new ShareSessionContext for session key {}, epoch {}: " + "added {}, updated {}, removed {}", shareSession.key(), shareSession.epoch, @@ -517,7 +516,7 @@ public void acknowledgeSessionUpdate(String groupId, ShareRequestMetadata reqMet shareSession.epoch, reqMetadata.epoch()); throw Errors.INVALID_SHARE_SESSION_EPOCH.exception(); } - cache.touch(shareSession, time.milliseconds()); + cache.updateNumPartitions(shareSession); shareSession.epoch = ShareRequestMetadata.nextEpoch(shareSession.epoch); } } diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 8f9b983cb782f..cbd2a91fbc5b9 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -427,8 +427,8 @@ class BrokerServer( val fetchManager = new FetchManager(Time.SYSTEM, new FetchSessionCache(fetchSessionCacheShards)) val shareFetchSessionCache : ShareSessionCache = new ShareSessionCache( - config.shareGroupConfig.shareGroupMaxGroups * config.groupCoordinatorConfig.shareGroupMaxSize, - KafkaBroker.MIN_INCREMENTAL_FETCH_SESSION_EVICTION_MS) + config.shareGroupConfig.shareGroupMaxGroups * config.groupCoordinatorConfig.shareGroupMaxSize + ) sharePartitionManager = new SharePartitionManager( replicaManager, diff --git a/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java b/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java index a69c6c83071b2..83601a0220161 100644 --- a/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java +++ b/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java @@ -121,7 +121,6 @@ import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; @@ -185,7 +184,7 @@ public void tearDown() throws Exception { @Test public void testNewContextReturnsFinalContextWithoutRequestData() { - ShareSessionCache cache = new ShareSessionCache(10, 1000); + ShareSessionCache cache = new ShareSessionCache(10); sharePartitionManager = SharePartitionManagerBuilder.builder() .withCache(cache) .build(); @@ -212,7 +211,7 @@ public void testNewContextReturnsFinalContextWithoutRequestData() { @Test public void testNewContextReturnsFinalContextWithRequestData() { - ShareSessionCache cache = new ShareSessionCache(10, 1000); + ShareSessionCache cache = new ShareSessionCache(10); sharePartitionManager = SharePartitionManagerBuilder.builder() .withCache(cache) .build(); @@ -244,7 +243,7 @@ public void testNewContextReturnsFinalContextWithRequestData() { @Test public void testNewContextReturnsFinalContextWhenTopicPartitionsArePresentInRequestData() { - ShareSessionCache cache = new ShareSessionCache(10, 1000); + ShareSessionCache cache = new ShareSessionCache(10); sharePartitionManager = SharePartitionManagerBuilder.builder() .withCache(cache) .build(); @@ -275,7 +274,7 @@ public void testNewContextReturnsFinalContextWhenTopicPartitionsArePresentInRequ @Test public void testNewContext() { - ShareSessionCache cache = new ShareSessionCache(10, 1000); + ShareSessionCache cache = new ShareSessionCache(10); sharePartitionManager = SharePartitionManagerBuilder.builder() .withCache(cache) .build(); @@ -372,100 +371,6 @@ public void testNewContext() { assertEquals(0, cache.size()); } - @Test - public void testShareSessionExpiration() { - ShareSessionCache cache = new ShareSessionCache(2, 1000); - sharePartitionManager = SharePartitionManagerBuilder.builder() - .withCache(cache) - .withTime(time) - .build(); - - Map topicNames = new HashMap<>(); - Uuid fooId = Uuid.randomUuid(); - topicNames.put(fooId, "foo"); - TopicIdPartition foo0 = new TopicIdPartition(fooId, new TopicPartition("foo", 0)); - TopicIdPartition foo1 = new TopicIdPartition(fooId, new TopicPartition("foo", 1)); - - // Create a new share session, session 1 - List session1req = List.of(foo0, foo1); - - String groupId = "grp"; - ShareRequestMetadata reqMetadata1 = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH); - - ShareFetchContext session1context = sharePartitionManager.newContext(groupId, session1req, EMPTY_PART_LIST, reqMetadata1, false); - assertInstanceOf(ShareSessionContext.class, session1context); - - LinkedHashMap respData1 = new LinkedHashMap<>(); - respData1.put(foo0, new ShareFetchResponseData.PartitionData().setPartitionIndex(foo0.partition())); - respData1.put(foo1, new ShareFetchResponseData.PartitionData().setPartitionIndex(foo1.partition())); - - ShareFetchResponse session1resp = session1context.updateAndGenerateResponseData(groupId, reqMetadata1.memberId(), respData1); - assertEquals(Errors.NONE, session1resp.error()); - assertEquals(2, session1resp.responseData(topicNames).size()); - - ShareSessionKey session1Key = new ShareSessionKey(groupId, reqMetadata1.memberId()); - // check share session entered into cache - assertNotNull(cache.get(session1Key)); - - time.sleep(500); - - // Create a second new share session - List session2req = List.of(foo0, foo1); - - ShareRequestMetadata reqMetadata2 = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH); - - ShareFetchContext session2context = sharePartitionManager.newContext(groupId, session2req, EMPTY_PART_LIST, reqMetadata2, false); - assertInstanceOf(ShareSessionContext.class, session2context); - - LinkedHashMap respData2 = new LinkedHashMap<>(); - respData2.put(foo0, new ShareFetchResponseData.PartitionData().setPartitionIndex(foo0.partition())); - respData2.put(foo1, new ShareFetchResponseData.PartitionData().setPartitionIndex(foo1.partition())); - - ShareFetchResponse session2resp = session2context.updateAndGenerateResponseData(groupId, reqMetadata2.memberId(), respData2); - assertEquals(Errors.NONE, session2resp.error()); - assertEquals(2, session2resp.responseData(topicNames).size()); - - ShareSessionKey session2Key = new ShareSessionKey(groupId, reqMetadata2.memberId()); - - // both newly created entries are present in cache - assertNotNull(cache.get(session1Key)); - assertNotNull(cache.get(session2Key)); - - time.sleep(500); - - // Create a subsequent share fetch context for session 1 - ShareFetchContext session1context2 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, - new ShareRequestMetadata(reqMetadata1.memberId(), 1), true); - assertInstanceOf(ShareSessionContext.class, session1context2); - - // total sleep time will now be large enough that share session 1 will be evicted if not correctly touched - time.sleep(501); - - // create one final share session to test that the least recently used entry is evicted - // the second share session should be evicted because the first share session was incrementally fetched - // more recently than the second session was created - List session3req = List.of(foo0, foo1); - - ShareRequestMetadata reqMetadata3 = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH); - - ShareFetchContext session3context = sharePartitionManager.newContext(groupId, session3req, EMPTY_PART_LIST, reqMetadata3, false); - - LinkedHashMap respData3 = new LinkedHashMap<>(); - respData3.put(foo0, new ShareFetchResponseData.PartitionData().setPartitionIndex(foo0.partition())); - respData3.put(foo1, new ShareFetchResponseData.PartitionData().setPartitionIndex(foo1.partition())); - - ShareFetchResponse session3resp = session3context.updateAndGenerateResponseData(groupId, reqMetadata3.memberId(), respData3); - assertEquals(Errors.NONE, session3resp.error()); - assertEquals(2, session3resp.responseData(topicNames).size()); - - ShareSessionKey session3Key = new ShareSessionKey(groupId, reqMetadata3.memberId()); - - assertNotNull(cache.get(session1Key)); - assertNull(cache.get(session2Key), "share session 2 should have been evicted by latest share session, " + - "as share session 1 was used more recently"); - assertNotNull(cache.get(session3Key)); - } - @Test public void testSubsequentShareSession() { sharePartitionManager = SharePartitionManagerBuilder.builder().build(); @@ -530,7 +435,7 @@ public void testSubsequentShareSession() { @Test public void testZeroSizeShareSession() { - ShareSessionCache cache = new ShareSessionCache(10, 1000); + ShareSessionCache cache = new ShareSessionCache(10); sharePartitionManager = SharePartitionManagerBuilder.builder() .withCache(cache) .build(); @@ -576,7 +481,7 @@ public void testZeroSizeShareSession() { @Test public void testToForgetPartitions() { String groupId = "grp"; - ShareSessionCache cache = new ShareSessionCache(10, 1000); + ShareSessionCache cache = new ShareSessionCache(10); sharePartitionManager = SharePartitionManagerBuilder.builder() .withCache(cache) .build(); @@ -614,7 +519,7 @@ public void testToForgetPartitions() { @Test public void testShareSessionUpdateTopicIdsBrokerSide() { String groupId = "grp"; - ShareSessionCache cache = new ShareSessionCache(10, 1000); + ShareSessionCache cache = new ShareSessionCache(10); sharePartitionManager = SharePartitionManagerBuilder.builder() .withCache(cache) .build(); @@ -665,7 +570,7 @@ public void testShareSessionUpdateTopicIdsBrokerSide() { @Test public void testGetErroneousAndValidTopicIdPartitions() { - ShareSessionCache cache = new ShareSessionCache(10, 1000); + ShareSessionCache cache = new ShareSessionCache(10); sharePartitionManager = SharePartitionManagerBuilder.builder() .withCache(cache) .build(); @@ -758,7 +663,7 @@ public void testGetErroneousAndValidTopicIdPartitions() { @Test public void testShareFetchContextResponseSize() { - ShareSessionCache cache = new ShareSessionCache(10, 1000); + ShareSessionCache cache = new ShareSessionCache(10); sharePartitionManager = SharePartitionManagerBuilder.builder() .withCache(cache) .build(); @@ -859,7 +764,7 @@ public void testShareFetchContextResponseSize() { @Test public void testCachedTopicPartitionsWithNoTopicPartitions() { - ShareSessionCache cache = new ShareSessionCache(10, 1000); + ShareSessionCache cache = new ShareSessionCache(10); sharePartitionManager = SharePartitionManagerBuilder.builder() .withCache(cache) .build(); @@ -870,7 +775,7 @@ public void testCachedTopicPartitionsWithNoTopicPartitions() { @Test public void testCachedTopicPartitionsForValidShareSessions() { - ShareSessionCache cache = new ShareSessionCache(10, 1000); + ShareSessionCache cache = new ShareSessionCache(10); sharePartitionManager = SharePartitionManagerBuilder.builder() .withCache(cache) .build(); @@ -3108,7 +3013,7 @@ static class SharePartitionManagerBuilder { private final Persister persister = new NoOpStatePersister(); private ReplicaManager replicaManager = mock(ReplicaManager.class); private Time time = new MockTime(); - private ShareSessionCache cache = new ShareSessionCache(10, 1000); + private ShareSessionCache cache = new ShareSessionCache(10); private Map partitionCacheMap = new HashMap<>(); private Timer timer = new MockTimer(); private ShareGroupMetrics shareGroupMetrics = new ShareGroupMetrics(time); diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 84e1f1ebb673b..02541097d4c32 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -4046,9 +4046,8 @@ class KafkaApisTest extends Logging { when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenThrow( Errors.INVALID_REQUEST.exception() - ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( - new ShareSessionKey(groupId, memberId), cachedSharePartitions, 0L, 0L, 2 - ))) + ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), + new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2))) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( any[RequestChannel.Request](), anyDouble, anyLong)).thenReturn(0) @@ -4299,9 +4298,8 @@ class KafkaApisTest extends Logging { new TopicIdPartition(topicId, partitionIndex, topicName), false)) when(sharePartitionManager.newContext(any(), any(), any(), any(), any())) - .thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( - new ShareSessionKey(groupId, memberId), cachedSharePartitions, 0L, 0L, 2)) - ) + .thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), + new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2))) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( any[RequestChannel.Request](), anyDouble, anyLong)).thenReturn(0) @@ -4361,9 +4359,8 @@ class KafkaApisTest extends Logging { new TopicIdPartition(topicId, partitionIndex, topicName), false)) when(sharePartitionManager.newContext(any(), any(), any(), any(), any())) - .thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( - new ShareSessionKey(groupId, memberId), cachedSharePartitions, 0L, 0L, 2)) - ) + .thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), + new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2))) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( any[RequestChannel.Request](), anyDouble, anyLong)).thenReturn(0) @@ -4718,10 +4715,10 @@ class KafkaApisTest extends Logging { new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( new TopicIdPartition(topicId, partitionIndex, topicName) )) - ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( - new ShareSessionKey(groupId, memberId), cachedSharePartitions, 0L, 0L, 2)) - ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 2), new ShareSession( - new ShareSessionKey(groupId, memberId), cachedSharePartitions, 0L, 10L, 3)) + ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), + new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2)) + ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 2), + new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions, 3)) ) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( @@ -4986,10 +4983,10 @@ class KafkaApisTest extends Logging { new TopicIdPartition(topicId2, new TopicPartition(topicName2, 0)), new TopicIdPartition(topicId2, new TopicPartition(topicName2, 1)) )) - ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( - new ShareSessionKey(groupId, memberId), cachedSharePartitions1, 0L, 0L, 2)) - ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 2), new ShareSession( - new ShareSessionKey(groupId, memberId), cachedSharePartitions2, 0L, 0L, 3)) + ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), + new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions1, 2)) + ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 2), + new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions2, 3)) ).thenReturn(new FinalContext()) when(sharePartitionManager.releaseSession(any(), any())).thenReturn( @@ -5963,9 +5960,8 @@ class KafkaApisTest extends Logging { new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( new TopicIdPartition(topicId, partitionIndex, topicName) )) - ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( - new ShareSessionKey(groupId, memberId), cachedSharePartitions, 0L, 0L, 2)) - ) + ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), + new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2))) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( any[RequestChannel.Request](), anyDouble, anyLong)).thenReturn(0) diff --git a/server/src/main/java/org/apache/kafka/server/share/session/LastUsedKey.java b/server/src/main/java/org/apache/kafka/server/share/session/LastUsedKey.java deleted file mode 100644 index e6084f8e9b5f6..0000000000000 --- a/server/src/main/java/org/apache/kafka/server/share/session/LastUsedKey.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.server.share.session; - -import java.util.Objects; - -public class LastUsedKey implements Comparable { - private final ShareSessionKey key; - private final long lastUsedMs; - - public LastUsedKey(ShareSessionKey key, long lastUsedMs) { - this.key = key; - this.lastUsedMs = lastUsedMs; - } - - public ShareSessionKey key() { - return key; - } - - public long lastUsedMs() { - return lastUsedMs; - } - - @Override - public int hashCode() { - return Objects.hash(key, lastUsedMs); - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (obj == null) { - return false; - } - if (getClass() != obj.getClass()) { - return false; - } - LastUsedKey other = (LastUsedKey) obj; - return lastUsedMs == other.lastUsedMs && Objects.equals(key, other.key); - } - - @Override - public int compareTo(LastUsedKey other) { - int res = Long.compare(lastUsedMs, other.lastUsedMs); - if (res != 0) - return res; - return Integer.compare(key.hashCode(), other.key.hashCode()); - } -} diff --git a/server/src/main/java/org/apache/kafka/server/share/session/ShareSession.java b/server/src/main/java/org/apache/kafka/server/share/session/ShareSession.java index 362f32e61975e..5cb800c552453 100644 --- a/server/src/main/java/org/apache/kafka/server/share/session/ShareSession.java +++ b/server/src/main/java/org/apache/kafka/server/share/session/ShareSession.java @@ -38,9 +38,7 @@ public enum ModifiedTopicIdPartitionType { private final ShareSessionKey key; private final ImplicitLinkedHashCollection partitionMap; - private final long creationMs; - private long lastUsedMs; // visible for testing public int epoch; // This is used by the ShareSessionCache to store the last known size of this session. @@ -54,17 +52,11 @@ public enum ModifiedTopicIdPartitionType { * * @param key The share session key to identify the share session uniquely. * @param partitionMap The CachedPartitionMap. - * @param creationMs The time in milliseconds when this share session was created. - * @param lastUsedMs The last used time in milliseconds. This should only be updated by - * ShareSessionCache#touch. * @param epoch The share session sequence number. */ - public ShareSession(ShareSessionKey key, ImplicitLinkedHashCollection partitionMap, - long creationMs, long lastUsedMs, int epoch) { + public ShareSession(ShareSessionKey key, ImplicitLinkedHashCollection partitionMap, int epoch) { this.key = key; this.partitionMap = partitionMap; - this.creationMs = creationMs; - this.lastUsedMs = lastUsedMs; this.epoch = epoch; } @@ -76,18 +68,6 @@ public synchronized int cachedSize() { return cachedSize; } - public synchronized void cachedSize(int size) { - cachedSize = size; - } - - public synchronized long lastUsedMs() { - return lastUsedMs; - } - - public synchronized void lastUsedMs(long ts) { - lastUsedMs = ts; - } - public synchronized ImplicitLinkedHashCollection partitionMap() { return partitionMap; } @@ -105,10 +85,6 @@ public synchronized Boolean isEmpty() { return partitionMap.isEmpty(); } - public synchronized LastUsedKey lastUsedKey() { - return new LastUsedKey(key, lastUsedMs); - } - // Update the cached partition data based on the request. public synchronized Map> update( List shareFetchData, @@ -138,6 +114,17 @@ public synchronized Map> up return result; } + /** + * Updates the cached size of the session to represent the current partitionMap size. + * @return The difference between the current cached size and the previously stored cached size. This is required to + * update the total number of share partitions stored in the share session cache. + */ + public synchronized int updateCachedSize() { + var previousSize = cachedSize; + cachedSize = partitionMap.size(); + return previousSize != -1 ? cachedSize - previousSize : cachedSize; + } + public static String partitionsToLogString(Collection partitions, Boolean traceEnabled) { if (traceEnabled) { return String.format("( %s )", String.join(", ", partitions.toString())); @@ -149,8 +136,6 @@ public String toString() { return "ShareSession(" + "key=" + key + ", partitionMap=" + partitionMap + - ", creationMs=" + creationMs + - ", lastUsedMs=" + lastUsedMs + ", epoch=" + epoch + ", cachedSize=" + cachedSize + ")"; diff --git a/server/src/main/java/org/apache/kafka/server/share/session/ShareSessionCache.java b/server/src/main/java/org/apache/kafka/server/share/session/ShareSessionCache.java index dc870fc9c252c..0b06ea535be7e 100644 --- a/server/src/main/java/org/apache/kafka/server/share/session/ShareSessionCache.java +++ b/server/src/main/java/org/apache/kafka/server/share/session/ShareSessionCache.java @@ -27,7 +27,6 @@ import java.util.HashMap; import java.util.Map; -import java.util.TreeMap; import java.util.concurrent.TimeUnit; /** @@ -53,19 +52,14 @@ public class ShareSessionCache { private final Meter evictionsMeter; private final int maxEntries; - private final long evictionMs; private long numPartitions = 0; // A map of session key to ShareSession. private final Map sessions = new HashMap<>(); - // Maps last used times to sessions. - private final TreeMap lastUsed = new TreeMap<>(); - @SuppressWarnings("this-escape") - public ShareSessionCache(int maxEntries, long evictionMs) { + public ShareSessionCache(int maxEntries) { this.maxEntries = maxEntries; - this.evictionMs = evictionMs; // Register metrics for ShareSessionCache. KafkaMetricsGroup metricsGroup = new KafkaMetricsGroup("kafka.server", "ShareSessionCache"); metricsGroup.newGauge(SHARE_SESSIONS_COUNT, this::size); @@ -108,9 +102,6 @@ public synchronized ShareSession remove(ShareSessionKey key) { * @return The removed session, or None if there was no such session. */ public synchronized ShareSession remove(ShareSession session) { - synchronized (session) { - lastUsed.remove(session.lastUsedKey()); - } ShareSession removeResult = sessions.remove(session.key()); if (removeResult != null) { numPartitions = numPartitions - session.cachedSize(); @@ -119,64 +110,27 @@ public synchronized ShareSession remove(ShareSession session) { } /** - * Update a session's position in the lastUsed tree. + * Update the size of the cache by updating the total number of share partitions. * * @param session The session. - * @param now The current time in milliseconds. */ - public synchronized void touch(ShareSession session, long now) { - synchronized (session) { - // Update the lastUsed map. - lastUsed.remove(session.lastUsedKey()); - session.lastUsedMs(now); - lastUsed.put(session.lastUsedKey(), session); - - int oldSize = session.cachedSize(); - if (oldSize != -1) { - numPartitions = numPartitions - oldSize; - } - session.cachedSize(session.size()); - numPartitions = numPartitions + session.cachedSize(); - } - } - - /** - * Try to evict an entry from the session cache. - *

- * A proposed new element A may evict an existing element B if: - * B is considered "stale" because it has been inactive for a long time. - * - * @param now The current time in milliseconds. - * @return True if an entry was evicted; false otherwise. - */ - public synchronized boolean tryEvict(long now) { - // Try to evict an entry which is stale. - Map.Entry lastUsedEntry = lastUsed.firstEntry(); - if (lastUsedEntry == null) { - return false; - } else if (now - lastUsedEntry.getKey().lastUsedMs() > evictionMs) { - ShareSession session = lastUsedEntry.getValue(); - remove(session); - evictionsMeter.mark(); - return true; - } - return false; + public synchronized void updateNumPartitions(ShareSession session) { + numPartitions += session.updateCachedSize(); } /** * Maybe create a new session and add it to the cache. * @param groupId - The group id in the share fetch request. * @param memberId - The member id in the share fetch request. - * @param now - The current time in milliseconds. * @param partitionMap - The topic partitions to be added to the session. * @return - The session key if the session was created, or null if the session was not created. */ - public synchronized ShareSessionKey maybeCreateSession(String groupId, Uuid memberId, long now, ImplicitLinkedHashCollection partitionMap) { - if (sessions.size() < maxEntries || tryEvict(now)) { + public synchronized ShareSessionKey maybeCreateSession(String groupId, Uuid memberId, ImplicitLinkedHashCollection partitionMap) { + if (sessions.size() < maxEntries) { ShareSession session = new ShareSession(new ShareSessionKey(groupId, memberId), partitionMap, - now, now, ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH)); + ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH)); sessions.put(session.key(), session); - touch(session, now); + updateNumPartitions(session); return session.key(); } return null; diff --git a/server/src/test/java/org/apache/kafka/server/share/session/ShareSessionCacheTest.java b/server/src/test/java/org/apache/kafka/server/share/session/ShareSessionCacheTest.java index 4de1ffa497582..ca18de5b65c47 100644 --- a/server/src/test/java/org/apache/kafka/server/share/session/ShareSessionCacheTest.java +++ b/server/src/test/java/org/apache/kafka/server/share/session/ShareSessionCacheTest.java @@ -33,7 +33,6 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; -import static org.junit.jupiter.api.Assertions.assertTrue; public class ShareSessionCacheTest { @@ -44,46 +43,24 @@ public void setUp() { @Test public void testShareSessionCache() throws InterruptedException { - ShareSessionCache cache = new ShareSessionCache(3, 100); + ShareSessionCache cache = new ShareSessionCache(3); assertEquals(0, cache.size()); - ShareSessionKey key1 = cache.maybeCreateSession("grp", Uuid.randomUuid(), 0, mockedSharePartitionMap(10)); - ShareSessionKey key2 = cache.maybeCreateSession("grp", Uuid.randomUuid(), 10, mockedSharePartitionMap(20)); - ShareSessionKey key3 = cache.maybeCreateSession("grp", Uuid.randomUuid(), 20, mockedSharePartitionMap(30)); - assertNull(cache.maybeCreateSession("grp", Uuid.randomUuid(), 30, mockedSharePartitionMap(40))); - assertNull(cache.maybeCreateSession("grp", Uuid.randomUuid(), 40, mockedSharePartitionMap(5))); + ShareSessionKey key1 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(10)); + ShareSessionKey key2 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(20)); + ShareSessionKey key3 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(30)); + assertNull(cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(40))); + assertNull(cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(5))); assertShareCacheContains(cache, List.of(key1, key2, key3)); - TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_SESSIONS_COUNT).intValue() == 3, - "Share session count should be 3."); - TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_PARTITIONS_COUNT).intValue() == 60, - "Share partition count should be 60."); - assertEquals(0, cache.evictionsMeter().count()); - - // Touch the sessions to update the last used time, so that the key-2 can be evicted. - cache.touch(cache.get(key1), 200); - ShareSessionKey key4 = cache.maybeCreateSession("grp", Uuid.randomUuid(), 210, mockedSharePartitionMap(11)); - assertShareCacheContains(cache, List.of(key1, key3, key4)); - - TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_SESSIONS_COUNT).intValue() == 3, - "Share session count should be 3."); - TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_PARTITIONS_COUNT).intValue() == 51, - "Share partition count should be 51."); - assertEquals(1, cache.evictionsMeter().count()); - assertTrue(cache.evictionsMeter().meanRate() > 0); - - cache.touch(cache.get(key1), 400); - cache.touch(cache.get(key3), 390); - cache.touch(cache.get(key4), 400); - // No key should be evicted as all the sessions are touched to latest time. - assertNull(cache.maybeCreateSession("grp", Uuid.randomUuid(), 410, mockedSharePartitionMap(50))); + assertMetricsValues(3, 60, 0, cache); } @Test public void testResizeCachedSessions() throws InterruptedException { - ShareSessionCache cache = new ShareSessionCache(2, 100); + ShareSessionCache cache = new ShareSessionCache(2); assertEquals(0, cache.size()); assertEquals(0, cache.totalPartitions()); - ShareSessionKey key1 = cache.maybeCreateSession("grp", Uuid.randomUuid(), 0, mockedSharePartitionMap(2)); + ShareSessionKey key1 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(2)); assertNotNull(key1); assertShareCacheContains(cache, List.of(key1)); ShareSession session1 = cache.get(key1); @@ -91,28 +68,20 @@ public void testResizeCachedSessions() throws InterruptedException { assertEquals(2, cache.totalPartitions()); assertEquals(1, cache.size()); - TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_SESSIONS_COUNT).intValue() == 1, - "Share session count should be 1."); - TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_PARTITIONS_COUNT).intValue() == 2, - "Share partition count should be 2."); - assertEquals(0, cache.evictionsMeter().count()); + assertMetricsValues(1, 2, 0, cache); - ShareSessionKey key2 = cache.maybeCreateSession("grp", Uuid.randomUuid(), 0, mockedSharePartitionMap(4)); + ShareSessionKey key2 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(4)); assertNotNull(key2); assertShareCacheContains(cache, List.of(key1, key2)); ShareSession session2 = cache.get(key2); assertEquals(6, cache.totalPartitions()); assertEquals(2, cache.size()); - cache.touch(session1, 200); - cache.touch(session2, 200); + cache.updateNumPartitions(session1); + cache.updateNumPartitions(session2); - TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_SESSIONS_COUNT).intValue() == 2, - "Share session count should be 2."); - TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_PARTITIONS_COUNT).intValue() == 6, - "Share partition count should be 6."); - assertEquals(0, cache.evictionsMeter().count()); + assertMetricsValues(2, 6, 0, cache); - ShareSessionKey key3 = cache.maybeCreateSession("grp", Uuid.randomUuid(), 200, mockedSharePartitionMap(5)); + ShareSessionKey key3 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(5)); assertNull(key3); assertShareCacheContains(cache, List.of(key1, key2)); assertEquals(6, cache.totalPartitions()); @@ -122,11 +91,7 @@ public void testResizeCachedSessions() throws InterruptedException { assertEquals(1, cache.size()); assertEquals(4, cache.totalPartitions()); - TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_SESSIONS_COUNT).intValue() == 1, - "Share session count should be 1."); - TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_PARTITIONS_COUNT).intValue() == 4, - "Share partition count should be 4."); - assertEquals(0, cache.evictionsMeter().count()); + assertMetricsValues(1, 4, 0, cache); Iterator iterator = session2.partitionMap().iterator(); iterator.next(); @@ -137,15 +102,11 @@ public void testResizeCachedSessions() throws InterruptedException { assertEquals(4, session2.cachedSize()); assertEquals(4, cache.totalPartitions()); // Touch the session to update the changes in cache and session's cached size. - cache.touch(session2, session2.lastUsedMs()); + cache.updateNumPartitions(session2); assertEquals(3, session2.cachedSize()); assertEquals(3, cache.totalPartitions()); - TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_SESSIONS_COUNT).intValue() == 1, - "Share session count should be 1."); - TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_PARTITIONS_COUNT).intValue() == 3, - "Share partition count should be 3."); - assertEquals(0, cache.evictionsMeter().count()); + assertMetricsValues(1, 3, 0, cache); } private ImplicitLinkedHashCollection mockedSharePartitionMap(int size) { @@ -165,4 +126,17 @@ private void assertShareCacheContains(ShareSessionCache cache, "Missing session " + ++i + " out of " + sessionKeys.size() + " ( " + sessionKey + " )"); } } + + private void assertMetricsValues( + int shareSessionsCount, + int sharePartitionsCount, + int evictionsCount, + ShareSessionCache cache + ) throws InterruptedException { + TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_SESSIONS_COUNT).intValue() == shareSessionsCount, + "Share session count should be " + shareSessionsCount); + TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_PARTITIONS_COUNT).intValue() == sharePartitionsCount, + "Share partition count should be " + sharePartitionsCount); + assertEquals(evictionsCount, cache.evictionsMeter().count()); + } } From e78e106221e7ce72af90b32fe6d6b3b454b1667b Mon Sep 17 00:00:00 2001 From: Andrew Schofield Date: Tue, 22 Apr 2025 15:54:05 +0100 Subject: [PATCH 12/37] MINOR: Improve javadoc for share consumer (#19533) Small improvements to share consumer javadoc. Reviewers: Apoorv Mittal --- .../clients/consumer/KafkaShareConsumer.java | 36 ++++++++++--------- 1 file changed, 20 insertions(+), 16 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaShareConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaShareConsumer.java index b5a862f239d0d..923c85a398060 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaShareConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaShareConsumer.java @@ -51,8 +51,7 @@ /** * A client that consumes records from a Kafka cluster using a share group. *

- * This is an early access feature under development which is introduced by KIP-932. - * It is not suitable for production use until it is fully implemented and released. + * This is a preview feature introduced by KIP-932. It is not yet recommended for production use. * *

Cross-Version Compatibility

* This client can communicate with brokers that are a version that supports share groups. You will receive an @@ -100,7 +99,7 @@ * of the topic-partitions that match its subscriptions. Records are acquired for delivery to this consumer with a * time-limited acquisition lock. While a record is acquired, it is not available for another consumer. By default, * the lock duration is 30 seconds, but it can also be controlled using the group {@code group.share.record.lock.duration.ms} - * configuration parameter. The idea is that the lock is automatically released once the lock duration has elapsed, and + * configuration property. The idea is that the lock is automatically released once the lock duration has elapsed, and * then the record is available to be given to another consumer. The consumer which holds the lock can deal with it in * the following ways: *
    @@ -116,8 +115,8 @@ * {@code group.share.record.lock.partition.limit}. By limiting the duration of the acquisition lock and automatically * releasing the locks, the broker ensures delivery progresses even in the presence of consumer failures. *

    - * The consumer can choose to use implicit or explicit acknowledgement of the records it processes by configuring the - * consumer {@code share.acknowledgement.mode} property. + * The consumer can choose to use implicit or explicit acknowledgement of the records it processes by using the + * consumer {@code share.acknowledgement.mode} configuration property. *

    * If the application sets the property to "implicit" or does not set it at all, then the consumer is using * implicit acknowledgement. In this mode, the application acknowledges delivery by: @@ -129,7 +128,7 @@ * the delivered records as processed successfully and commits the acknowledgements to Kafka. *

  • Calling {@link #close()} which releases any acquired records without acknowledgement.
  • *
- * If the application sets the property to "explicit", then the consumer is using explicit acknowledgment. + * If the application sets the property to "explicit", then the consumer is using explicit acknowledgement. * The application must acknowledge all records returned from {@link #poll(Duration)} using * {@link #acknowledge(ConsumerRecord, AcknowledgeType)} before its next call to {@link #poll(Duration)}. * If the application calls {@link #poll(Duration)} without having acknowledged all records, an @@ -162,6 +161,7 @@ * props.setProperty("group.id", "test"); * props.setProperty("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); * props.setProperty("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); + * * KafkaShareConsumer<String, String> consumer = new KafkaShareConsumer<>(props); * consumer.subscribe(Arrays.asList("foo")); * while (true) { @@ -181,6 +181,7 @@ * props.setProperty("group.id", "test"); * props.setProperty("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); * props.setProperty("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); + * * KafkaShareConsumer<String, String> consumer = new KafkaShareConsumer<>(props); * consumer.subscribe(Arrays.asList("foo")); * while (true) { @@ -203,6 +204,7 @@ * props.setProperty("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); * props.setProperty("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); * props.setProperty("share.acknowledgement.mode", "explicit"); + * * KafkaShareConsumer<String, String> consumer = new KafkaShareConsumer<>(props); * consumer.subscribe(Arrays.asList("foo")); * while (true) { @@ -443,7 +445,7 @@ public void unsubscribe() { } /** - * Fetch data for the topics specified using {@link #subscribe(Collection)}. It is an error to not have + * Deliver records for the topics specified using {@link #subscribe(Collection)}. It is an error to not have * subscribed to any topics before polling for data. * *

@@ -452,13 +454,14 @@ public void unsubscribe() { * * @param timeout The maximum time to block (must not be greater than {@link Long#MAX_VALUE} milliseconds) * - * @return map of topic to records since the last fetch for the subscribed list of topics + * @return map of topic to records * * @throws AuthenticationException if authentication fails. See the exception for more details * @throws AuthorizationException if caller lacks Read access to any of the subscribed * topics or to the share group. See the exception for more details * @throws IllegalArgumentException if the timeout value is negative - * @throws IllegalStateException if the consumer is not subscribed to any topics + * @throws IllegalStateException if the consumer is not subscribed to any topics, or it is using + * explicit acknowledgement and has not acknowledged all records previously delivered * @throws ArithmeticException if the timeout is greater than {@link Long#MAX_VALUE} milliseconds. * @throws InvalidTopicException if the current subscription contains any invalid * topic (per {@link org.apache.kafka.common.internals.Topic#validate(String)}) @@ -475,11 +478,12 @@ public ConsumerRecords poll(Duration timeout) { * Acknowledge successful delivery of a record returned on the last {@link #poll(Duration)} call. * The acknowledgement is committed on the next {@link #commitSync()}, {@link #commitAsync()} or * {@link #poll(Duration)} call. + *

This method can only be used if the consumer is using explicit acknowledgement. * * @param record The record to acknowledge * - * @throws IllegalStateException if the record is not waiting to be acknowledged, or the consumer has already - * used implicit acknowledgement + * @throws IllegalStateException if the record is not waiting to be acknowledged, or the consumer is not using + * explicit acknowledgement */ @Override public void acknowledge(ConsumerRecord record) { @@ -489,14 +493,14 @@ public void acknowledge(ConsumerRecord record) { /** * Acknowledge delivery of a record returned on the last {@link #poll(Duration)} call indicating whether * it was processed successfully. The acknowledgement is committed on the next {@link #commitSync()}, - * {@link #commitAsync()} or {@link #poll(Duration)} call. By using this method, the consumer is using - * explicit acknowledgement. + * {@link #commitAsync()} or {@link #poll(Duration)} call. + *

This method can only be used if the consumer is using explicit acknowledgement. * * @param record The record to acknowledge * @param type The acknowledgement type which indicates whether it was processed successfully * - * @throws IllegalStateException if the record is not waiting to be acknowledged, or the consumer has already - * used implicit acknowledgement + * @throws IllegalStateException if the record is not waiting to be acknowledged, or the consumer is not using + * explicit acknowledgement */ @Override public void acknowledge(ConsumerRecord record, AcknowledgeType type) { @@ -585,7 +589,7 @@ public void setAcknowledgementCommitCallback(AcknowledgementCommitCallback callb * client to complete the request. *

* Client telemetry is controlled by the {@link ConsumerConfig#ENABLE_METRICS_PUSH_CONFIG} - * configuration option. + * configuration property. * * @param timeout The maximum time to wait for consumer client to determine its client instance ID. * The value must be non-negative. Specifying a timeout of zero means do not From 5af3547c6e74c3ed7d2e1cc46eecf7ba71c7f40d Mon Sep 17 00:00:00 2001 From: grace <147528351+ukpagrace@users.noreply.github.com> Date: Tue, 22 Apr 2025 21:32:23 +0100 Subject: [PATCH 13/37] KAFKA-18572: Update Kafka Streams metric documenation (#18673) Updated the Kafka Streams documentation to include metrics for tasks, process nodes, and threads that were missing. I was unable to find metrics such as stream-state-metrics, client-metrics, state-store-metrics, and record-cache-metrics in the codebase, so they are not included in this update. Reviewers: Bill Bejeck --- docs/ops.html | 95 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 95 insertions(+) diff --git a/docs/ops.html b/docs/ops.html index 4d6d60c6a4dbd..f5b2a76bbea2c 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -3011,6 +3011,11 @@