apoorvmittal10 commented on code in PR #18725: URL: https://github.com/apache/kafka/pull/18725#discussion_r1933609359
########## core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala: ########## @@ -5977,6 +5980,64 @@ class ReplicaManagerTest { ) } + @Test + def testDelayedShareFetchPurgatoryOperationExpiration(): Unit = { + val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_))) + val rm = new ReplicaManager( + metrics = metrics, + config = config, + time = time, + scheduler = new MockScheduler(time), + logManager = mockLogMgr, + quotaManagers = quotaManager, + metadataCache = MetadataCache.kRaftMetadataCache(config.brokerId, () => KRaftVersion.KRAFT_VERSION_0), + logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size), + alterPartitionManager = alterPartitionManager) + + val groupId = "grp" + val tp1 = new TopicIdPartition(Uuid.randomUuid, new TopicPartition("foo1", 0)) + val partitionMaxBytes = new util.LinkedHashMap[TopicIdPartition, Integer] + partitionMaxBytes.put(tp1, 1000) + + val sp1 = mock(classOf[SharePartition]) + val sharePartitions = new util.LinkedHashMap[TopicIdPartition, SharePartition] + sharePartitions.put(tp1, sp1) + + val future = new CompletableFuture[util.Map[TopicIdPartition, ShareFetchResponseData.PartitionData]] + val shareFetch = new ShareFetch( + new FetchParams(ApiKeys.SHARE_FETCH.latestVersion, FetchRequest.ORDINARY_CONSUMER_ID, -1, 500, 1, 1024 * 1024, FetchIsolation.HIGH_WATERMARK, Optional.empty, true), + groupId, + Uuid.randomUuid.toString, + future, + partitionMaxBytes, + 500, + 100, + brokerTopicStats) + + val delayedShareFetch = new DelayedShareFetch( + shareFetch, + rm, + mock(classOf[BiConsumer[SharePartitionKey, Throwable]]), + sharePartitions) + + val delayedShareFetchWatchKeys : util.List[DelayedShareFetchKey] = new util.ArrayList[DelayedShareFetchKey] + partitionMaxBytes.keySet.forEach((topicIdPartition: TopicIdPartition) => delayedShareFetchWatchKeys.add(new DelayedShareFetchGroupKey(groupId, topicIdPartition.topicId, topicIdPartition.partition))) + + // You cannot acquire records for sp1, so request will be stored in purgatory waiting for timeout. + when(sp1.maybeAcquireFetchLock).thenReturn(false) + + rm.addDelayedShareFetchRequest(delayedShareFetch = delayedShareFetch, delayedShareFetchKeys = delayedShareFetchWatchKeys) + assertEquals(1, rm.delayedShareFetchPurgatory.watched) + + // Future is not complete initially. + assertFalse(future.isDone) + // Post timeout, share fetch request will timeout and the future should complete. + waitUntilTrue(() => future.isDone, "Processing in delayed share fetch purgatory never ended.", 1000) Review Comment: Shall we have a bit buffer say, 1200 and write in comments that 1000 is default timeout but kept a buffer of 200 so the task can alwawys timeout? ########## core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala: ########## @@ -5977,6 +5980,64 @@ class ReplicaManagerTest { ) } + @Test + def testDelayedShareFetchPurgatoryOperationExpiration(): Unit = { + val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_))) + val rm = new ReplicaManager( + metrics = metrics, + config = config, + time = time, + scheduler = new MockScheduler(time), + logManager = mockLogMgr, + quotaManagers = quotaManager, + metadataCache = MetadataCache.kRaftMetadataCache(config.brokerId, () => KRaftVersion.KRAFT_VERSION_0), + logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size), + alterPartitionManager = alterPartitionManager) + + val groupId = "grp" + val tp1 = new TopicIdPartition(Uuid.randomUuid, new TopicPartition("foo1", 0)) + val partitionMaxBytes = new util.LinkedHashMap[TopicIdPartition, Integer] + partitionMaxBytes.put(tp1, 1000) + + val sp1 = mock(classOf[SharePartition]) + val sharePartitions = new util.LinkedHashMap[TopicIdPartition, SharePartition] + sharePartitions.put(tp1, sp1) + + val future = new CompletableFuture[util.Map[TopicIdPartition, ShareFetchResponseData.PartitionData]] + val shareFetch = new ShareFetch( + new FetchParams(ApiKeys.SHARE_FETCH.latestVersion, FetchRequest.ORDINARY_CONSUMER_ID, -1, 500, 1, 1024 * 1024, FetchIsolation.HIGH_WATERMARK, Optional.empty, true), + groupId, + Uuid.randomUuid.toString, + future, + partitionMaxBytes, + 500, + 100, + brokerTopicStats) + + val delayedShareFetch = new DelayedShareFetch( + shareFetch, + rm, + mock(classOf[BiConsumer[SharePartitionKey, Throwable]]), + sharePartitions) Review Comment: Hmmm, sure but how do we guarantee that the request was timeout and not copleted in normal flow. By checking `assertFalse(future.isDone)` initially? Or should we also check the method call which should only happen on timeout was triggered i.e. `forceComplete`? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org