kamalcph commented on code in PR #16071:
URL: https://github.com/apache/kafka/pull/16071#discussion_r1617253547
##########
core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala:
##########
@@ -6597,6 +6597,79 @@ class ReplicaManagerTest {
))
}
}
+
+ @Test
+ def testRemoteReadQuotaExceeded(): Unit = {
+ when(mockRemoteLogManager.isRemoteLogFetchQuotaExceeded).thenReturn(true)
+
+ val tp0 = new TopicPartition(topic, 0)
+ val tpId0 = new TopicIdPartition(topicId, tp0)
+ val fetch: Seq[(TopicIdPartition, LogReadResult)] =
readFromLogWithOffsetOutOfRange(tp0)
+
+ assertEquals(1, fetch.size)
+ assertEquals(tpId0, fetch.head._1)
+ val fetchInfo = fetch.head._2.info
+ assertEquals(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA,
fetchInfo.fetchOffsetMetadata)
+ assertFalse(fetchInfo.records.records().iterator().hasNext)
+ assertFalse(fetchInfo.firstEntryIncomplete)
+ assertFalse(fetchInfo.abortedTransactions.isPresent)
+ assertFalse(fetchInfo.delayedRemoteStorageFetch.isPresent)
+ }
+
+ @Test
+ def testRemoteReadQuotaNotExceeded(): Unit = {
+ when(mockRemoteLogManager.isRemoteLogFetchQuotaExceeded).thenReturn(false)
+
+ val tp0 = new TopicPartition(topic, 0)
+ val tpId0 = new TopicIdPartition(topicId, tp0)
+ val fetch: Seq[(TopicIdPartition, LogReadResult)] =
readFromLogWithOffsetOutOfRange(tp0)
+
+ assertEquals(1, fetch.size)
+ assertEquals(tpId0, fetch.head._1)
+ val fetchInfo = fetch.head._2.info
+ assertEquals(1L, fetchInfo.fetchOffsetMetadata.messageOffset)
+ assertEquals(UnifiedLog.UnknownOffset,
fetchInfo.fetchOffsetMetadata.segmentBaseOffset)
+ assertEquals(-1, fetchInfo.fetchOffsetMetadata.relativePositionInSegment)
+ assertEquals(MemoryRecords.EMPTY, fetchInfo.records)
+ assertTrue(fetchInfo.delayedRemoteStorageFetch.isPresent)
+ }
+
+ private def readFromLogWithOffsetOutOfRange(tp: TopicPartition):
Seq[(TopicIdPartition, LogReadResult)] = {
+ val replicaManager = setupReplicaManagerWithMockedPurgatories(new
MockTimer(time), aliveBrokerIds = Seq(0, 1, 2), enableRemoteStorage = true,
shouldMockLog = true)
+ try {
+ val offsetCheckpoints = new
LazyOffsetCheckpoints(replicaManager.highWatermarkCheckpoints)
+ replicaManager.createPartition(tp).createLogIfNotExists(isNew = false,
isFutureReplica = false, offsetCheckpoints = offsetCheckpoints, None)
+ val partition0Replicas = Seq[Integer](0, 1).asJava
+ val topicIds = Map(tp.topic -> topicId).asJava
+ val leaderEpoch = 0
+ val leaderAndIsrRequest = new
LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0,
brokerEpoch,
+ Seq(
+ new LeaderAndIsrPartitionState()
+ .setTopicName(tp.topic)
+ .setPartitionIndex(tp.partition)
+ .setControllerEpoch(0)
+ .setLeader(leaderEpoch)
+ .setLeaderEpoch(0)
+ .setIsr(partition0Replicas)
+ .setPartitionEpoch(0)
+ .setReplicas(partition0Replicas)
+ .setIsNew(true)
+ ).asJava,
+ topicIds,
+ Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build()
+ replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest, (_, _) =>
())
+
+ val params = new FetchParams(ApiKeys.FETCH.latestVersion, -1, 1, 1000,
0, 100, FetchIsolation.HIGH_WATERMARK, None.asJava)
+ replicaManager.readFromLog(
+ params,
+ Seq(new TopicIdPartition(topicId, 0, topic) -> new
PartitionData(topicId, 1, 0, 100000, Optional.of[Integer](leaderEpoch),
Optional.of(leaderEpoch))),
Review Comment:
The test exposes an issue, the `fetchOffset` is 1 and the replica-manager
throws OffsetOutOfRangeException (since the log is empty) . Why we do have to
check the remoteLogFetch quota in this case?
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]