showuon commented on code in PR #16602:
URL: https://github.com/apache/kafka/pull/16602#discussion_r1749710607


##########
core/src/main/scala/kafka/server/DelayedRemoteListOffsets.scala:
##########
@@ -0,0 +1,174 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.server
+
+import com.yammer.metrics.core.Meter
+import kafka.log.AsyncOffsetReadFutureHolder
+import kafka.utils.Implicits._
+import kafka.utils.Pool
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.errors.ApiException
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.{ListOffsetsPartitionResponse,
 ListOffsetsTopicResponse}
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.record.FileRecords.TimestampAndOffset
+import org.apache.kafka.common.requests.ListOffsetsResponse
+import org.apache.kafka.server.metrics.KafkaMetricsGroup
+
+import java.util.concurrent.TimeUnit
+import scala.collection.{Map, mutable}
+import scala.jdk.CollectionConverters._
+
+case class ListOffsetsPartitionStatus(var responseOpt: 
Option[ListOffsetsPartitionResponse] = None,
+                                      futureHolderOpt: 
Option[AsyncOffsetReadFutureHolder[Either[Exception, 
Option[TimestampAndOffset]]]] = None,
+                                      lastFetchableOffset: Option[Long] = None,
+                                      maybeOffsetsError: Option[ApiException] 
= None) {
+  @volatile var completed = false
+
+  override def toString: String = {
+    s"[responseOpt: $responseOpt, lastFetchableOffset: $lastFetchableOffset, " 
+
+      s"maybeOffsetsError: $maybeOffsetsError, completed: $completed]"
+  }
+}
+
+case class ListOffsetsMetadata(statusByPartition: mutable.Map[TopicPartition, 
ListOffsetsPartitionStatus]) {
+
+  override def toString: String = {
+    s"ListOffsetsMetadata(statusByPartition=$statusByPartition)"
+  }
+}
+
+class DelayedRemoteListOffsets(delayMs: Long,
+                               version: Int,
+                               metadata: ListOffsetsMetadata,
+                               responseCallback: 
List[ListOffsetsTopicResponse] => Unit) extends DelayedOperation(delayMs) {
+
+  // Mark the status as completed, if there is no async task to track.
+  // If there is a task to track, then build the response as REQUEST_TIMED_OUT 
by default.
+  metadata.statusByPartition.forKeyValue { (topicPartition, status) =>
+    status.completed = status.futureHolderOpt.isEmpty
+    if (status.futureHolderOpt.isDefined) {
+      status.responseOpt = Some(buildErrorResponse(Errors.REQUEST_TIMED_OUT, 
topicPartition.partition()))
+    }
+    trace(s"Initial partition status for $topicPartition is $status")
+  }
+
+  /**
+   * Call-back to execute when a delayed operation gets expired and hence 
forced to complete.
+   */
+  override def onExpiration(): Unit = {
+    metadata.statusByPartition.forKeyValue { (topicPartition, status) =>
+      if (!status.completed) {
+        debug(s"Expiring list offset request for partition $topicPartition 
with status $status")
+        status.futureHolderOpt.foreach(futureHolder => 
futureHolder.jobFuture.cancel(true))
+        DelayedRemoteListOffsetsMetrics.recordExpiration(topicPartition)
+      }
+    }
+  }
+
+  /**
+   * Process for completing an operation; This function needs to be defined
+   * in subclasses and will be called exactly once in forceComplete()
+   */
+  override def onComplete(): Unit = {
+    val responseTopics = metadata.statusByPartition.groupBy(e => 
e._1.topic()).map {
+      case (topic, status) =>
+        new 
ListOffsetsTopicResponse().setName(topic).setPartitions(status.values.flatMap(s 
=> s.responseOpt).toList.asJava)
+    }.toList
+    responseCallback(responseTopics)
+  }
+
+  /**
+   * Try to complete the delayed operation by first checking if the operation
+   * can be completed by now. If yes execute the completion logic by calling
+   * forceComplete() and return true iff forceComplete returns true; otherwise 
return false
+   *
+   * This function needs to be defined in subclasses
+   */
+  override def tryComplete(): Boolean = {
+    var completable = true
+    metadata.statusByPartition.forKeyValue { (partition, status) =>
+      if (!status.completed) {
+        status.futureHolderOpt.foreach { futureHolder =>
+          if (futureHolder.taskFuture.isDone) {
+            val response = futureHolder.taskFuture.get() match {
+              case Left(e) =>
+                buildErrorResponse(Errors.forException(e), 
partition.partition())
+
+              case Right(None) =>
+                val error = status.maybeOffsetsError
+                  .map(e => if (version >= 5) Errors.forException(e) else 
Errors.LEADER_NOT_AVAILABLE)
+                  .getOrElse(Errors.NONE)
+                buildErrorResponse(error, partition.partition())
+
+              case Right(Some(found)) =>
+                var partitionResponse = buildErrorResponse(Errors.NONE, 
partition.partition())
+                if (status.lastFetchableOffset.isDefined && found.offset >= 
status.lastFetchableOffset.get) {
+                  if (status.maybeOffsetsError.isDefined) {
+                    val error = if (version >= 5) 
Errors.forException(status.maybeOffsetsError.get) else 
Errors.LEADER_NOT_AVAILABLE
+                    partitionResponse.setErrorCode(error.code())
+                  }

Review Comment:
   It's weird there's no `else` case when lastFetchableOffset <= found offset. 
From the original logic, we  should still return error, right?



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1447,11 +1466,119 @@ class ReplicaManager(val config: KafkaConfig,
     }
   }
 
+  def fetchOffset(topics: Seq[ListOffsetsTopic],
+                  duplicatePartitions: Set[TopicPartition],
+                  isolationLevel: IsolationLevel,
+                  replicaId: Int,
+                  clientId: String,
+                  correlationId: Int,
+                  version: Short,
+                  buildErrorResponse: (Errors, ListOffsetsPartition) => 
ListOffsetsPartitionResponse,
+                  responseCallback: List[ListOffsetsTopicResponse] => Unit): 
Unit = {
+    val statusByPartition = mutable.Map[TopicPartition, 
ListOffsetsPartitionStatus]()
+    topics.foreach { topic =>
+      topic.partitions.asScala.foreach { partition =>
+        val topicPartition = new TopicPartition(topic.name, 
partition.partitionIndex)
+        if (duplicatePartitions.contains(topicPartition)) {
+          debug(s"OffsetRequest with correlation id $correlationId from client 
$clientId on partition $topicPartition " +
+            s"failed because the partition is duplicated in the request.")
+          statusByPartition += topicPartition -> 
ListOffsetsPartitionStatus(Some(buildErrorResponse(Errors.INVALID_REQUEST, 
partition)))
+        } else if (isListOffsetsTimestampUnsupported(partition.timestamp(), 
version)) {
+          statusByPartition += topicPartition -> 
ListOffsetsPartitionStatus(Some(buildErrorResponse(Errors.UNSUPPORTED_VERSION, 
partition)))
+        } else {
+          try {
+            val fetchOnlyFromLeader = replicaId != 
ListOffsetsRequest.DEBUGGING_REPLICA_ID
+            val isClientRequest = replicaId == 
ListOffsetsRequest.CONSUMER_REPLICA_ID
+            val isolationLevelOpt = if (isClientRequest)
+              Some(isolationLevel)
+            else
+              None
+
+            val resultHolder = fetchOffsetForTimestamp(topicPartition,
+              partition.timestamp,
+              isolationLevelOpt,
+              if (partition.currentLeaderEpoch == 
ListOffsetsResponse.UNKNOWN_EPOCH) Optional.empty() else 
Optional.of(partition.currentLeaderEpoch),
+              fetchOnlyFromLeader)
+
+            val status = resultHolder match {
+              case OffsetResultHolder(Some(found), _) =>
+                var partitionResponse = buildErrorResponse(Errors.NONE, 
partition)
+                if (resultHolder.lastFetchableOffset.isDefined &&
+                  found.offset >= resultHolder.lastFetchableOffset.get) {
+                  resultHolder.maybeOffsetsError.map(e => throw e)
+                } else {
+                  partitionResponse = new ListOffsetsPartitionResponse()
+                    .setPartitionIndex(partition.partitionIndex)
+                    .setErrorCode(Errors.NONE.code)
+                    .setTimestamp(found.timestamp)
+                    .setOffset(found.offset)
+                  if (found.leaderEpoch.isPresent && version >= 4)
+                    partitionResponse.setLeaderEpoch(found.leaderEpoch.get)
+                }
+                ListOffsetsPartitionStatus(Some(partitionResponse))
+              case OffsetResultHolder(None, None) =>
+                resultHolder.maybeOffsetsError.map(e => throw e)
+                
ListOffsetsPartitionStatus(Some(buildErrorResponse(Errors.NONE, partition)))
+              case OffsetResultHolder(None, Some(futureHolder)) =>
+                ListOffsetsPartitionStatus(None, Some(futureHolder), 
resultHolder.lastFetchableOffset, resultHolder.maybeOffsetsError)
+            }
+            statusByPartition += topicPartition -> status
+          } catch {
+            // NOTE: These exceptions are special cases since these error 
messages are typically transient or the client
+            // would have received a clear exception and there is no value in 
logging the entire stack trace for the same
+            case e @ (_ : UnknownTopicOrPartitionException |
+                      _ : NotLeaderOrFollowerException |
+                      _ : UnknownLeaderEpochException |
+                      _ : FencedLeaderEpochException |
+                      _ : KafkaStorageException |
+                      _ : UnsupportedForMessageFormatException) =>
+              debug(s"Offset request with correlation id $correlationId from 
client $clientId on " +
+                s"partition $topicPartition failed due to ${e.getMessage}")
+              statusByPartition += topicPartition -> 
ListOffsetsPartitionStatus(Some(buildErrorResponse(Errors.forException(e), 
partition)))
+
+            // Only V5 and newer ListOffset calls should get 
OFFSET_NOT_AVAILABLE
+            case e: OffsetNotAvailableException =>
+              if (version >= 5) {
+                statusByPartition += topicPartition -> 
ListOffsetsPartitionStatus(Some(buildErrorResponse(Errors.forException(e), 
partition)))
+              } else {
+                statusByPartition += topicPartition -> 
ListOffsetsPartitionStatus(Some(buildErrorResponse(Errors.LEADER_NOT_AVAILABLE, 
partition)))
+              }
+
+            case e: Throwable =>
+              error("Error while responding to offset request", e)
+              statusByPartition += topicPartition -> 
ListOffsetsPartitionStatus(Some(buildErrorResponse(Errors.forException(e), 
partition)))
+          }
+        }
+      }
+    }
+
+    if (delayedRemoteListOffsetsRequired(statusByPartition)) {
+      val timeout = 
config.remoteLogManagerConfig.remoteListOffsetsRequestTimeoutMs()

Review Comment:
   So we will include client timeout value in the follow-up PR?



##########
core/src/test/java/kafka/log/remote/RemoteLogOffsetReaderTest.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.log.remote;
+
+import kafka.log.AsyncOffsetReadFutureHolder;
+import kafka.utils.TestUtils;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.config.AbstractConfig;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.util.MockTime;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpointFile;
+import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache;
+import org.apache.kafka.storage.internals.log.LogDirFailureChannel;
+import org.apache.kafka.storage.log.metrics.BrokerTopicStats;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import scala.Option;
+import scala.util.Either;
+
+import static org.apache.kafka.common.record.FileRecords.TimestampAndOffset;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+class RemoteLogOffsetReaderTest {
+
+    private final MockTime time = new MockTime();
+    private final TopicPartition topicPartition = new TopicPartition("test", 
0);
+    private Path logDir;
+    private LeaderEpochFileCache cache;
+    private MockRemoteLogManager rlm;
+
+    @BeforeEach
+    void setUp() throws IOException {
+        logDir = Files.createTempDirectory("kafka-test");
+        LeaderEpochCheckpointFile checkpoint = new 
LeaderEpochCheckpointFile(TestUtils.tempFile(), new LogDirFailureChannel(1));
+        cache = new LeaderEpochFileCache(topicPartition, checkpoint, 
time.scheduler);
+        rlm = new MockRemoteLogManager(2, 10, logDir.toString());
+    }
+
+    @AfterEach
+    void tearDown() throws IOException {
+        rlm.close();
+        Utils.delete(logDir.toFile());
+    }
+
+    @Test
+    public void testReadRemoteLog() throws Exception {
+        AsyncOffsetReadFutureHolder<Either<Exception, 
Option<TimestampAndOffset>>> asyncOffsetReadFutureHolder =
+                rlm.asyncOffsetRead(topicPartition, time.milliseconds(), 0L, 
cache, Option::empty);
+        asyncOffsetReadFutureHolder.taskFuture().get(1, TimeUnit.SECONDS);
+        assertTrue(asyncOffsetReadFutureHolder.taskFuture().isDone());
+
+        Either<Exception, Option<TimestampAndOffset>> result = 
asyncOffsetReadFutureHolder.taskFuture().get();
+        assertFalse(result.isLeft());
+        assertTrue(result.isRight());
+        assertEquals(Option.apply(new TimestampAndOffset(100L, 90L, 
Optional.of(3))),
+                result.right().get());
+    }
+
+    @Test
+    public void testTaskQueueFullAndCancelTask() throws Exception {
+        rlm.pause();
+
+        List<AsyncOffsetReadFutureHolder<Either<Exception, 
Option<TimestampAndOffset>>>> holderList = new ArrayList<>();
+        // Task queue size is 10 and number of threads is 2, so it can accept 
at-most 12 items
+        for (int i = 0; i < 12; i++) {
+            holderList.add(rlm.asyncOffsetRead(topicPartition, 
time.milliseconds(), 0L, cache, Option::empty));
+        }
+        assertThrows(TimeoutException.class, () -> 
holderList.get(0).taskFuture().get(10, TimeUnit.MILLISECONDS));
+        assertEquals(0, holderList.stream().filter(h -> 
h.taskFuture().isDone()).count());
+
+        assertThrows(RejectedExecutionException.class, () ->
+                holderList.add(rlm.asyncOffsetRead(topicPartition, 
time.milliseconds(), 0L, cache, Option::empty)));
+
+        holderList.get(2).jobFuture().cancel(false);

Review Comment:
   I see. Thanks.



##########
core/src/main/scala/kafka/cluster/Partition.scala:
##########
@@ -1601,21 +1601,25 @@ class Partition(val topicPartition: TopicPartition,
         s"high watermark (${localLog.highWatermark}) is lagging behind the " +
         s"start offset from the beginning of this epoch ($epochStart)."))
 
-    def getOffsetByTimestamp: Option[TimestampAndOffset] = {
-      logManager.getLog(topicPartition).flatMap(log => 
log.fetchOffsetByTimestamp(timestamp, remoteLogManager))
+    def getOffsetByTimestamp: OffsetResultHolder = {
+      logManager.getLog(topicPartition)
+        .map(log => log.fetchOffsetByTimestamp(timestamp, remoteLogManager))
+        .getOrElse(OffsetResultHolder(timestampAndOffsetOpt = None))
     }
 
     // If we're in the lagging HW state after a leader election, throw 
OffsetNotAvailable for "latest" offset
     // or for a timestamp lookup that is beyond the last fetchable offset.
     timestamp match {
       case ListOffsetsRequest.LATEST_TIMESTAMP =>
         maybeOffsetsError.map(e => throw e)
-          .orElse(Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, 
lastFetchableOffset, Optional.of(leaderEpoch))))
+          .getOrElse(OffsetResultHolder(Some(new 
TimestampAndOffset(RecordBatch.NO_TIMESTAMP, lastFetchableOffset, 
Optional.of(leaderEpoch)))))
       case ListOffsetsRequest.EARLIEST_TIMESTAMP | 
ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP =>
         getOffsetByTimestamp
       case _ =>
-        getOffsetByTimestamp.filter(timestampAndOffset => 
timestampAndOffset.offset < lastFetchableOffset)
-          .orElse(maybeOffsetsError.map(e => throw e))
+        val offsetResultHolder = getOffsetByTimestamp
+        offsetResultHolder.maybeOffsetsError = maybeOffsetsError
+        offsetResultHolder.lastFetchableOffset = Some(lastFetchableOffset)
+        offsetResultHolder
     }

Review Comment:
   OK, I see.



-- 
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]

Reply via email to