You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "satishd (via GitHub)" <gi...@apache.org> on 2023/04/11 06:50:47 UTC

[GitHub] [kafka] satishd opened a new pull request, #13535: [DRAFT] KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

satishd opened a new pull request, #13535:
URL: https://github.com/apache/kafka/pull/13535

   This PR is not yet ready for review. It is built on top of other PR-13487
   
   This PR includes 
   - Recognize the fetch requests with out of range local log offsets
   - Add fetch implementation for the data lying in the range of [logStartOffset, localLogStartOffset]
   - Add a new purgatory for async remote read requests which are served through a specific thread pool
   
   todo: Add more tests for the newly introduced changes. There are some tests available for these scenarios in 2.8.x, refactoring with the trunk changes. 
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1172535684


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,6 +622,176 @@ public String toString() {
         }
     }
 
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadata = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadata.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        int startPos = lookupPositionForOffset(rlsMetadata.get(), offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(rlsMetadata.get(), startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            int updatedFetchSize =

Review Comment:
   There is no risk here but it is good to be consistent with the local read pattern to return empty records for that case. Updated with the latest commit.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13535:
URL: https://github.com/apache/kafka/pull/13535#issuecomment-1558447525

   @dajac They do not seem to be related to this PR. Please take a look at the [comment](https://github.com/apache/kafka/pull/13535#issuecomment-1549993361). 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd merged pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd merged PR #13535:
URL: https://github.com/apache/kafka/pull/13535


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1188470147


##########
core/src/test/java/kafka/log/remote/RemoteLogReaderTest.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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 org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.OffsetOutOfRangeException;
+import org.apache.kafka.common.record.Records;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.storage.internals.log.FetchDataInfo;
+import org.apache.kafka.storage.internals.log.LogOffsetMetadata;
+import org.apache.kafka.storage.internals.log.RemoteLogReadResult;
+import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo;
+import org.junit.jupiter.api.Test;
+import org.mockito.ArgumentCaptor;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+
+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.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class RemoteLogReaderTest {
+    RemoteLogManager mockRLM = mock(RemoteLogManager.class);
+    LogOffsetMetadata logOffsetMetadata = new LogOffsetMetadata(100);
+    Records records = mock(Records.class);
+
+
+    @Test
+    public void testRemoteLogReaderWithoutError() throws RemoteStorageException, IOException {
+        FetchDataInfo fetchDataInfo = new FetchDataInfo(logOffsetMetadata, records);
+        when(mockRLM.read(any(RemoteStorageFetchInfo.class))).thenReturn(fetchDataInfo);
+
+        Consumer<RemoteLogReadResult> callback = mock(Consumer.class);
+        RemoteStorageFetchInfo remoteStorageFetchInfo = new RemoteStorageFetchInfo(0, false, new TopicPartition("test", 0), null, null, false);
+        RemoteLogReader remoteLogReader = new RemoteLogReader(remoteStorageFetchInfo, mockRLM, callback);
+        remoteLogReader.call();
+
+        // verify the callback did get invoked with the expected remoteLogReadResult
+        ArgumentCaptor<RemoteLogReadResult> remoteLogReadResultArg = ArgumentCaptor.forClass(RemoteLogReadResult.class);
+        verify(callback, times(1)).accept(remoteLogReadResultArg.capture());
+        RemoteLogReadResult actualRemoteLogReadResult = remoteLogReadResultArg.getValue();
+        assertFalse(actualRemoteLogReadResult.error.isPresent());
+        assertTrue(actualRemoteLogReadResult.fetchDataInfo.isPresent());
+        assertEquals(fetchDataInfo, actualRemoteLogReadResult.fetchDataInfo.get());
+    }
+
+    @Test
+    public void testRemoteLogReaderWithError() throws RemoteStorageException, IOException {

Review Comment:
   We will add more tests in followup PRs.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1189455928


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,25 +622,208 @@ public String toString() {
         }
     }
 
-    long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws RemoteStorageException {
-        Optional<Long> offset = Optional.empty();
-        Optional<UnifiedLog> maybeLog = fetchLog.apply(topicIdPartition.topicPartition());
-        if (maybeLog.isPresent()) {
-            UnifiedLog log = maybeLog.get();
-            Option<LeaderEpochFileCache> maybeLeaderEpochFileCache = log.leaderEpochCache();
-            if (maybeLeaderEpochFileCache.isDefined()) {
-                LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get();
-                OptionalInt epoch = cache.latestEpoch();
-                while (!offset.isPresent() && epoch.isPresent()) {
-                    offset = remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch.getAsInt());
-                    epoch = cache.previousEpoch(epoch.getAsInt());
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadataOptional = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadataOptional.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        RemoteLogSegmentMetadata remoteLogSegmentMetadata = rlsMetadataOptional.get();
+        int startPos = lookupPositionForOffset(remoteLogSegmentMetadata, offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            // An empty record is sent instead of an incomplete batch when there is no minimum-one-message constraint
+            // and for FetchRequest version 3 and above and the first batch size is more than maximum bytes that can be sent.
+            int firstBatchSize = firstBatch.sizeInBytes();
+            if (!remoteStorageFetchInfo.minOneMessage &&
+                    !remoteStorageFetchInfo.hardMaxBytesLimit &&
+                    firstBatchSize > maxBytes) {
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY);
+            }
+
+            int updatedFetchSize =
+                    remoteStorageFetchInfo.minOneMessage && firstBatchSize > maxBytes ? firstBatchSize : maxBytes;
+
+            ByteBuffer buffer = ByteBuffer.allocate(updatedFetchSize);
+            int remainingBytes = updatedFetchSize;
+
+            firstBatch.writeTo(buffer);
+            remainingBytes -= firstBatchSize;
+
+            if (remainingBytes > 0) {
+                // read the input stream until min of (EOF stream or buffer's remaining capacity).
+                Utils.readFully(remoteSegInputStream, buffer);
+            }
+            buffer.flip();
+
+            FetchDataInfo fetchDataInfo = new FetchDataInfo(
+                    new LogOffsetMetadata(offset, remoteLogSegmentMetadata.startOffset(), startPos),
+                    MemoryRecords.readableRecords(buffer));
+            if (includeAbortedTxns) {
+                fetchDataInfo = addAbortedTransactions(firstBatch.baseOffset(), remoteLogSegmentMetadata, fetchDataInfo, logOptional.get());
+            }
+
+            return fetchDataInfo;
+        } finally {
+            Utils.closeQuietly(remoteSegInputStream, "RemoteLogSegmentInputStream");
+        }
+    }
+
+    private int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        return indexCache.lookupOffset(remoteLogSegmentMetadata, offset);
+    }
+
+    private FetchDataInfo addAbortedTransactions(long startOffset,
+                                                 RemoteLogSegmentMetadata segmentMetadata,
+                                                 FetchDataInfo fetchInfo,
+                                                 UnifiedLog log) throws RemoteStorageException {
+        int fetchSize = fetchInfo.records.sizeInBytes();
+        OffsetPosition startOffsetPosition = new OffsetPosition(fetchInfo.fetchOffsetMetadata.messageOffset,
+                fetchInfo.fetchOffsetMetadata.relativePositionInSegment);
+
+        OffsetIndex offsetIndex = indexCache.getIndexEntry(segmentMetadata).offsetIndex();
+        long upperBoundOffset = offsetIndex.fetchUpperBoundOffset(startOffsetPosition, fetchSize)
+                .map(x -> x.offset).orElse(segmentMetadata.endOffset() + 1);
+
+        final Set<FetchResponseData.AbortedTransaction> abortedTransactions = new HashSet<>();
+
+        Consumer<List<AbortedTxn>> accumulator =
+                abortedTxns -> abortedTransactions.addAll(abortedTxns.stream()
+                        .map(AbortedTxn::asAbortedTransaction).collect(Collectors.toList()));
+
+        collectAbortedTransactions(startOffset, upperBoundOffset, segmentMetadata, accumulator, log);
+
+        return new FetchDataInfo(fetchInfo.fetchOffsetMetadata,
+                fetchInfo.records,
+                fetchInfo.firstEntryIncomplete,
+                Optional.of(abortedTransactions.isEmpty() ? Collections.emptyList() : new ArrayList<>(abortedTransactions)));
+    }
+
+    private void collectAbortedTransactions(long startOffset,
+                                            long upperBoundOffset,
+                                            RemoteLogSegmentMetadata segmentMetadata,
+                                            Consumer<List<AbortedTxn>> accumulator,
+                                            UnifiedLog log) throws RemoteStorageException {
+        Iterator<LogSegment> localLogSegments = JavaConverters.asJavaIterator(log.logSegments().iterator());
+
+        boolean searchInLocalLog = false;
+        Optional<RemoteLogSegmentMetadata> nextSegmentMetadataOpt = Optional.of(segmentMetadata);
+        Optional<TransactionIndex> txnIndexOpt = nextSegmentMetadataOpt.map(metadata -> indexCache.getIndexEntry(metadata).txnIndex());
+
+        while (txnIndexOpt.isPresent()) {
+            TxnIndexSearchResult searchResult = txnIndexOpt.get().collectAbortedTxns(startOffset, upperBoundOffset);
+            accumulator.accept(searchResult.abortedTransactions);
+            if (!searchResult.isComplete) {
+                if (!searchInLocalLog) {
+                    nextSegmentMetadataOpt = findNextSegmentMetadata(nextSegmentMetadataOpt.get(), log);
+
+                    txnIndexOpt = nextSegmentMetadataOpt.map(x -> indexCache.getIndexEntry(x).txnIndex());
+                    if (!txnIndexOpt.isPresent()) {
+                        searchInLocalLog = true;
+                    }
+                }
+
+                if (searchInLocalLog) {
+                    txnIndexOpt = (localLogSegments.hasNext()) ? Optional.of(localLogSegments.next().txnIndex()) : Optional.empty();
                 }
+            } else {
+                return;
+            }
+        }
+    }
+
+    private Optional<RemoteLogSegmentMetadata> findNextSegmentMetadata(RemoteLogSegmentMetadata segmentMetadata, UnifiedLog log) throws RemoteStorageException {
+        Option<LeaderEpochFileCache> leaderEpochFileCacheOption = log.leaderEpochCache();
+        if (leaderEpochFileCacheOption.isEmpty()) {
+            return Optional.empty();
+        }
+
+        TopicPartition topicPartition = segmentMetadata.topicIdPartition().topicPartition();
+        long nextSegmentBaseOffset = segmentMetadata.endOffset() + 1;
+        OptionalInt epoch = OptionalInt.of(segmentMetadata.segmentLeaderEpochs().lastEntry().getKey());

Review Comment:
   Good point. We can use `leaderepochCache.epochForOffset()` which we introduced for all other usages and missed replacing it here.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1181518738


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,25 +622,208 @@ public String toString() {
         }
     }
 
-    long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws RemoteStorageException {
-        Optional<Long> offset = Optional.empty();
-        Optional<UnifiedLog> maybeLog = fetchLog.apply(topicIdPartition.topicPartition());
-        if (maybeLog.isPresent()) {
-            UnifiedLog log = maybeLog.get();
-            Option<LeaderEpochFileCache> maybeLeaderEpochFileCache = log.leaderEpochCache();
-            if (maybeLeaderEpochFileCache.isDefined()) {
-                LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get();
-                OptionalInt epoch = cache.latestEpoch();
-                while (!offset.isPresent() && epoch.isPresent()) {
-                    offset = remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch.getAsInt());
-                    epoch = cache.previousEpoch(epoch.getAsInt());
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadataOptional = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadataOptional.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        RemoteLogSegmentMetadata remoteLogSegmentMetadata = rlsMetadataOptional.get();
+        int startPos = lookupPositionForOffset(remoteLogSegmentMetadata, offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);

Review Comment:
   Sure, we can definitely look into that.  We did not see much of GC issues remote read throughputs ~750 MBps on a broker but there are plans to improve by exploring buffer pool mechanisms(variations of pool used in producers).



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeqo commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "jeqo (via GitHub)" <gi...@apache.org>.
jeqo commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1183258439


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1118,9 +1122,13 @@ class ReplicaManager(val config: KafkaConfig,
     responseCallback: Seq[(TopicIdPartition, FetchPartitionData)] => Unit
   ): Unit = {
     // check if this fetch request can be satisfied right away
-    val logReadResults = readFromLocalLog(params, fetchInfos, quota, readFromPurgatory = false)
+    val logReadResults = readFromLog(params, fetchInfos, quota, readFromPurgatory = false)
     var bytesReadable: Long = 0
     var errorReadingData = false
+
+    // The 1st topic-partition that has to be read from remote storage
+    var remoteFetchInfo: Optional[RemoteStorageFetchInfo] = Optional.empty()

Review Comment:
   Sure, thanks.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1192195822


##########
core/src/test/scala/integration/kafka/server/DelayedRemoteFetchTest.scala:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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 kafka.cluster.Partition
+import org.apache.kafka.common.errors.NotLeaderOrFollowerException
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.record.MemoryRecords
+import org.apache.kafka.common.requests.FetchRequest
+import org.apache.kafka.common.{TopicIdPartition, Uuid}
+import org.apache.kafka.storage.internals.log._
+import org.junit.jupiter.api.Assertions._
+import org.junit.jupiter.api.Test
+import org.mockito.Mockito.{mock, when}
+
+import java.util.Optional
+import java.util.concurrent.CompletableFuture
+
+import scala.collection._

Review Comment:
   We have a few more tests that need to be pulled out from the existing 2.8.x repo with other changes and refactor from EasyMock to Mockito, we can convert this test into Java in that PR.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] junrao commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "junrao (via GitHub)" <gi...@apache.org>.
junrao commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1195723455


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1175,49 +1216,101 @@ class ReplicaManager(val config: KafkaConfig,
           fetchPartitionStatus += (topicIdPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
         })
       }
-      val delayedFetch = new DelayedFetch(
-        params = params,
-        fetchPartitionStatus = fetchPartitionStatus,
-        replicaManager = this,
-        quota = quota,
-        responseCallback = responseCallback
-      )
-
-      // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
-      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp) }
-
-      // try to complete the request immediately, otherwise put it into the purgatory;
-      // this is because while the delayed fetch operation is being created, new requests
-      // may arrive and hence make this operation completable.
-      delayedFetchPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys)
+
+      if (remoteFetchInfo.isPresent) {
+        val key = new TopicPartitionOperationKey(remoteFetchInfo.get.topicPartition.topic(), remoteFetchInfo.get.topicPartition.partition())

Review Comment:
   `fetchMessages` is getting a bit large now. Would it be better to put the logic in the `if` part to a separate method?



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1288,17 +1373,46 @@ class ReplicaManager(val config: KafkaConfig,
                  _: FencedLeaderEpochException |
                  _: ReplicaNotAvailableException |
                  _: KafkaStorageException |
-                 _: OffsetOutOfRangeException |
                  _: InconsistentTopicIdException) =>
-          LogReadResult(info = new FetchDataInfo(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, MemoryRecords.EMPTY),
-            divergingEpoch = None,
-            highWatermark = UnifiedLog.UnknownOffset,
-            leaderLogStartOffset = UnifiedLog.UnknownOffset,
-            leaderLogEndOffset = UnifiedLog.UnknownOffset,
-            followerLogStartOffset = UnifiedLog.UnknownOffset,
-            fetchTimeMs = -1L,
-            lastStableOffset = None,
-            exception = Some(e))
+          createLogReadResult(e)
+        case e: OffsetOutOfRangeException =>
+          // In case of offset out of range errors, check for remote log manager for non-compacted topics

Review Comment:
   The logic for handling OffsetOutOfRangeException is getting a bit large now. Would it be better to put that into a separate method?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] junrao commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "junrao (via GitHub)" <gi...@apache.org>.
junrao commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1184289360


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1160,48 +1172,100 @@ class ReplicaManager(val config: KafkaConfig,
           fetchPartitionStatus += (topicIdPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
         })
       }
-      val delayedFetch = new DelayedFetch(
-        params = params,
-        fetchPartitionStatus = fetchPartitionStatus,
-        replicaManager = this,
-        quota = quota,
-        responseCallback = responseCallback
-      )
-
-      // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
-      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp) }
-
-      // try to complete the request immediately, otherwise put it into the purgatory;
-      // this is because while the delayed fetch operation is being created, new requests
-      // may arrive and hence make this operation completable.
-      delayedFetchPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys)
+
+      if (remoteFetchInfo.isPresent) {
+        val key = new TopicPartitionOperationKey(remoteFetchInfo.get.topicPartition.topic(), remoteFetchInfo.get.topicPartition.partition())
+        val remoteFetchResult = new CompletableFuture[RemoteLogReadResult]
+        var remoteFetchTask: Future[Void] = null
+        try {
+          remoteFetchTask = remoteLogManager.get.asyncRead(remoteFetchInfo.get, (result: RemoteLogReadResult) => {
+            remoteFetchResult.complete(result)
+            delayedRemoteFetchPurgatory.checkAndComplete(key)
+          })
+        } catch {
+          // if the task queue of remote storage reader thread pool is full, return what we currently have
+          // (the data read from local log segment for the other topic-partitions) and an error for the topic-partition that
+          // we couldn't read from remote storage
+          case e: RejectedExecutionException =>
+            val fetchPartitionData = logReadResults.map { case (tp, result) =>
+              val r = {
+                if (tp.topicPartition().equals(remoteFetchInfo.get.topicPartition))
+                  createLogReadResult(e)
+                else
+                  result
+              }
+
+              tp -> r.toFetchPartitionData(false)
+            }
+            responseCallback(fetchPartitionData)
+            return
+        }
+
+        // If there is remote data, we will read remote data, instead of waiting for new data.
+        val remoteFetch = new DelayedRemoteFetch(remoteFetchTask, remoteFetchResult, remoteFetchInfo.get,
+          fetchPartitionStatus, params, logReadResults, this, responseCallback)
+
+        delayedRemoteFetchPurgatory.tryCompleteElseWatch(remoteFetch, Seq(key))
+      } else {
+        // If there is not enough data to respond and there is no remote data, we will let the fetch request
+        // to wait for new data.

Review Comment:
   let the fetch request to wait => let the fetch request wait 



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1320,6 +1405,33 @@ class ReplicaManager(val config: KafkaConfig,
     result
   }
 
+  private def createLogReadResult(highWatermark: Long,

Review Comment:
   Should this be in `object ReplicaManager`?



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1320,6 +1405,33 @@ class ReplicaManager(val config: KafkaConfig,
     result
   }
 
+  private def createLogReadResult(highWatermark: Long,
+                          leaderLogStartOffset: Long,
+                          leaderLogEndOffset: Long,
+                          e: Throwable) = {
+    LogReadResult(info = new FetchDataInfo(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, MemoryRecords.EMPTY),
+      divergingEpoch = None,
+      highWatermark,
+      leaderLogStartOffset,
+      leaderLogEndOffset,
+      followerLogStartOffset = -1L,
+      fetchTimeMs = -1L,
+      lastStableOffset = None,
+      exception = Some(e))
+  }
+
+  def createLogReadResult(e: Throwable): LogReadResult = {

Review Comment:
   Should this be in `object ReplicaManager`?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,25 +622,208 @@ public String toString() {
         }
     }
 
-    long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws RemoteStorageException {
-        Optional<Long> offset = Optional.empty();
-        Optional<UnifiedLog> maybeLog = fetchLog.apply(topicIdPartition.topicPartition());
-        if (maybeLog.isPresent()) {
-            UnifiedLog log = maybeLog.get();
-            Option<LeaderEpochFileCache> maybeLeaderEpochFileCache = log.leaderEpochCache();
-            if (maybeLeaderEpochFileCache.isDefined()) {
-                LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get();
-                OptionalInt epoch = cache.latestEpoch();
-                while (!offset.isPresent() && epoch.isPresent()) {
-                    offset = remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch.getAsInt());
-                    epoch = cache.previousEpoch(epoch.getAsInt());
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadataOptional = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadataOptional.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        RemoteLogSegmentMetadata remoteLogSegmentMetadata = rlsMetadataOptional.get();
+        int startPos = lookupPositionForOffset(remoteLogSegmentMetadata, offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            // An empty record is sent instead of an incomplete batch when there is no minimum-one-message constraint
+            // and for FetchRequest version 3 and above and the first batch size is more than maximum bytes that can be sent.

Review Comment:
   This line doesn't read well.  Also, should it be put above the following line?
   
   ```
               int updatedFetchSize =
                       remoteStorageFetchInfo.minOneMessage && firstBatchSize > maxBytes ? firstBatchSize : maxBytes;
   
   ```



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1273,17 +1329,46 @@ class ReplicaManager(val config: KafkaConfig,
                  _: FencedLeaderEpochException |
                  _: ReplicaNotAvailableException |
                  _: KafkaStorageException |
-                 _: OffsetOutOfRangeException |
                  _: InconsistentTopicIdException) =>
-          LogReadResult(info = new FetchDataInfo(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, MemoryRecords.EMPTY),
-            divergingEpoch = None,
-            highWatermark = UnifiedLog.UnknownOffset,
-            leaderLogStartOffset = UnifiedLog.UnknownOffset,
-            leaderLogEndOffset = UnifiedLog.UnknownOffset,
-            followerLogStartOffset = UnifiedLog.UnknownOffset,
-            fetchTimeMs = -1L,
-            lastStableOffset = None,
-            exception = Some(e))
+          createLogReadResult(e)
+        case e: OffsetOutOfRangeException =>
+          // In case of offset out of range errors, check for remote log manager for non-compacted topics
+          // to fetch from remote storage. `log` instance should not be null here as that would have caught earlier with

Review Comment:
   that would have caught => that would have been caught



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1273,17 +1329,46 @@ class ReplicaManager(val config: KafkaConfig,
                  _: FencedLeaderEpochException |
                  _: ReplicaNotAvailableException |
                  _: KafkaStorageException |
-                 _: OffsetOutOfRangeException |
                  _: InconsistentTopicIdException) =>
-          LogReadResult(info = new FetchDataInfo(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, MemoryRecords.EMPTY),
-            divergingEpoch = None,
-            highWatermark = UnifiedLog.UnknownOffset,
-            leaderLogStartOffset = UnifiedLog.UnknownOffset,
-            leaderLogEndOffset = UnifiedLog.UnknownOffset,
-            followerLogStartOffset = UnifiedLog.UnknownOffset,
-            fetchTimeMs = -1L,
-            lastStableOffset = None,
-            exception = Some(e))
+          createLogReadResult(e)
+        case e: OffsetOutOfRangeException =>
+          // In case of offset out of range errors, check for remote log manager for non-compacted topics
+          // to fetch from remote storage. `log` instance should not be null here as that would have caught earlier with
+          // NotLeaderForPartitionException or ReplicaNotAvailableException.
+          // If it is from a follower then send the offset metadata only as the data is already available in remote
+          // storage.
+          if (remoteLogManager.isDefined && log != null && log.remoteLogEnabled() &&
+            // Check that the fetch offset is with in the offset range with in the remote storage layer.
+            log.logStartOffset <= offset && offset < log.localLogStartOffset()) {
+            // For follower fetch requests, throw an error saying that this offset is moved to tiered storage.
+            val highWatermark = log.highWatermark
+            val leaderLogStartOffset = log.logStartOffset
+            val leaderLogEndOffset = log.logEndOffset
+            if (params.isFromFollower) {
+              createLogReadResult(highWatermark, leaderLogStartOffset, leaderLogEndOffset,
+                new OffsetMovedToTieredStorageException("Given offset" + offset + " is moved to tiered storage"))
+            } else {
+              // Create a dummy FetchDataInfo with the remote storage fetch information.
+              // For the first topic-partition that needs remote data, we will use this information to read the data in another thread.
+              // For the following topic-partitions, we return an empty record set
+              val fetchDataInfo =
+                new FetchDataInfo(new LogOffsetMetadata(fetchInfo.fetchOffset), MemoryRecords.EMPTY, false, Optional.empty(),
+                  Optional.of(new RemoteStorageFetchInfo(adjustedMaxBytes, minOneMessage, tp.topicPartition(),
+                    fetchInfo, params.isolation, params.hardMaxBytesLimit())))
+
+              LogReadResult(checkFetchDataInfo(partition, fetchDataInfo),
+                divergingEpoch = None,
+                highWatermark,
+                leaderLogStartOffset,
+                leaderLogEndOffset,
+                followerLogStartOffset,
+                time.milliseconds,

Review Comment:
   Could we reuse fetchTimeMs instead of calling time.milliseconds again?



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1083,48 +1095,100 @@ class ReplicaManager(val config: KafkaConfig,
           fetchPartitionStatus += (topicIdPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
         })
       }
-      val delayedFetch = new DelayedFetch(
-        params = params,
-        fetchPartitionStatus = fetchPartitionStatus,
-        replicaManager = this,
-        quota = quota,
-        responseCallback = responseCallback
-      )
-
-      // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
-      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp) }
-
-      // try to complete the request immediately, otherwise put it into the purgatory;
-      // this is because while the delayed fetch operation is being created, new requests
-      // may arrive and hence make this operation completable.
-      delayedFetchPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys)
+
+      if (remoteFetchInfo.isPresent) {

Review Comment:
   In the following code, we should go into that branch only if `remoteFetchInfo` is empty, right? Otherwise, if we could get into a situation that a remote partition is never served because the fetch request  is always satisfied with new local data on other partitions.
   ```
       if (params.maxWaitMs <= 0 || fetchInfos.isEmpty || bytesReadable >= params.minBytes || errorReadingData ||
         hasDivergingEpoch || hasPreferredReadReplica) {
   ```



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,25 +622,208 @@ public String toString() {
         }
     }
 
-    long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws RemoteStorageException {
-        Optional<Long> offset = Optional.empty();
-        Optional<UnifiedLog> maybeLog = fetchLog.apply(topicIdPartition.topicPartition());
-        if (maybeLog.isPresent()) {
-            UnifiedLog log = maybeLog.get();
-            Option<LeaderEpochFileCache> maybeLeaderEpochFileCache = log.leaderEpochCache();
-            if (maybeLeaderEpochFileCache.isDefined()) {
-                LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get();
-                OptionalInt epoch = cache.latestEpoch();
-                while (!offset.isPresent() && epoch.isPresent()) {
-                    offset = remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch.getAsInt());
-                    epoch = cache.previousEpoch(epoch.getAsInt());
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadataOptional = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadataOptional.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        RemoteLogSegmentMetadata remoteLogSegmentMetadata = rlsMetadataOptional.get();
+        int startPos = lookupPositionForOffset(remoteLogSegmentMetadata, offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            // An empty record is sent instead of an incomplete batch when there is no minimum-one-message constraint
+            // and for FetchRequest version 3 and above and the first batch size is more than maximum bytes that can be sent.
+            int firstBatchSize = firstBatch.sizeInBytes();
+            if (!remoteStorageFetchInfo.minOneMessage &&
+                    !remoteStorageFetchInfo.hardMaxBytesLimit &&
+                    firstBatchSize > maxBytes) {
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY);
+            }
+
+            int updatedFetchSize =
+                    remoteStorageFetchInfo.minOneMessage && firstBatchSize > maxBytes ? firstBatchSize : maxBytes;
+
+            ByteBuffer buffer = ByteBuffer.allocate(updatedFetchSize);
+            int remainingBytes = updatedFetchSize;
+
+            firstBatch.writeTo(buffer);
+            remainingBytes -= firstBatchSize;
+
+            if (remainingBytes > 0) {
+                // read the input stream until min of (EOF stream or buffer's remaining capacity).
+                Utils.readFully(remoteSegInputStream, buffer);
+            }
+            buffer.flip();
+
+            FetchDataInfo fetchDataInfo = new FetchDataInfo(
+                    new LogOffsetMetadata(offset, remoteLogSegmentMetadata.startOffset(), startPos),
+                    MemoryRecords.readableRecords(buffer));
+            if (includeAbortedTxns) {
+                fetchDataInfo = addAbortedTransactions(firstBatch.baseOffset(), remoteLogSegmentMetadata, fetchDataInfo, logOptional.get());
+            }
+
+            return fetchDataInfo;
+        } finally {
+            Utils.closeQuietly(remoteSegInputStream, "RemoteLogSegmentInputStream");
+        }
+    }
+
+    private int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        return indexCache.lookupOffset(remoteLogSegmentMetadata, offset);
+    }
+
+    private FetchDataInfo addAbortedTransactions(long startOffset,
+                                                 RemoteLogSegmentMetadata segmentMetadata,
+                                                 FetchDataInfo fetchInfo,
+                                                 UnifiedLog log) throws RemoteStorageException {
+        int fetchSize = fetchInfo.records.sizeInBytes();
+        OffsetPosition startOffsetPosition = new OffsetPosition(fetchInfo.fetchOffsetMetadata.messageOffset,
+                fetchInfo.fetchOffsetMetadata.relativePositionInSegment);
+
+        OffsetIndex offsetIndex = indexCache.getIndexEntry(segmentMetadata).offsetIndex();
+        long upperBoundOffset = offsetIndex.fetchUpperBoundOffset(startOffsetPosition, fetchSize)
+                .map(x -> x.offset).orElse(segmentMetadata.endOffset() + 1);
+
+        final Set<FetchResponseData.AbortedTransaction> abortedTransactions = new HashSet<>();
+
+        Consumer<List<AbortedTxn>> accumulator =
+                abortedTxns -> abortedTransactions.addAll(abortedTxns.stream()
+                        .map(AbortedTxn::asAbortedTransaction).collect(Collectors.toList()));
+
+        collectAbortedTransactions(startOffset, upperBoundOffset, segmentMetadata, accumulator, log);
+
+        return new FetchDataInfo(fetchInfo.fetchOffsetMetadata,
+                fetchInfo.records,
+                fetchInfo.firstEntryIncomplete,
+                Optional.of(abortedTransactions.isEmpty() ? Collections.emptyList() : new ArrayList<>(abortedTransactions)));
+    }
+
+    private void collectAbortedTransactions(long startOffset,
+                                            long upperBoundOffset,
+                                            RemoteLogSegmentMetadata segmentMetadata,
+                                            Consumer<List<AbortedTxn>> accumulator,
+                                            UnifiedLog log) throws RemoteStorageException {
+        Iterator<LogSegment> localLogSegments = JavaConverters.asJavaIterator(log.logSegments().iterator());
+
+        boolean searchInLocalLog = false;
+        Optional<RemoteLogSegmentMetadata> nextSegmentMetadataOpt = Optional.of(segmentMetadata);
+        Optional<TransactionIndex> txnIndexOpt = nextSegmentMetadataOpt.map(metadata -> indexCache.getIndexEntry(metadata).txnIndex());
+
+        while (txnIndexOpt.isPresent()) {
+            TxnIndexSearchResult searchResult = txnIndexOpt.get().collectAbortedTxns(startOffset, upperBoundOffset);
+            accumulator.accept(searchResult.abortedTransactions);
+            if (!searchResult.isComplete) {
+                if (!searchInLocalLog) {
+                    nextSegmentMetadataOpt = findNextSegmentMetadata(nextSegmentMetadataOpt.get(), log);
+
+                    txnIndexOpt = nextSegmentMetadataOpt.map(x -> indexCache.getIndexEntry(x).txnIndex());
+                    if (!txnIndexOpt.isPresent()) {
+                        searchInLocalLog = true;
+                    }
+                }
+
+                if (searchInLocalLog) {
+                    txnIndexOpt = (localLogSegments.hasNext()) ? Optional.of(localLogSegments.next().txnIndex()) : Optional.empty();
                 }
+            } else {
+                return;
+            }
+        }
+    }
+
+    private Optional<RemoteLogSegmentMetadata> findNextSegmentMetadata(RemoteLogSegmentMetadata segmentMetadata, UnifiedLog log) throws RemoteStorageException {
+        Option<LeaderEpochFileCache> leaderEpochFileCacheOption = log.leaderEpochCache();
+        if (leaderEpochFileCacheOption.isEmpty()) {
+            return Optional.empty();
+        }
+
+        TopicPartition topicPartition = segmentMetadata.topicIdPartition().topicPartition();
+        long nextSegmentBaseOffset = segmentMetadata.endOffset() + 1;
+        OptionalInt epoch = OptionalInt.of(segmentMetadata.segmentLeaderEpochs().lastEntry().getKey());

Review Comment:
   You mentioned the following in the other comment.
   `We should find the respective epoch for the target offset and use that to find the remote log segment metadata.`
   
   However, here, epoch seems to be for the offset before nextSegmentBaseOffset, not at nextSegmentBaseOffset?



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1273,17 +1329,46 @@ class ReplicaManager(val config: KafkaConfig,
                  _: FencedLeaderEpochException |
                  _: ReplicaNotAvailableException |
                  _: KafkaStorageException |
-                 _: OffsetOutOfRangeException |
                  _: InconsistentTopicIdException) =>
-          LogReadResult(info = new FetchDataInfo(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, MemoryRecords.EMPTY),
-            divergingEpoch = None,
-            highWatermark = UnifiedLog.UnknownOffset,
-            leaderLogStartOffset = UnifiedLog.UnknownOffset,
-            leaderLogEndOffset = UnifiedLog.UnknownOffset,
-            followerLogStartOffset = UnifiedLog.UnknownOffset,
-            fetchTimeMs = -1L,
-            lastStableOffset = None,
-            exception = Some(e))
+          createLogReadResult(e)
+        case e: OffsetOutOfRangeException =>
+          // In case of offset out of range errors, check for remote log manager for non-compacted topics
+          // to fetch from remote storage. `log` instance should not be null here as that would have caught earlier with
+          // NotLeaderForPartitionException or ReplicaNotAvailableException.
+          // If it is from a follower then send the offset metadata only as the data is already available in remote
+          // storage.
+          if (remoteLogManager.isDefined && log != null && log.remoteLogEnabled() &&
+            // Check that the fetch offset is with in the offset range with in the remote storage layer.

Review Comment:
   two occurrences
   with in => within



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,6 +622,176 @@ public String toString() {
         }
     }
 
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadata = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadata.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        int startPos = lookupPositionForOffset(rlsMetadata.get(), offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(rlsMetadata.get(), startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            int updatedFetchSize =
+                    remoteStorageFetchInfo.minOneMessage && firstBatch.sizeInBytes() > maxBytes
+                            ? firstBatch.sizeInBytes() : maxBytes;
+
+            ByteBuffer buffer = ByteBuffer.allocate(updatedFetchSize);
+            int remainingBytes = updatedFetchSize;
+
+            firstBatch.writeTo(buffer);
+            remainingBytes -= firstBatch.sizeInBytes();
+
+            if (remainingBytes > 0) {
+                // input stream is read till (startPos - 1) while getting the batch of records earlier.
+                // read the input stream until min of (EOF stream or buffer's remaining capacity).
+                Utils.readFully(remoteSegInputStream, buffer);
+            }
+            buffer.flip();
+
+            FetchDataInfo fetchDataInfo = new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.readableRecords(buffer));
+            if (includeAbortedTxns) {
+                fetchDataInfo = addAbortedTransactions(firstBatch.baseOffset(), rlsMetadata.get(), fetchDataInfo);
+            }
+
+            return fetchDataInfo;
+        } finally {
+            Utils.closeQuietly(remoteSegInputStream, "RemoteLogSegmentInputStream");
+        }
+    }
+
+    private int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        return indexCache.lookupOffset(remoteLogSegmentMetadata, offset);
+    }
+
+    private FetchDataInfo addAbortedTransactions(long startOffset,
+                                         RemoteLogSegmentMetadata segmentMetadata,
+                                         FetchDataInfo fetchInfo) throws RemoteStorageException {
+        int fetchSize = fetchInfo.records.sizeInBytes();
+        OffsetPosition startOffsetPosition = new OffsetPosition(fetchInfo.fetchOffsetMetadata.messageOffset,
+                fetchInfo.fetchOffsetMetadata.relativePositionInSegment);
+
+        OffsetIndex offsetIndex = indexCache.getIndexEntry(segmentMetadata).offsetIndex();
+        long upperBoundOffset = offsetIndex.fetchUpperBoundOffset(startOffsetPosition, fetchSize)
+                .map(x -> x.offset).orElse(segmentMetadata.endOffset() + 1);
+
+        final List<FetchResponseData.AbortedTransaction> abortedTransactions = new ArrayList<>();
+
+        Consumer<List<AbortedTxn>> accumulator =
+                abortedTxns -> abortedTransactions.addAll(abortedTxns.stream()
+                        .map(AbortedTxn::asAbortedTransaction).collect(Collectors.toList()));
+
+        collectAbortedTransactions(startOffset, upperBoundOffset, segmentMetadata, accumulator);
+
+        return new FetchDataInfo(fetchInfo.fetchOffsetMetadata,
+                fetchInfo.records,
+                fetchInfo.firstEntryIncomplete,
+                Optional.of(abortedTransactions));
+    }
+
+    private void collectAbortedTransactions(long startOffset,
+                                    long upperBoundOffset,
+                                    RemoteLogSegmentMetadata segmentMetadata,
+                                    Consumer<List<AbortedTxn>> accumulator) throws RemoteStorageException {
+        TopicPartition topicPartition = segmentMetadata.topicIdPartition().topicPartition();
+        Iterator<LogSegment> localLogSegments = fetchLog.apply(topicPartition)
+                .map(log -> JavaConverters.asJavaCollection(log.logSegments()))
+                .map(Collection::iterator)
+                .orElse(Collections.emptyIterator());
+
+        boolean searchInLocalLog = false;
+        Optional<RemoteLogSegmentMetadata> nextSegmentMetadataOpt = Optional.of(segmentMetadata);
+        Optional<TransactionIndex> txnIndexOpt = nextSegmentMetadataOpt.map(metadata -> indexCache.getIndexEntry(metadata).txnIndex());
+
+        while (txnIndexOpt.isPresent()) {
+            TxnIndexSearchResult searchResult = txnIndexOpt.get().collectAbortedTxns(startOffset, upperBoundOffset);
+            accumulator.accept(searchResult.abortedTransactions);
+            if (!searchResult.isComplete) {
+                if (!searchInLocalLog) {
+                    nextSegmentMetadataOpt = findNextSegmentMetadata(nextSegmentMetadataOpt.get());
+
+                    txnIndexOpt = nextSegmentMetadataOpt.map(x -> indexCache.getIndexEntry(x).txnIndex());
+                    if (!txnIndexOpt.isPresent()) {
+                        searchInLocalLog = true;
+                    }
+                }
+
+                if (searchInLocalLog) {
+                    txnIndexOpt = (localLogSegments.hasNext()) ? Optional.of(localLogSegments.next().txnIndex()) : Optional.empty();

Review Comment:
   Where is the logic to remove the duplicates?
   
   Also, remote segments could overlap too. Is there  any logic to remove the duplicates from remote segments too?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1203666314


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1335,6 +1433,50 @@ class ReplicaManager(val config: KafkaConfig,
     result
   }
 
+  private def handleOffsetOutOfRangeError(tp: TopicIdPartition, params: FetchParams, fetchInfo: PartitionData,
+                                          adjustedMaxBytes: Int, minOneMessage:
+                                          Boolean, log: UnifiedLog, fetchTimeMs: Long,
+                                          exception: OffsetOutOfRangeException): LogReadResult = {

Review Comment:
   We usually don't format method like this. Could we put one argument per line?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13535:
URL: https://github.com/apache/kafka/pull/13535#issuecomment-1551099626

   Thanks @junrao for the updated review. Addressed your latest minor review comments.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] showuon commented on pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on PR #13535:
URL: https://github.com/apache/kafka/pull/13535#issuecomment-1527204055

   @Hangleton @junrao @jeqo , any other comments to this PR? We hope we can merge it in the early stage of a release, so that we can have enough time to test the stability and have more improvement. Thanks.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] Hangleton commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1166846747


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,6 +622,176 @@ public String toString() {
         }
     }
 
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadata = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadata.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        int startPos = lookupPositionForOffset(rlsMetadata.get(), offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(rlsMetadata.get(), startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            int updatedFetchSize =

Review Comment:
   Is there a risk here to overflow the `buffer` if it is allocated up to `maxBytes` and the first batch is greater than that?
   
   This could happen if the partition read here is not the first one from the Fetch request in which case `minOneMessage` is false [*]. In the local counterpart, an empty set of records is returned ([source](https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/ReplicaManager.scala#L1247-L1250), where `firstEntryIncomplete` describes the case where (using the equivalent notation used here) `minOneMessage` was false and `firstBatch.sizeInBytes() > maxBytes`).
   
   [*] Even if only one remote partition is served per Fetch request, there can still be another partition from that request served from its local replica log hence a possibility for not at least one message requested to be read here.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,6 +622,176 @@ public String toString() {
         }
     }
 
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadata = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadata.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        int startPos = lookupPositionForOffset(rlsMetadata.get(), offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(rlsMetadata.get(), startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            int updatedFetchSize =
+                    remoteStorageFetchInfo.minOneMessage && firstBatch.sizeInBytes() > maxBytes
+                            ? firstBatch.sizeInBytes() : maxBytes;
+
+            ByteBuffer buffer = ByteBuffer.allocate(updatedFetchSize);
+            int remainingBytes = updatedFetchSize;
+
+            firstBatch.writeTo(buffer);
+            remainingBytes -= firstBatch.sizeInBytes();
+
+            if (remainingBytes > 0) {
+                // input stream is read till (startPos - 1) while getting the batch of records earlier.
+                // read the input stream until min of (EOF stream or buffer's remaining capacity).
+                Utils.readFully(remoteSegInputStream, buffer);
+            }
+            buffer.flip();
+
+            FetchDataInfo fetchDataInfo = new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.readableRecords(buffer));
+            if (includeAbortedTxns) {
+                fetchDataInfo = addAbortedTransactions(firstBatch.baseOffset(), rlsMetadata.get(), fetchDataInfo);
+            }
+
+            return fetchDataInfo;
+        } finally {
+            Utils.closeQuietly(remoteSegInputStream, "RemoteLogSegmentInputStream");
+        }
+    }
+
+    private int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        return indexCache.lookupOffset(remoteLogSegmentMetadata, offset);
+    }
+
+    private FetchDataInfo addAbortedTransactions(long startOffset,
+                                         RemoteLogSegmentMetadata segmentMetadata,
+                                         FetchDataInfo fetchInfo) throws RemoteStorageException {
+        int fetchSize = fetchInfo.records.sizeInBytes();
+        OffsetPosition startOffsetPosition = new OffsetPosition(fetchInfo.fetchOffsetMetadata.messageOffset,
+                fetchInfo.fetchOffsetMetadata.relativePositionInSegment);
+
+        OffsetIndex offsetIndex = indexCache.getIndexEntry(segmentMetadata).offsetIndex();
+        long upperBoundOffset = offsetIndex.fetchUpperBoundOffset(startOffsetPosition, fetchSize)
+                .map(x -> x.offset).orElse(segmentMetadata.endOffset() + 1);
+
+        final List<FetchResponseData.AbortedTransaction> abortedTransactions = new ArrayList<>();
+
+        Consumer<List<AbortedTxn>> accumulator =
+                abortedTxns -> abortedTransactions.addAll(abortedTxns.stream()
+                        .map(AbortedTxn::asAbortedTransaction).collect(Collectors.toList()));
+
+        collectAbortedTransactions(startOffset, upperBoundOffset, segmentMetadata, accumulator);
+
+        return new FetchDataInfo(fetchInfo.fetchOffsetMetadata,
+                fetchInfo.records,
+                fetchInfo.firstEntryIncomplete,
+                Optional.of(abortedTransactions));
+    }
+
+    private void collectAbortedTransactions(long startOffset,
+                                    long upperBoundOffset,
+                                    RemoteLogSegmentMetadata segmentMetadata,
+                                    Consumer<List<AbortedTxn>> accumulator) throws RemoteStorageException {
+        TopicPartition topicPartition = segmentMetadata.topicIdPartition().topicPartition();
+        Iterator<LogSegment> localLogSegments = fetchLog.apply(topicPartition)
+                .map(log -> JavaConverters.asJavaCollection(log.logSegments()))
+                .map(Collection::iterator)
+                .orElse(Collections.emptyIterator());
+
+        boolean searchInLocalLog = false;
+        Optional<RemoteLogSegmentMetadata> nextSegmentMetadataOpt = Optional.of(segmentMetadata);
+        Optional<TransactionIndex> txnIndexOpt = nextSegmentMetadataOpt.map(metadata -> indexCache.getIndexEntry(metadata).txnIndex());
+
+        while (txnIndexOpt.isPresent()) {
+            TxnIndexSearchResult searchResult = txnIndexOpt.get().collectAbortedTxns(startOffset, upperBoundOffset);
+            accumulator.accept(searchResult.abortedTransactions);
+            if (!searchResult.isComplete) {
+                if (!searchInLocalLog) {
+                    nextSegmentMetadataOpt = findNextSegmentMetadata(nextSegmentMetadataOpt.get());
+
+                    txnIndexOpt = nextSegmentMetadataOpt.map(x -> indexCache.getIndexEntry(x).txnIndex());

Review Comment:
   There may be an availability risk here. The remote index cache allows up to 1,024 entries by default and it is possible to keep a transaction index entry permanently rotated in and out of the cache so that retries of Fetch requests for a partition at a given (constant) offset permanently trigger a download of the index. This, plus the fact there is a global lock for read and write access on the remote index cache, can lead to lock contention on the remote log reader thread pool. In the worst case, it is possible for a partition to be prevented from making progress on the fetch path.



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1118,9 +1122,13 @@ class ReplicaManager(val config: KafkaConfig,
     responseCallback: Seq[(TopicIdPartition, FetchPartitionData)] => Unit
   ): Unit = {
     // check if this fetch request can be satisfied right away
-    val logReadResults = readFromLocalLog(params, fetchInfos, quota, readFromPurgatory = false)
+    val logReadResults = readFromLog(params, fetchInfos, quota, readFromPurgatory = false)
     var bytesReadable: Long = 0
     var errorReadingData = false
+
+    // The 1st topic-partition that has to be read from remote storage
+    var remoteFetchInfo: Optional[RemoteStorageFetchInfo] = Optional.empty()

Review Comment:
   Agreed - there are consumption patterns which diverge from the local case with this approach (that is, uneven progress across the partitions consumed from a topic [with said partitions of the same nature w.r.t. record batch size and overall size]).
   
   It may be preferable not to diverge from the local approach and read from all the remote partitions found in the `fetchInfos`. Then, a different read pattern which provides greater performance for a specific operational environment and workload could be enforced via a configuration property. 



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1273,17 +1328,45 @@ class ReplicaManager(val config: KafkaConfig,
                  _: FencedLeaderEpochException |
                  _: ReplicaNotAvailableException |
                  _: KafkaStorageException |
-                 _: OffsetOutOfRangeException |
                  _: InconsistentTopicIdException) =>
-          LogReadResult(info = new FetchDataInfo(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, MemoryRecords.EMPTY),
-            divergingEpoch = None,
-            highWatermark = UnifiedLog.UnknownOffset,
-            leaderLogStartOffset = UnifiedLog.UnknownOffset,
-            leaderLogEndOffset = UnifiedLog.UnknownOffset,
-            followerLogStartOffset = UnifiedLog.UnknownOffset,
-            fetchTimeMs = -1L,
-            lastStableOffset = None,
-            exception = Some(e))
+          createLogReadResult(e)
+        case e: OffsetOutOfRangeException =>

Review Comment:
   I assume eventually we won't use exception-based branch control and not rely on this exception to be re-directed to the remote read code path?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] showuon commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1165153511


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1061,6 +1072,7 @@ class ReplicaManager(val config: KafkaConfig,
       logReadResultMap.put(topicIdPartition, logReadResult)
     }
 
+    val delayedRemoteFetchPurgatory: DelayedOperationPurgatory[DelayedRemoteFetch] = null;

Review Comment:
   Why should we make `delayedRemoteFetchPurgatory` null here? We've init `delayedRemoteFetchPurgatory` in L217, and want to use it below L1131 immediately. So this line should be removed.



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1061,6 +1072,7 @@ class ReplicaManager(val config: KafkaConfig,
       logReadResultMap.put(topicIdPartition, logReadResult)
     }
 
+    val delayedRemoteFetchPurgatory: DelayedOperationPurgatory[DelayedRemoteFetch] = null;

Review Comment:
   OK, I've run a test, and confirmed this line will make it throw NPE while fetching data from remote segment.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1171250580


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1118,9 +1122,13 @@ class ReplicaManager(val config: KafkaConfig,
     responseCallback: Seq[(TopicIdPartition, FetchPartitionData)] => Unit
   ): Unit = {
     // check if this fetch request can be satisfied right away
-    val logReadResults = readFromLocalLog(params, fetchInfos, quota, readFromPurgatory = false)
+    val logReadResults = readFromLog(params, fetchInfos, quota, readFromPurgatory = false)
     var bytesReadable: Long = 0
     var errorReadingData = false
+
+    // The 1st topic-partition that has to be read from remote storage
+    var remoteFetchInfo: Optional[RemoteStorageFetchInfo] = Optional.empty()

Review Comment:
   As I already called out in this PR description, that it is followed up with a PR. We will describe the config on different options with respective scenarios. The default value will be to fetch from multiple partitions as it does with local log segments. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: [DRAFT] KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1164325078


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1083,48 +1095,100 @@ class ReplicaManager(val config: KafkaConfig,
           fetchPartitionStatus += (topicIdPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
         })
       }
-      val delayedFetch = new DelayedFetch(
-        params = params,
-        fetchPartitionStatus = fetchPartitionStatus,
-        replicaManager = this,
-        quota = quota,
-        responseCallback = responseCallback
-      )
-
-      // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
-      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp) }
-
-      // try to complete the request immediately, otherwise put it into the purgatory;
-      // this is because while the delayed fetch operation is being created, new requests
-      // may arrive and hence make this operation completable.
-      delayedFetchPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys)
+
+      if (remoteFetchInfo.isPresent) {
+        val key = new TopicPartitionOperationKey(remoteFetchInfo.get.topicPartition.topic(), remoteFetchInfo.get.topicPartition.partition())
+        val remoteFetchResult = new CompletableFuture[RemoteLogReadResult]
+        var remoteFetchTask: Future[Void] = null
+        try {
+          remoteFetchTask = remoteLogManager.get.asyncRead(remoteFetchInfo.get, (result: RemoteLogReadResult) => {
+            remoteFetchResult.complete(result)
+            delayedRemoteFetchPurgatory.checkAndComplete(key)
+          })
+        } catch {
+          // if the task queue of remote storage reader thread pool is full, return what we currently have
+          // (the data read from local log segment for the other topic-partitions) and an error for the topic-partition that
+          // we couldn't read from remote storage
+          case e: RejectedExecutionException =>
+            val fetchPartitionData = logReadResults.map { case (tp, result) =>
+              val r = {
+                if (tp.topicPartition().equals(remoteFetchInfo.get.topicPartition))
+                  createLogReadResult(e)
+                else
+                  result
+              }
+
+              tp -> r.toFetchPartitionData(false)
+            }
+            responseCallback(fetchPartitionData)
+            return
+        }
+
+        // If there is remote data, we will read remote data, instead of waiting for new data.

Review Comment:
   No, this comment is for the next 2 lines, mainly for the below line. 
   
   ```
   delayedRemoteFetchPurgatory.tryCompleteElseWatch(remoteFetch, Seq(key))
   ```



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1168261768


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1243,6 +1327,33 @@ class ReplicaManager(val config: KafkaConfig,
     result
   }
 
+  def createLogReadResult(highWatermark: Long,

Review Comment:
   `createLogReadResult(e: Throwable)` can not be private as it is used in `DelayedRemoteFetch`. But this method can be used. It is going to be used in test classes that we are going to add in this PR or followup PR. 



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,6 +622,176 @@ public String toString() {
         }
     }
 
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();

Review Comment:
   afaik, `lastFetchedEpoch` is the epoch of the last fetched record. That can be different from the fetch offset’s epoch. We should find the respective epoch for the target offset and use that to find the remote log segment metadata.
   



##########
core/src/main/java/kafka/log/remote/RemoteLogReader.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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;

Review Comment:
   `RemoteLogReader` can not be moved to storage module as it currently depends on `RemoteLogManager`. I will move along with `RemoteLogManager` later. 
   `RemoteLogReadResult` and `RemoteStorageThreadPool` are moved to storage module. 



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1083,48 +1095,100 @@ class ReplicaManager(val config: KafkaConfig,
           fetchPartitionStatus += (topicIdPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
         })
       }
-      val delayedFetch = new DelayedFetch(
-        params = params,
-        fetchPartitionStatus = fetchPartitionStatus,
-        replicaManager = this,
-        quota = quota,
-        responseCallback = responseCallback
-      )
-
-      // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
-      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp) }
-
-      // try to complete the request immediately, otherwise put it into the purgatory;
-      // this is because while the delayed fetch operation is being created, new requests
-      // may arrive and hence make this operation completable.
-      delayedFetchPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys)
+
+      if (remoteFetchInfo.isPresent) {

Review Comment:
   I did not understand the comment here.  



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,6 +622,176 @@ public String toString() {
         }
     }
 
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadata = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadata.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        int startPos = lookupPositionForOffset(rlsMetadata.get(), offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(rlsMetadata.get(), startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            int updatedFetchSize =
+                    remoteStorageFetchInfo.minOneMessage && firstBatch.sizeInBytes() > maxBytes
+                            ? firstBatch.sizeInBytes() : maxBytes;
+
+            ByteBuffer buffer = ByteBuffer.allocate(updatedFetchSize);
+            int remainingBytes = updatedFetchSize;
+
+            firstBatch.writeTo(buffer);
+            remainingBytes -= firstBatch.sizeInBytes();
+
+            if (remainingBytes > 0) {
+                // input stream is read till (startPos - 1) while getting the batch of records earlier.
+                // read the input stream until min of (EOF stream or buffer's remaining capacity).
+                Utils.readFully(remoteSegInputStream, buffer);
+            }
+            buffer.flip();
+
+            FetchDataInfo fetchDataInfo = new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.readableRecords(buffer));
+            if (includeAbortedTxns) {
+                fetchDataInfo = addAbortedTransactions(firstBatch.baseOffset(), rlsMetadata.get(), fetchDataInfo);
+            }
+
+            return fetchDataInfo;
+        } finally {
+            Utils.closeQuietly(remoteSegInputStream, "RemoteLogSegmentInputStream");
+        }
+    }
+
+    private int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        return indexCache.lookupOffset(remoteLogSegmentMetadata, offset);
+    }
+
+    private FetchDataInfo addAbortedTransactions(long startOffset,
+                                         RemoteLogSegmentMetadata segmentMetadata,
+                                         FetchDataInfo fetchInfo) throws RemoteStorageException {
+        int fetchSize = fetchInfo.records.sizeInBytes();
+        OffsetPosition startOffsetPosition = new OffsetPosition(fetchInfo.fetchOffsetMetadata.messageOffset,
+                fetchInfo.fetchOffsetMetadata.relativePositionInSegment);
+
+        OffsetIndex offsetIndex = indexCache.getIndexEntry(segmentMetadata).offsetIndex();
+        long upperBoundOffset = offsetIndex.fetchUpperBoundOffset(startOffsetPosition, fetchSize)
+                .map(x -> x.offset).orElse(segmentMetadata.endOffset() + 1);
+
+        final List<FetchResponseData.AbortedTransaction> abortedTransactions = new ArrayList<>();
+
+        Consumer<List<AbortedTxn>> accumulator =
+                abortedTxns -> abortedTransactions.addAll(abortedTxns.stream()
+                        .map(AbortedTxn::asAbortedTransaction).collect(Collectors.toList()));
+
+        collectAbortedTransactions(startOffset, upperBoundOffset, segmentMetadata, accumulator);
+
+        return new FetchDataInfo(fetchInfo.fetchOffsetMetadata,
+                fetchInfo.records,
+                fetchInfo.firstEntryIncomplete,
+                Optional.of(abortedTransactions));
+    }
+
+    private void collectAbortedTransactions(long startOffset,
+                                    long upperBoundOffset,
+                                    RemoteLogSegmentMetadata segmentMetadata,
+                                    Consumer<List<AbortedTxn>> accumulator) throws RemoteStorageException {
+        TopicPartition topicPartition = segmentMetadata.topicIdPartition().topicPartition();
+        Iterator<LogSegment> localLogSegments = fetchLog.apply(topicPartition)
+                .map(log -> JavaConverters.asJavaCollection(log.logSegments()))
+                .map(Collection::iterator)
+                .orElse(Collections.emptyIterator());
+
+        boolean searchInLocalLog = false;
+        Optional<RemoteLogSegmentMetadata> nextSegmentMetadataOpt = Optional.of(segmentMetadata);
+        Optional<TransactionIndex> txnIndexOpt = nextSegmentMetadataOpt.map(metadata -> indexCache.getIndexEntry(metadata).txnIndex());
+
+        while (txnIndexOpt.isPresent()) {
+            TxnIndexSearchResult searchResult = txnIndexOpt.get().collectAbortedTxns(startOffset, upperBoundOffset);
+            accumulator.accept(searchResult.abortedTransactions);
+            if (!searchResult.isComplete) {
+                if (!searchInLocalLog) {
+                    nextSegmentMetadataOpt = findNextSegmentMetadata(nextSegmentMetadataOpt.get());
+
+                    txnIndexOpt = nextSegmentMetadataOpt.map(x -> indexCache.getIndexEntry(x).txnIndex());
+                    if (!txnIndexOpt.isPresent()) {
+                        searchInLocalLog = true;
+                    }
+                }
+
+                if (searchInLocalLog) {
+                    txnIndexOpt = (localLogSegments.hasNext()) ? Optional.of(localLogSegments.next().txnIndex()) : Optional.empty();

Review Comment:
   Right, it can have duplicates. But consumer already handles the duplicate aborted transactions. Updated the code to remove duplicates incase any consumer implementation can not handle duplicate aborted transactions. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] junrao commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "junrao (via GitHub)" <gi...@apache.org>.
junrao commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1196786120


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1175,49 +1260,78 @@ class ReplicaManager(val config: KafkaConfig,
           fetchPartitionStatus += (topicIdPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
         })
       }
-      val delayedFetch = new DelayedFetch(
-        params = params,
-        fetchPartitionStatus = fetchPartitionStatus,
-        replicaManager = this,
-        quota = quota,
-        responseCallback = responseCallback
-      )
-
-      // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
-      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp) }
-
-      // try to complete the request immediately, otherwise put it into the purgatory;
-      // this is because while the delayed fetch operation is being created, new requests
-      // may arrive and hence make this operation completable.
-      delayedFetchPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys)
+
+      if (remoteFetchInfo.isPresent) {
+        val maybeLogReadResultWithError = processRemoteFetch(remoteFetchInfo.get(), params, responseCallback, logReadResults, fetchPartitionStatus)
+        if(maybeLogReadResultWithError.isDefined) {

Review Comment:
   space after `if `



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1175,49 +1260,78 @@ class ReplicaManager(val config: KafkaConfig,
           fetchPartitionStatus += (topicIdPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
         })
       }
-      val delayedFetch = new DelayedFetch(
-        params = params,
-        fetchPartitionStatus = fetchPartitionStatus,
-        replicaManager = this,
-        quota = quota,
-        responseCallback = responseCallback
-      )
-
-      // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
-      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp) }
-
-      // try to complete the request immediately, otherwise put it into the purgatory;
-      // this is because while the delayed fetch operation is being created, new requests
-      // may arrive and hence make this operation completable.
-      delayedFetchPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys)
+
+      if (remoteFetchInfo.isPresent) {
+        val maybeLogReadResultWithError = processRemoteFetch(remoteFetchInfo.get(), params, responseCallback, logReadResults, fetchPartitionStatus)
+        if(maybeLogReadResultWithError.isDefined) {
+          // If there is an error in scheduling the remote fetch task, return what we currently have
+          // (the data read from local log segment for the other topic-partitions) and an error for the topic-partition
+          // that we couldn't read from remote storage
+          val partitionToFetchPartitionData = buildPartitionToFetchPartitionData(logReadResults, remoteFetchInfo.get().topicPartition, maybeLogReadResultWithError.get)
+          responseCallback(partitionToFetchPartitionData)
+          return

Review Comment:
   It seems that `return` is unnecessary here?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on PR #13535:
URL: https://github.com/apache/kafka/pull/13535#issuecomment-1559262765

   @satishd Weird... It fails all the time on my laptop.
   
   ```
   Gradle Test Run :core:test > Gradle Test Executor 9 > ListOffsetsRequestTest > testResponseIncludesLeaderEpoch() FAILED
       org.opentest4j.AssertionFailedError: expected: <(10,1,0)> but was: <(-1,-1,78)>
           at app//org.junit.jupiter.api.AssertionFailureBuilder.build(AssertionFailureBuilder.java:151)
           at app//org.junit.jupiter.api.AssertionFailureBuilder.buildAndThrow(AssertionFailureBuilder.java:132)
           at app//org.junit.jupiter.api.AssertEquals.failNotEqual(AssertEquals.java:197)
           at app//org.junit.jupiter.api.AssertEquals.assertEquals(AssertEquals.java:182)
           at app//org.junit.jupiter.api.AssertEquals.assertEquals(AssertEquals.java:177)
           at app//org.junit.jupiter.api.Assertions.assertEquals(Assertions.java:1142)
           at app//kafka.server.ListOffsetsRequestTest.testResponseIncludesLeaderEpoch(ListOffsetsRequestTest.scala:210)
   ```
   
   ```
   % git rev-parse --verify HEAD
   15f8705246e094f7825b76a38d9f12f95d626ee5
   ```


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] divijvaidya commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "divijvaidya (via GitHub)" <gi...@apache.org>.
divijvaidya commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1182330923


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,25 +622,208 @@ public String toString() {
         }
     }
 
-    long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws RemoteStorageException {
-        Optional<Long> offset = Optional.empty();
-        Optional<UnifiedLog> maybeLog = fetchLog.apply(topicIdPartition.topicPartition());
-        if (maybeLog.isPresent()) {
-            UnifiedLog log = maybeLog.get();
-            Option<LeaderEpochFileCache> maybeLeaderEpochFileCache = log.leaderEpochCache();
-            if (maybeLeaderEpochFileCache.isDefined()) {
-                LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get();
-                OptionalInt epoch = cache.latestEpoch();
-                while (!offset.isPresent() && epoch.isPresent()) {
-                    offset = remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch.getAsInt());
-                    epoch = cache.previousEpoch(epoch.getAsInt());
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadataOptional = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadataOptional.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        RemoteLogSegmentMetadata remoteLogSegmentMetadata = rlsMetadataOptional.get();
+        int startPos = lookupPositionForOffset(remoteLogSegmentMetadata, offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);

Review Comment:
   Thanks, I added a JIRA https://issues.apache.org/jira/browse/KAFKA-14954 to ensure that we are tracking it and don't forget about it.
   
   I agree that this is not a major blocker from GC perspective as JVMs are getting more intelligent about allocation/deallocation of byte[] but this adds up in situations when we have large number of allocation. Another PR where we removed data copy and allocation resulted in 2% CPU improvement (measured from flamegraph) with JDK17. JDK 8 will probably have a larger impact.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1187999169


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1083,48 +1095,100 @@ class ReplicaManager(val config: KafkaConfig,
           fetchPartitionStatus += (topicIdPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
         })
       }
-      val delayedFetch = new DelayedFetch(
-        params = params,
-        fetchPartitionStatus = fetchPartitionStatus,
-        replicaManager = this,
-        quota = quota,
-        responseCallback = responseCallback
-      )
-
-      // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
-      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp) }
-
-      // try to complete the request immediately, otherwise put it into the purgatory;
-      // this is because while the delayed fetch operation is being created, new requests
-      // may arrive and hence make this operation completable.
-      delayedFetchPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys)
+
+      if (remoteFetchInfo.isPresent) {

Review Comment:
   Sure, updated it with the latest commit.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] Hangleton commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1188387671


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,25 +623,210 @@ public String toString() {
         }
     }
 
-    long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws RemoteStorageException {
-        Optional<Long> offset = Optional.empty();
-        Optional<UnifiedLog> maybeLog = fetchLog.apply(topicIdPartition.topicPartition());
-        if (maybeLog.isPresent()) {
-            UnifiedLog log = maybeLog.get();
-            Option<LeaderEpochFileCache> maybeLeaderEpochFileCache = log.leaderEpochCache();
-            if (maybeLeaderEpochFileCache.isDefined()) {
-                LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get();
-                OptionalInt epoch = cache.latestEpoch();
-                while (!offset.isPresent() && epoch.isPresent()) {
-                    offset = remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch.getAsInt());
-                    epoch = cache.previousEpoch(epoch.getAsInt());
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadataOptional = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadataOptional.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        RemoteLogSegmentMetadata remoteLogSegmentMetadata = rlsMetadataOptional.get();
+        int startPos = lookupPositionForOffset(remoteLogSegmentMetadata, offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            int firstBatchSize = firstBatch.sizeInBytes();
+            // An empty record is sent instead of an incomplete batch when
+            //  - there is no minimum-one-message constraint and
+            //  - the first batch size is more than maximum bytes that can be sent.
+            //  - for FetchRequest version 3 or above and
+            if (!remoteStorageFetchInfo.minOneMessage &&
+                    !remoteStorageFetchInfo.hardMaxBytesLimit &&
+                    firstBatchSize > maxBytes) {
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY);
+            }
+
+            int updatedFetchSize =
+                    remoteStorageFetchInfo.minOneMessage && firstBatchSize > maxBytes ? firstBatchSize : maxBytes;
+
+            ByteBuffer buffer = ByteBuffer.allocate(updatedFetchSize);
+            int remainingBytes = updatedFetchSize;
+
+            firstBatch.writeTo(buffer);
+            remainingBytes -= firstBatchSize;
+
+            if (remainingBytes > 0) {
+                // read the input stream until min of (EOF stream or buffer's remaining capacity).
+                Utils.readFully(remoteSegInputStream, buffer);
+            }
+            buffer.flip();
+
+            FetchDataInfo fetchDataInfo = new FetchDataInfo(
+                    new LogOffsetMetadata(offset, remoteLogSegmentMetadata.startOffset(), startPos),
+                    MemoryRecords.readableRecords(buffer));
+            if (includeAbortedTxns) {
+                fetchDataInfo = addAbortedTransactions(firstBatch.baseOffset(), remoteLogSegmentMetadata, fetchDataInfo, logOptional.get());
+            }
+
+            return fetchDataInfo;
+        } finally {
+            Utils.closeQuietly(remoteSegInputStream, "RemoteLogSegmentInputStream");
+        }
+    }
+
+    private int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        return indexCache.lookupOffset(remoteLogSegmentMetadata, offset);
+    }
+
+    private FetchDataInfo addAbortedTransactions(long startOffset,

Review Comment:
   Do we have a unit test for this method and/or the associated functionality taken in isolation?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1181518576


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1160,48 +1171,100 @@ class ReplicaManager(val config: KafkaConfig,
           fetchPartitionStatus += (topicIdPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
         })
       }
-      val delayedFetch = new DelayedFetch(
-        params = params,
-        fetchPartitionStatus = fetchPartitionStatus,
-        replicaManager = this,
-        quota = quota,
-        responseCallback = responseCallback
-      )
-
-      // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
-      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp) }
-
-      // try to complete the request immediately, otherwise put it into the purgatory;
-      // this is because while the delayed fetch operation is being created, new requests
-      // may arrive and hence make this operation completable.
-      delayedFetchPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys)
+
+      if (remoteFetchInfo.isPresent) {
+        val key = new TopicPartitionOperationKey(remoteFetchInfo.get.topicPartition.topic(), remoteFetchInfo.get.topicPartition.partition())
+        val remoteFetchResult = new CompletableFuture[RemoteLogReadResult]
+        var remoteFetchTask: Future[Void] = null
+        try {
+          remoteFetchTask = remoteLogManager.get.asyncRead(remoteFetchInfo.get, (result: RemoteLogReadResult) => {
+            remoteFetchResult.complete(result)
+            delayedRemoteFetchPurgatory.checkAndComplete(key)
+          })
+        } catch {
+          // if the task queue of remote storage reader thread pool is full, return what we currently have
+          // (the data read from local log segment for the other topic-partitions) and an error for the topic-partition that
+          // we couldn't read from remote storage
+          case e: RejectedExecutionException =>
+            val fetchPartitionData = logReadResults.map { case (tp, result) =>
+              val r = {
+                if (tp.topicPartition().equals(remoteFetchInfo.get.topicPartition))
+                  createLogReadResult(e)

Review Comment:
   This error is [propagated](https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/ReplicaManager.scala#L86) as unexpected error (UnknownServerException) to the consumer client and it is already handled. 



##########
core/src/main/scala/kafka/server/DelayedRemoteFetch.scala:
##########
@@ -0,0 +1,112 @@
+/**
+ * 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 org.apache.kafka.common.TopicIdPartition
+import org.apache.kafka.common.errors._
+import org.apache.kafka.storage.internals.log.{FetchParams, FetchPartitionData, LogOffsetMetadata, RemoteLogReadResult, RemoteStorageFetchInfo}
+
+import java.util.concurrent.{CompletableFuture, Future}
+import java.util.{Optional, OptionalInt, OptionalLong}
+import scala.collection._
+
+/**
+ * A remote fetch operation that can be created by the replica manager and watched
+ * in the remote fetch operation purgatory
+ */
+class DelayedRemoteFetch(remoteFetchTask: Future[Void],
+                         remoteFetchResult: CompletableFuture[RemoteLogReadResult],
+                         remoteFetchInfo: RemoteStorageFetchInfo,
+                         fetchPartitionStatus: Seq[(TopicIdPartition, FetchPartitionStatus)],
+                         fetchParams: FetchParams,
+                         localReadResults: Seq[(TopicIdPartition, LogReadResult)],
+                         replicaManager: ReplicaManager,
+                         responseCallback: Seq[(TopicIdPartition, FetchPartitionData)] => Unit)
+  extends DelayedOperation(fetchParams.maxWaitMs) {
+
+  /**
+   * The operation can be completed if:
+   *
+   * Case a: This broker is no longer the leader of the partition it tries to fetch
+   * Case b: This broker does not know the partition it tries to fetch
+   * Case c: The remote storage read request completed (succeeded or failed)
+   * Case d: The partition is in an offline log directory on this broker
+   *
+   * Upon completion, should return whatever data is available for each valid partition
+   */
+  override def tryComplete(): Boolean = {
+    fetchPartitionStatus.foreach {
+      case (topicPartition, fetchStatus) =>
+        val fetchOffset = fetchStatus.startOffsetMetadata
+        try {
+          if (fetchOffset != LogOffsetMetadata.UNKNOWN_OFFSET_METADATA) {
+            replicaManager.getPartitionOrException(topicPartition.topicPartition())
+          }
+        } catch {
+          case _: KafkaStorageException => // Case d
+            debug(s"Partition $topicPartition is in an offline log directory, satisfy $fetchParams immediately")
+            return forceComplete()
+          case _: UnknownTopicOrPartitionException => // Case b
+            debug(s"Broker no longer knows of partition $topicPartition, satisfy $fetchParams immediately")
+            return forceComplete()
+          case _: NotLeaderOrFollowerException =>  // Case a
+            debug("Broker is no longer the leader or follower of %s, satisfy %s immediately".format(topicPartition, fetchParams))
+            return forceComplete()
+        }
+    }
+    if (remoteFetchResult.isDone) // Case c
+      forceComplete()
+    else
+      false
+  }
+
+  override def onExpiration():Unit = {

Review Comment:
   Yes, we have in the internal branch. There are other metrics too related to tiered storage.  They will be added in a followup PR. 
   
   <img width="1415" alt="image" src="https://user-images.githubusercontent.com/2577761/235444150-338d3472-c511-4252-bc56-1afa31c1ad2b.png">
   
   



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13535:
URL: https://github.com/apache/kafka/pull/13535#issuecomment-1529600204

   >LGTM! @satishd , will there be other tests added as in the PR description said, or there will be follow-up PRs to add them?
   
   @showuon Those will be added as followups. 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13535:
URL: https://github.com/apache/kafka/pull/13535#issuecomment-1538311434

   Thanks @junrao for the updated review comments. Addressed the comments with inline replies or with the latest commits. 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1187999169


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1083,48 +1095,100 @@ class ReplicaManager(val config: KafkaConfig,
           fetchPartitionStatus += (topicIdPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
         })
       }
-      val delayedFetch = new DelayedFetch(
-        params = params,
-        fetchPartitionStatus = fetchPartitionStatus,
-        replicaManager = this,
-        quota = quota,
-        responseCallback = responseCallback
-      )
-
-      // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
-      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp) }
-
-      // try to complete the request immediately, otherwise put it into the purgatory;
-      // this is because while the delayed fetch operation is being created, new requests
-      // may arrive and hence make this operation completable.
-      delayedFetchPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys)
+
+      if (remoteFetchInfo.isPresent) {

Review Comment:
   Sure, that check was missed while pulling the changes. Good catch. Updated it with the latest commit.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13535:
URL: https://github.com/apache/kafka/pull/13535#issuecomment-1559216261

   @dajac It is passed locally on my laptop.
   
   ```
   Gradle Test Run :core:test > Gradle Test Executor 65 > ListOffsetsRequestTest > testResponseDefaultOffsetAndLeaderEpochForAllVersions() PASSED
   
   Gradle Test Run :core:test > Gradle Test Executor 65 > ListOffsetsRequestTest > testListOffsetsMaxTimeStampOldestVersion() PASSED
   
   Gradle Test Run :core:test > Gradle Test Executor 65 > ListOffsetsRequestTest > testListOffsetsErrorCodes() PASSED
   
   Gradle Test Run :core:test > Gradle Test Executor 65 > ListOffsetsRequestTest > testCurrentEpochValidation() PASSED
   
   Gradle Test Run :core:test > Gradle Test Executor 65 > ListOffsetsRequestTest > testResponseIncludesLeaderEpoch() PASSED
   
   BUILD SUCCESSFUL in 1m 8s
   55 actionable tasks: 6 executed, 49 up-to-date
   ➜  kafka git:(apache-trunk) date
   Tue May 23 18:00:01 IST 2023
   ➜  kafka git:(apache-trunk) git rev-parse --verify HEAD
   15f8705246e094f7825b76a38d9f12f95d626ee5
   ➜  kafka git:(apache-trunk)
   ```
   
   ```
   > Task :core:test
   
   Gradle Test Run :core:test > Gradle Test Executor 71 > ListOffsetsRequestWithRemoteStoreTest > testResponseDefaultOffsetAndLeaderEpochForAllVersions() PASSED
   
   Gradle Test Run :core:test > Gradle Test Executor 71 > ListOffsetsRequestWithRemoteStoreTest > testListOffsetsMaxTimeStampOldestVersion() PASSED
   
   Gradle Test Run :core:test > Gradle Test Executor 71 > ListOffsetsRequestWithRemoteStoreTest > testListOffsetsErrorCodes() PASSED
   
   Gradle Test Run :core:test > Gradle Test Executor 71 > ListOffsetsRequestWithRemoteStoreTest > testCurrentEpochValidation() PASSED
   
   Gradle Test Run :core:test > Gradle Test Executor 71 > ListOffsetsRequestWithRemoteStoreTest > testResponseIncludesLeaderEpoch() PASSED
   
   BUILD SUCCESSFUL in 1m 9s
   55 actionable tasks: 6 executed, 49 up-to-date
   ➜  kafka git:(apache-trunk) date
   Tue May 23 18:05:20 IST 2023
   ➜  kafka git:(apache-trunk) git rev-parse --verify HEAD
   15f8705246e094f7825b76a38d9f12f95d626ee5
   ➜  kafka git:(apache-trunk)
   ```
   


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1171240495


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1273,17 +1328,45 @@ class ReplicaManager(val config: KafkaConfig,
                  _: FencedLeaderEpochException |
                  _: ReplicaNotAvailableException |
                  _: KafkaStorageException |
-                 _: OffsetOutOfRangeException |
                  _: InconsistentTopicIdException) =>
-          LogReadResult(info = new FetchDataInfo(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, MemoryRecords.EMPTY),
-            divergingEpoch = None,
-            highWatermark = UnifiedLog.UnknownOffset,
-            leaderLogStartOffset = UnifiedLog.UnknownOffset,
-            leaderLogEndOffset = UnifiedLog.UnknownOffset,
-            followerLogStartOffset = UnifiedLog.UnknownOffset,
-            fetchTimeMs = -1L,
-            lastStableOffset = None,
-            exception = Some(e))
+          createLogReadResult(e)
+        case e: OffsetOutOfRangeException =>

Review Comment:
   Yes. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1184873142


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -670,6 +875,14 @@ public void close() {
                 } catch (InterruptedException e) {
                     // ignore
                 }
+                remoteStorageReaderThreadPool.shutdownNow();
+                //waits for 2 mins to terminate the current tasks
+                try {
+                    remoteStorageReaderThreadPool.awaitTermination(2, TimeUnit.MINUTES);

Review Comment:
   That code `lifecycleManager.controlledShutdownFuture` is more about processing the controlled shutdown event to the controller for that broker. It will wait for 5 mins before proceeding with other sequence of actions. But that will not get affected because of the code introduced here. 
   Logging subsystem handles unclean shutdown for log segments and it would have been already finished before RemoteLogManager is closed. So, they will not get affected because of this timeout. But we can have a short duration here like 10 secs, we can revisit introducing a config if it is really needed for closing the remote log subsystem.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeqo commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "jeqo (via GitHub)" <gi...@apache.org>.
jeqo commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1183258537


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,6 +622,176 @@ public String toString() {
         }
     }
 
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadata = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadata.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        int startPos = lookupPositionForOffset(rlsMetadata.get(), offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(rlsMetadata.get(), startPos);

Review Comment:
   Sure, thanks.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1184873142


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -670,6 +875,14 @@ public void close() {
                 } catch (InterruptedException e) {
                     // ignore
                 }
+                remoteStorageReaderThreadPool.shutdownNow();
+                //waits for 2 mins to terminate the current tasks
+                try {
+                    remoteStorageReaderThreadPool.awaitTermination(2, TimeUnit.MINUTES);

Review Comment:
   It does not require that to be completed in 5 mins. That code `lifecycleManager.controlledShutdownFuture` is more about processing the controlled shutdown event to the controller for that broker. It will wait for 5 mins before proceeding with other sequence of actions. But that will not get affected because of the code introduced here. 
   Logging subsystem handles unclean shutdown for log segments and it would have been already finished before RemoteLogManager is closed. So, they will not get affected because of this timeout. But we can have a short duration here like 10 secs, we can revisit introducing a config if it is really needed for closing the remote log subsystem.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeqo commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "jeqo (via GitHub)" <gi...@apache.org>.
jeqo commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1165444921


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1118,9 +1122,13 @@ class ReplicaManager(val config: KafkaConfig,
     responseCallback: Seq[(TopicIdPartition, FetchPartitionData)] => Unit
   ): Unit = {
     // check if this fetch request can be satisfied right away
-    val logReadResults = readFromLocalLog(params, fetchInfos, quota, readFromPurgatory = false)
+    val logReadResults = readFromLog(params, fetchInfos, quota, readFromPurgatory = false)
     var bytesReadable: Long = 0
     var errorReadingData = false
+
+    // The 1st topic-partition that has to be read from remote storage
+    var remoteFetchInfo: Optional[RemoteStorageFetchInfo] = Optional.empty()

Review Comment:
   I understand a new PR will come to overcome this, but could we provide further context (on the source code or PR) about the implications of using the first topic-partition only?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13535:
URL: https://github.com/apache/kafka/pull/13535#issuecomment-1551837017

   Thanks @junrao for the latest comments, addressed them with the latest commit. 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13535:
URL: https://github.com/apache/kafka/pull/13535#issuecomment-1539215636

   Thanks @junrao ,  updated the [comment](https://github.com/apache/kafka/pull/13535#discussion_r1184274133) with the latest commit.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] Hangleton commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1190991070


##########
core/src/test/java/kafka/log/remote/RemoteLogReaderTest.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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 org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.OffsetOutOfRangeException;
+import org.apache.kafka.common.record.Records;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.storage.internals.log.FetchDataInfo;
+import org.apache.kafka.storage.internals.log.LogOffsetMetadata;
+import org.apache.kafka.storage.internals.log.RemoteLogReadResult;
+import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo;
+import org.junit.jupiter.api.Test;
+import org.mockito.ArgumentCaptor;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+
+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.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class RemoteLogReaderTest {

Review Comment:
   Should we add a test for this use case:
   ```
   minOneMessage = false
   hardMaxBytesLimit = false
   firstBatchSize > maxBytes?
   ```
   or a combination thereof?
   
   Maybe, please let me know if I can help with a few unit tests. Thanks!



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] junrao commented on pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "junrao (via GitHub)" <gi...@apache.org>.
junrao commented on PR #13535:
URL: https://github.com/apache/kafka/pull/13535#issuecomment-1546009306

   @satishd : Thanks for the updated PR. Are the test failures related to this PR especially the following ones related to remote store?
   
   ```
   [Build / JDK 11 and Scala 2.13 / kafka.server.ListOffsetsRequestWithRemoteStoreTest.testResponseIncludesLeaderEpoch()](https://ci-builds.apache.org/job/Kafka/job/kafka-pr/job/PR-13535/30/testReport/junit/kafka.server/ListOffsetsRequestWithRemoteStoreTest/Build___JDK_11_and_Scala_2_13___testResponseIncludesLeaderEpoch___2/)
   [Build / JDK 8 and Scala 2.12 / kafka.server.ListOffsetsRequestWithRemoteStoreTest.testResponseIncludesLeaderEpoch()](https://ci-builds.apache.org/job/Kafka/job/kafka-pr/job/PR-13535/30/testReport/junit/kafka.server/ListOffsetsRequestWithRemoteStoreTest/Build___JDK_8_and_Scala_2_12___testResponseIncludesLeaderEpoch__/)
   [Build / JDK 8 and Scala 2.12 / kafka.server.ListOffsetsRequestWithRemoteStoreTest.testResponseIncludesLeaderEpoch()](https://ci-builds.apache.org/job/Kafka/job/kafka-pr/job/PR-13535/30/testReport/junit/kafka.server/ListOffsetsRequestWithRemoteStoreTest/Build___JDK_8_and_Scala_2_12___testResponseIncludesLeaderEpoch___2/)
   ```


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1171245421


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,6 +622,176 @@ public String toString() {
         }
     }
 
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadata = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadata.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        int startPos = lookupPositionForOffset(rlsMetadata.get(), offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(rlsMetadata.get(), startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            int updatedFetchSize =

Review Comment:
   Good point. There is no risk here but it is good to be consistent with the local read pattern to return empty records for that case, will update with the changes. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on PR #13535:
URL: https://github.com/apache/kafka/pull/13535#issuecomment-1559352699

   @satishd I raised a PR to fix this: https://github.com/apache/kafka/pull/13747. Could you take a look?


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1171245421


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,6 +622,176 @@ public String toString() {
         }
     }
 
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadata = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadata.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        int startPos = lookupPositionForOffset(rlsMetadata.get(), offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(rlsMetadata.get(), startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            int updatedFetchSize =

Review Comment:
   Good point. There is no risk here but it is good to be consistent with the local read pattern to return empty records for that case, will update with the changes. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1171245421


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,6 +622,176 @@ public String toString() {
         }
     }
 
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadata = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadata.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        int startPos = lookupPositionForOffset(rlsMetadata.get(), offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(rlsMetadata.get(), startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            int updatedFetchSize =

Review Comment:
   Good point. There is no risk here but it is good to be consistent with the local read pattern to return empty records for that 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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] showuon commented on a diff in pull request #13535: [DRAFT] KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1163946844


##########
core/src/main/java/kafka/log/remote/RemoteLogReader.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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 org.apache.kafka.common.errors.OffsetOutOfRangeException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.storage.internals.log.FetchDataInfo;
+import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo;
+import org.slf4j.Logger;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.function.Consumer;
+
+public class RemoteLogReader implements Callable<Void> {
+
+    private final Logger logger;
+    private final RemoteStorageFetchInfo fetchInfo;
+    private final RemoteLogManager rlm;
+    private final Consumer<RemoteLogReadResult> callback;
+
+    public RemoteLogReader(RemoteStorageFetchInfo fetchInfo,
+                           RemoteLogManager rlm,
+                           Consumer<RemoteLogReadResult> callback) {
+        this.fetchInfo = fetchInfo;
+        this.rlm = rlm;
+        this.callback = callback;
+        logger = new LogContext() {
+            @Override
+            public String logPrefix() {
+                return "[" + Thread.currentThread().getName() + "]";
+            }
+        }.logger(RemoteLogReader.class);
+    }
+
+    @Override
+    public Void call() {
+        RemoteLogReadResult result;
+        try {
+            logger.debug("Reading remote bytes for {}", fetchInfo.topicPartition);
+
+            FetchDataInfo fetchDataInfo = rlm.read(fetchInfo);
+            result = new RemoteLogReadResult(Optional.of(fetchDataInfo), Optional.empty());
+        } catch (OffsetOutOfRangeException e) {
+            result = new RemoteLogReadResult(Optional.empty(), Optional.of(e));
+        } catch (Exception e) {
+            logger.error("Error occurred while reading the remote data for {}", fetchInfo.topicPartition, e);
+            result = new RemoteLogReadResult(Optional.empty(), Optional.of(e));
+        }
+
+        logger.debug("Finished reading remote bytes for {}", fetchInfo.topicPartition);

Review Comment:
   ditto



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -619,6 +811,49 @@ long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws RemoteSto
         return offset.orElse(-1L);
     }
 
+    /**
+     * A remote log read task returned by asyncRead(). The caller of asyncRead() can use this object to cancel a
+     * pending task or check if the task is done.
+     */
+    public class AsyncReadTask {

Review Comment:
   I don't see `AsyncReadTask` is used anywhere in the code. Could we remove it?



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1196,17 +1252,45 @@ class ReplicaManager(val config: KafkaConfig,
                  _: FencedLeaderEpochException |
                  _: ReplicaNotAvailableException |
                  _: KafkaStorageException |
-                 _: OffsetOutOfRangeException |
                  _: InconsistentTopicIdException) =>
-          LogReadResult(info = new FetchDataInfo(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, MemoryRecords.EMPTY),
-            divergingEpoch = None,
-            highWatermark = UnifiedLog.UnknownOffset,
-            leaderLogStartOffset = UnifiedLog.UnknownOffset,
-            leaderLogEndOffset = UnifiedLog.UnknownOffset,
-            followerLogStartOffset = UnifiedLog.UnknownOffset,
-            fetchTimeMs = -1L,
-            lastStableOffset = None,
-            exception = Some(e))
+          createLogReadResult(e)
+        case e: OffsetOutOfRangeException =>
+          // In case of offset out of range errors, check for remote log manager for non-compacted topics
+          // to fetch from remote storage. `log` instance should not be null here as that would have caught earlier with
+          // NotLeaderForPartitionException or ReplicaNotAvailableException.
+          // If it is from a follower then send the offset metadata but not the records data as that can be fetched

Review Comment:
   nit: `as [the follower] can be fetched from the remote store directly`



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1083,48 +1095,100 @@ class ReplicaManager(val config: KafkaConfig,
           fetchPartitionStatus += (topicIdPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
         })
       }
-      val delayedFetch = new DelayedFetch(
-        params = params,
-        fetchPartitionStatus = fetchPartitionStatus,
-        replicaManager = this,
-        quota = quota,
-        responseCallback = responseCallback
-      )
-
-      // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
-      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp) }
-
-      // try to complete the request immediately, otherwise put it into the purgatory;
-      // this is because while the delayed fetch operation is being created, new requests
-      // may arrive and hence make this operation completable.
-      delayedFetchPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys)
+
+      if (remoteFetchInfo.isPresent) {
+        val key = new TopicPartitionOperationKey(remoteFetchInfo.get.topicPartition.topic(), remoteFetchInfo.get.topicPartition.partition())
+        val remoteFetchResult = new CompletableFuture[RemoteLogReadResult]
+        var remoteFetchTask: Future[Void] = null
+        try {
+          remoteFetchTask = remoteLogManager.get.asyncRead(remoteFetchInfo.get, (result: RemoteLogReadResult) => {
+            remoteFetchResult.complete(result)
+            delayedRemoteFetchPurgatory.checkAndComplete(key)
+          })
+        } catch {
+          // if the task queue of remote storage reader thread pool is full, return what we currently have
+          // (the data read from local log segment for the other topic-partitions) and an error for the topic-partition that
+          // we couldn't read from remote storage
+          case e: RejectedExecutionException =>
+            val fetchPartitionData = logReadResults.map { case (tp, result) =>
+              val r = {
+                if (tp.topicPartition().equals(remoteFetchInfo.get.topicPartition))
+                  createLogReadResult(e)
+                else
+                  result
+              }
+
+              tp -> r.toFetchPartitionData(false)
+            }
+            responseCallback(fetchPartitionData)
+            return
+        }
+
+        // If there is remote data, we will read remote data, instead of waiting for new data.

Review Comment:
   I think this comment should be put on L1100 after `if (remoteFetchInfo.isPresent) {`



##########
core/src/main/java/kafka/log/remote/RemoteLogReader.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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 org.apache.kafka.common.errors.OffsetOutOfRangeException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.storage.internals.log.FetchDataInfo;
+import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo;
+import org.slf4j.Logger;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.function.Consumer;
+
+public class RemoteLogReader implements Callable<Void> {
+
+    private final Logger logger;
+    private final RemoteStorageFetchInfo fetchInfo;
+    private final RemoteLogManager rlm;
+    private final Consumer<RemoteLogReadResult> callback;
+
+    public RemoteLogReader(RemoteStorageFetchInfo fetchInfo,
+                           RemoteLogManager rlm,
+                           Consumer<RemoteLogReadResult> callback) {
+        this.fetchInfo = fetchInfo;
+        this.rlm = rlm;
+        this.callback = callback;
+        logger = new LogContext() {
+            @Override
+            public String logPrefix() {
+                return "[" + Thread.currentThread().getName() + "]";
+            }
+        }.logger(RemoteLogReader.class);
+    }
+
+    @Override
+    public Void call() {
+        RemoteLogReadResult result;
+        try {
+            logger.debug("Reading remote bytes for {}", fetchInfo.topicPartition);

Review Comment:
   The `remote bytes` is confusing me. How about `Reading [records] from remote storage for {}`?



##########
core/src/main/scala/kafka/server/DelayedRemoteFetch.scala:
##########
@@ -0,0 +1,121 @@
+/**
+ * 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 kafka.log.remote.RemoteLogReadResult
+import org.apache.kafka.common.TopicIdPartition
+import org.apache.kafka.common.errors._
+import org.apache.kafka.storage.internals.log.{FetchParams, FetchPartitionData, LogOffsetMetadata, RemoteStorageFetchInfo}
+
+import java.util.concurrent.{CompletableFuture, Future}
+import java.util.{Optional, OptionalInt, OptionalLong}
+import scala.collection._
+
+/**
+ * A remote fetch operation that can be created by the replica manager and watched
+ * in the remote fetch operation purgatory
+ */
+class DelayedRemoteFetch(remoteFetchTask: Future[Void],
+                         remoteFetchResult: CompletableFuture[RemoteLogReadResult],
+                         remoteFetchInfo: RemoteStorageFetchInfo,
+                         fetchPartitionStatus: Seq[(TopicIdPartition, FetchPartitionStatus)],
+                         fetchParams: FetchParams,
+                         localReadResults: Seq[(TopicIdPartition, LogReadResult)],
+                         replicaManager: ReplicaManager,
+                         quota: ReplicaQuota,
+                         responseCallback: Seq[(TopicIdPartition, FetchPartitionData)] => Unit)
+  extends DelayedOperation(fetchParams.maxWaitMs) {
+
+  /**
+   * The operation can be completed if:
+   *
+   * Case a: This broker is no longer the leader of the partition it tries to fetch
+   * Case b: This broker does not know the partition it tries to fetch
+   * Case c: The remote storage read request completed (succeeded or failed)
+   * Case d: The partition is in an offline log directory on this broker
+   * Case e: This broker is the leader, but the requested epoch is now fenced
+   *
+   * Upon completion, should return whatever data is available for each valid partition
+   */
+  override def tryComplete(): Boolean = {
+    fetchPartitionStatus.foreach {
+      case (topicPartition, fetchStatus) =>
+        val fetchOffset = fetchStatus.startOffsetMetadata
+        val fetchLeaderEpoch = fetchStatus.fetchInfo.currentLeaderEpoch
+        try {
+          if (fetchOffset != LogOffsetMetadata.UNKNOWN_OFFSET_METADATA) {
+            replicaManager.getPartitionOrException(topicPartition.topicPartition())
+          }
+        } catch {
+          case _: KafkaStorageException => // Case d
+            debug(s"Partition $topicPartition is in an offline log directory, satisfy $fetchParams immediately")
+            return forceComplete()
+          case _: UnknownTopicOrPartitionException => // Case b
+            debug(s"Broker no longer knows of partition $topicPartition, satisfy $fetchParams immediately")
+            return forceComplete()
+          case _: FencedLeaderEpochException => // Case e
+            debug(s"Broker is the leader of partition $topicPartition, but the requested epoch " +
+              s"$fetchLeaderEpoch is fenced by the latest leader epoch, satisfy $fetchParams immediately")
+            return forceComplete()
+          case _: NotLeaderOrFollowerException =>  // Case a
+            debug("Broker is no longer the leader of %s, satisfy %s immediately".format(topicPartition, fetchParams))
+            return forceComplete()
+        }
+    }
+    if (remoteFetchResult.isDone)

Review Comment:
   should we add `case C` for this case?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,6 +622,176 @@ public String toString() {
         }
     }
 
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadata = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadata.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier. Try again later.");

Review Comment:
   Are we sure `Try again later` can fix this issue? Especially, it's from application, this log doesn't help at all. Could we remove it? Thoughts?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] Hangleton commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1166887367


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,6 +622,176 @@ public String toString() {
         }
     }
 
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadata = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadata.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        int startPos = lookupPositionForOffset(rlsMetadata.get(), offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(rlsMetadata.get(), startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            int updatedFetchSize =
+                    remoteStorageFetchInfo.minOneMessage && firstBatch.sizeInBytes() > maxBytes
+                            ? firstBatch.sizeInBytes() : maxBytes;
+
+            ByteBuffer buffer = ByteBuffer.allocate(updatedFetchSize);
+            int remainingBytes = updatedFetchSize;
+
+            firstBatch.writeTo(buffer);
+            remainingBytes -= firstBatch.sizeInBytes();
+
+            if (remainingBytes > 0) {
+                // input stream is read till (startPos - 1) while getting the batch of records earlier.
+                // read the input stream until min of (EOF stream or buffer's remaining capacity).
+                Utils.readFully(remoteSegInputStream, buffer);
+            }
+            buffer.flip();
+
+            FetchDataInfo fetchDataInfo = new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.readableRecords(buffer));
+            if (includeAbortedTxns) {
+                fetchDataInfo = addAbortedTransactions(firstBatch.baseOffset(), rlsMetadata.get(), fetchDataInfo);
+            }
+
+            return fetchDataInfo;
+        } finally {
+            Utils.closeQuietly(remoteSegInputStream, "RemoteLogSegmentInputStream");
+        }
+    }
+
+    private int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        return indexCache.lookupOffset(remoteLogSegmentMetadata, offset);
+    }
+
+    private FetchDataInfo addAbortedTransactions(long startOffset,
+                                         RemoteLogSegmentMetadata segmentMetadata,
+                                         FetchDataInfo fetchInfo) throws RemoteStorageException {
+        int fetchSize = fetchInfo.records.sizeInBytes();
+        OffsetPosition startOffsetPosition = new OffsetPosition(fetchInfo.fetchOffsetMetadata.messageOffset,
+                fetchInfo.fetchOffsetMetadata.relativePositionInSegment);
+
+        OffsetIndex offsetIndex = indexCache.getIndexEntry(segmentMetadata).offsetIndex();
+        long upperBoundOffset = offsetIndex.fetchUpperBoundOffset(startOffsetPosition, fetchSize)
+                .map(x -> x.offset).orElse(segmentMetadata.endOffset() + 1);
+
+        final List<FetchResponseData.AbortedTransaction> abortedTransactions = new ArrayList<>();
+
+        Consumer<List<AbortedTxn>> accumulator =
+                abortedTxns -> abortedTransactions.addAll(abortedTxns.stream()
+                        .map(AbortedTxn::asAbortedTransaction).collect(Collectors.toList()));
+
+        collectAbortedTransactions(startOffset, upperBoundOffset, segmentMetadata, accumulator);
+
+        return new FetchDataInfo(fetchInfo.fetchOffsetMetadata,
+                fetchInfo.records,
+                fetchInfo.firstEntryIncomplete,
+                Optional.of(abortedTransactions));
+    }
+
+    private void collectAbortedTransactions(long startOffset,
+                                    long upperBoundOffset,
+                                    RemoteLogSegmentMetadata segmentMetadata,
+                                    Consumer<List<AbortedTxn>> accumulator) throws RemoteStorageException {
+        TopicPartition topicPartition = segmentMetadata.topicIdPartition().topicPartition();
+        Iterator<LogSegment> localLogSegments = fetchLog.apply(topicPartition)
+                .map(log -> JavaConverters.asJavaCollection(log.logSegments()))
+                .map(Collection::iterator)
+                .orElse(Collections.emptyIterator());
+
+        boolean searchInLocalLog = false;
+        Optional<RemoteLogSegmentMetadata> nextSegmentMetadataOpt = Optional.of(segmentMetadata);
+        Optional<TransactionIndex> txnIndexOpt = nextSegmentMetadataOpt.map(metadata -> indexCache.getIndexEntry(metadata).txnIndex());
+
+        while (txnIndexOpt.isPresent()) {
+            TxnIndexSearchResult searchResult = txnIndexOpt.get().collectAbortedTxns(startOffset, upperBoundOffset);
+            accumulator.accept(searchResult.abortedTransactions);
+            if (!searchResult.isComplete) {
+                if (!searchInLocalLog) {
+                    nextSegmentMetadataOpt = findNextSegmentMetadata(nextSegmentMetadataOpt.get());
+
+                    txnIndexOpt = nextSegmentMetadataOpt.map(x -> indexCache.getIndexEntry(x).txnIndex());

Review Comment:
   There may be an availability risk here. The remote index cache allows up to 1,024 entries by default and it is possible to keep a transaction index entry permanently rotated in and out of the cache so that retries of Fetch requests for a partition at a given (constant) fetch offset continuously trigger a download of the index. This, plus the fact there is a global lock for read and write access on the remote index cache, can lead to lock contention on the remote log reader thread pool. In the worst case, it is possible for a partition to be prevented from making progress on the fetch path.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1186714886


##########
core/src/main/java/kafka/log/remote/RemoteLogReader.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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 org.apache.kafka.common.errors.OffsetOutOfRangeException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.storage.internals.log.FetchDataInfo;
+import org.apache.kafka.storage.internals.log.RemoteLogReadResult;
+import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo;
+import org.slf4j.Logger;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.function.Consumer;
+
+public class RemoteLogReader implements Callable<Void> {
+    private final Logger logger;
+    private final RemoteStorageFetchInfo fetchInfo;
+    private final RemoteLogManager rlm;
+    private final Consumer<RemoteLogReadResult> callback;
+
+    public RemoteLogReader(RemoteStorageFetchInfo fetchInfo,
+                           RemoteLogManager rlm,
+                           Consumer<RemoteLogReadResult> callback) {
+        this.fetchInfo = fetchInfo;
+        this.rlm = rlm;
+        this.callback = callback;
+        logger = new LogContext() {
+            @Override
+            public String logPrefix() {
+                return "[" + Thread.currentThread().getName() + "]";
+            }
+        }.logger(RemoteLogReader.class);
+    }
+
+    @Override
+    public Void call() {
+        RemoteLogReadResult result;
+        try {
+            logger.debug("Reading records from remote storage for topic partition {}", fetchInfo.topicPartition);
+
+            FetchDataInfo fetchDataInfo = rlm.read(fetchInfo);
+            result = new RemoteLogReadResult(Optional.of(fetchDataInfo), Optional.empty());
+        } catch (OffsetOutOfRangeException e) {
+            result = new RemoteLogReadResult(Optional.empty(), Optional.of(e));
+        } catch (Exception e) {

Review Comment:
   As the issue is not related to this set of changes, we can look into it later. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1187999169


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1083,48 +1095,100 @@ class ReplicaManager(val config: KafkaConfig,
           fetchPartitionStatus += (topicIdPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
         })
       }
-      val delayedFetch = new DelayedFetch(
-        params = params,
-        fetchPartitionStatus = fetchPartitionStatus,
-        replicaManager = this,
-        quota = quota,
-        responseCallback = responseCallback
-      )
-
-      // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
-      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp) }
-
-      // try to complete the request immediately, otherwise put it into the purgatory;
-      // this is because while the delayed fetch operation is being created, new requests
-      // may arrive and hence make this operation completable.
-      delayedFetchPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys)
+
+      if (remoteFetchInfo.isPresent) {

Review Comment:
   Sure, that check was missed. Good catch. Updated it with the latest commit.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1188566365


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1288,17 +1373,46 @@ class ReplicaManager(val config: KafkaConfig,
                  _: FencedLeaderEpochException |
                  _: ReplicaNotAvailableException |
                  _: KafkaStorageException |
-                 _: OffsetOutOfRangeException |
                  _: InconsistentTopicIdException) =>
-          LogReadResult(info = new FetchDataInfo(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, MemoryRecords.EMPTY),
-            divergingEpoch = None,
-            highWatermark = UnifiedLog.UnknownOffset,
-            leaderLogStartOffset = UnifiedLog.UnknownOffset,
-            leaderLogEndOffset = UnifiedLog.UnknownOffset,
-            followerLogStartOffset = UnifiedLog.UnknownOffset,
-            fetchTimeMs = -1L,
-            lastStableOffset = None,
-            exception = Some(e))
+          createLogReadResult(e)
+        case e: OffsetOutOfRangeException =>
+          // In case of offset out of range errors, check for remote log manager for non-compacted topics
+          // to fetch from remote storage. `log` instance should not be null here as that would have been caught earlier
+          // with NotLeaderForPartitionException or ReplicaNotAvailableException.
+          // If it is from a follower then send the offset metadata only as the data is already available in remote
+          // storage.
+          if (remoteLogManager.isDefined && log != null && log.remoteLogEnabled() &&
+            // Check that the fetch offset is within the offset range within the remote storage layer.
+            log.logStartOffset <= offset && offset < log.localLogStartOffset()) {
+            // For follower fetch requests, throw an error saying that this offset is moved to tiered storage.
+            val highWatermark = log.highWatermark
+            val leaderLogStartOffset = log.logStartOffset

Review Comment:
   It is fine as the offset can always be updated, we will send whatever is the value available at that time. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] Hangleton commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1183397975


##########
core/src/main/java/kafka/log/remote/RemoteLogReader.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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 org.apache.kafka.common.errors.OffsetOutOfRangeException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.storage.internals.log.FetchDataInfo;
+import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo;
+import org.slf4j.Logger;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.function.Consumer;
+
+public class RemoteLogReader implements Callable<Void> {
+
+    private final Logger logger;
+    private final RemoteStorageFetchInfo fetchInfo;
+    private final RemoteLogManager rlm;
+    private final Consumer<RemoteLogReadResult> callback;
+
+    public RemoteLogReader(RemoteStorageFetchInfo fetchInfo,
+                           RemoteLogManager rlm,
+                           Consumer<RemoteLogReadResult> callback) {
+        this.fetchInfo = fetchInfo;
+        this.rlm = rlm;
+        this.callback = callback;
+        logger = new LogContext() {
+            @Override
+            public String logPrefix() {
+                return "[" + Thread.currentThread().getName() + "]";
+            }
+        }.logger(RemoteLogReader.class);
+    }
+
+    @Override
+    public Void call() {
+        RemoteLogReadResult result;
+        try {
+            logger.debug("Reading records from remote storage for topic partition {}", fetchInfo.topicPartition);
+
+            FetchDataInfo fetchDataInfo = rlm.read(fetchInfo);
+            result = new RemoteLogReadResult(Optional.of(fetchDataInfo), Optional.empty());
+        } catch (OffsetOutOfRangeException e) {
+            result = new RemoteLogReadResult(Optional.empty(), Optional.of(e));
+        } catch (Exception e) {
+            logger.error("Error occurred while reading the remote data for {}", fetchInfo.topicPartition, e);
+            result = new RemoteLogReadResult(Optional.empty(), Optional.of(e));
+        }
+
+        logger.debug("Finished reading records from remote storage for topic partition {}", fetchInfo.topicPartition);

Review Comment:
   Should we report of offset out of range when applicable?



##########
core/src/test/java/kafka/log/remote/RemoteLogReaderTest.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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 org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.OffsetOutOfRangeException;
+import org.apache.kafka.common.record.Records;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.storage.internals.log.FetchDataInfo;
+import org.apache.kafka.storage.internals.log.LogOffsetMetadata;
+import org.apache.kafka.storage.internals.log.RemoteLogReadResult;
+import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo;
+import org.junit.jupiter.api.Test;
+import org.mockito.ArgumentCaptor;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+
+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.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class RemoteLogReaderTest {
+    RemoteLogManager mockRLM = mock(RemoteLogManager.class);
+    LogOffsetMetadata logOffsetMetadata = new LogOffsetMetadata(100);
+    Records records = mock(Records.class);
+
+
+    @Test
+    public void testRemoteLogReaderWithoutError() throws RemoteStorageException, IOException {
+        FetchDataInfo fetchDataInfo = new FetchDataInfo(logOffsetMetadata, records);
+        when(mockRLM.read(any(RemoteStorageFetchInfo.class))).thenReturn(fetchDataInfo);
+
+        Consumer<RemoteLogReadResult> callback = mock(Consumer.class);
+        RemoteStorageFetchInfo remoteStorageFetchInfo = new RemoteStorageFetchInfo(0, false, new TopicPartition("test", 0), null, null, false);
+        RemoteLogReader remoteLogReader = new RemoteLogReader(remoteStorageFetchInfo, mockRLM, callback);
+        remoteLogReader.call();
+
+        // verify the callback did get invoked with the expected remoteLogReadResult
+        ArgumentCaptor<RemoteLogReadResult> remoteLogReadResultArg = ArgumentCaptor.forClass(RemoteLogReadResult.class);
+        verify(callback, times(1)).accept(remoteLogReadResultArg.capture());
+        RemoteLogReadResult actualRemoteLogReadResult = remoteLogReadResultArg.getValue();
+        assertFalse(actualRemoteLogReadResult.error.isPresent());
+        assertTrue(actualRemoteLogReadResult.fetchDataInfo.isPresent());
+        assertEquals(fetchDataInfo, actualRemoteLogReadResult.fetchDataInfo.get());
+    }
+
+    @Test
+    public void testRemoteLogReaderWithError() throws RemoteStorageException, IOException {

Review Comment:
   Should we test cases where the plugins throw an exception?



##########
core/src/main/java/kafka/log/remote/RemoteLogReader.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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 org.apache.kafka.common.errors.OffsetOutOfRangeException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.storage.internals.log.FetchDataInfo;
+import org.apache.kafka.storage.internals.log.RemoteLogReadResult;
+import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo;
+import org.slf4j.Logger;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.function.Consumer;
+
+public class RemoteLogReader implements Callable<Void> {
+    private final Logger logger;
+    private final RemoteStorageFetchInfo fetchInfo;
+    private final RemoteLogManager rlm;
+    private final Consumer<RemoteLogReadResult> callback;
+
+    public RemoteLogReader(RemoteStorageFetchInfo fetchInfo,
+                           RemoteLogManager rlm,
+                           Consumer<RemoteLogReadResult> callback) {
+        this.fetchInfo = fetchInfo;
+        this.rlm = rlm;
+        this.callback = callback;
+        logger = new LogContext() {
+            @Override
+            public String logPrefix() {
+                return "[" + Thread.currentThread().getName() + "]";
+            }
+        }.logger(RemoteLogReader.class);
+    }
+
+    @Override
+    public Void call() {
+        RemoteLogReadResult result;
+        try {
+            logger.debug("Reading records from remote storage for topic partition {}", fetchInfo.topicPartition);
+
+            FetchDataInfo fetchDataInfo = rlm.read(fetchInfo);
+            result = new RemoteLogReadResult(Optional.of(fetchDataInfo), Optional.empty());
+        } catch (OffsetOutOfRangeException e) {
+            result = new RemoteLogReadResult(Optional.empty(), Optional.of(e));
+        } catch (Exception e) {

Review Comment:
   Not specific to this code path  but one downside of not propagating interrupt exception from the plugins is that they are logged as exceptions in Kafka server logs. It may be preferable to avoid this by handling these interrupt as first-class exceptions. What do you think?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] divijvaidya commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "divijvaidya (via GitHub)" <gi...@apache.org>.
divijvaidya commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1182398774


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -670,6 +875,14 @@ public void close() {
                 } catch (InterruptedException e) {
                     // ignore
                 }
+                remoteStorageReaderThreadPool.shutdownNow();
+                //waits for 2 mins to terminate the current tasks
+                try {
+                    remoteStorageReaderThreadPool.awaitTermination(2, TimeUnit.MINUTES);

Review Comment:
   How did we decide on 2min. here? I don't think we should block shutdown of the broker on this over here because there are other limits associated with clean vs unclean shutdown. If we do plan to block, we should tie it to overall shutdown timeout. As an example, clean shutdown is expected to be completed in 5 min. see `lifecycleManager.controlledShutdownFuture.get(5L, TimeUnit.MINUTES)` in BrokerServer.scala.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -670,6 +875,14 @@ public void close() {
                 } catch (InterruptedException e) {
                     // ignore
                 }
+                remoteStorageReaderThreadPool.shutdownNow();

Review Comment:
   1. Should we try to complete the pending reads first using `shutdown()` instead of interrupting them via `shutdownNow()`? This would potentially reduce abrupt behaviour if RSM doesn't handle interruption correctly.
   
   I will suggest the following pattern (also see `shutdownAndAwaitTermination` at https://docs.oracle.com/en/java/javase/17/docs/api/java.base/java/util/concurrent/ExecutorService.html)
   ```
           remoteStorageReaderThreadPool.shutdown(); // stop accepting new requests to this pool
   
           try {
                   // wait until the pool has terminated
                   if(!remoteStorageReaderThreadPool.awaitTermination(2, TimeUnit.MINUTES)) {
                       LOGGER.warn("Shutdown for remoteStorageReaderThreadPool didn't complete all tasks.");
                       remoteStorageReaderThreadPool.shutdownNow();    
                   }
           } catch (InterruptedException e) {
               remoteStorageReaderThreadPool.shutdownNow();
               LOGGER.warn("Shutdown for remoteStorageReaderThreadPool interrupted");
           }
   
           LOGGER.info("RemoteStorageReaderThreadPool stopped.");
   ```
   
   2. I would also suggest to explain why aren't we re-interrupting the current thread using Thread.currentThread().interrupt().
   
   3. Please add warn and info logs that help debug the correct order of shutdown.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] junrao commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "junrao (via GitHub)" <gi...@apache.org>.
junrao commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1164654776


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1083,48 +1095,100 @@ class ReplicaManager(val config: KafkaConfig,
           fetchPartitionStatus += (topicIdPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
         })
       }
-      val delayedFetch = new DelayedFetch(
-        params = params,
-        fetchPartitionStatus = fetchPartitionStatus,
-        replicaManager = this,
-        quota = quota,
-        responseCallback = responseCallback
-      )
-
-      // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
-      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp) }
-
-      // try to complete the request immediately, otherwise put it into the purgatory;
-      // this is because while the delayed fetch operation is being created, new requests
-      // may arrive and hence make this operation completable.
-      delayedFetchPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys)
+
+      if (remoteFetchInfo.isPresent) {

Review Comment:
   In line 1082, we should further test `!remoteFetchInfo.isPresent`, right?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,6 +622,176 @@ public String toString() {
         }
     }
 
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadata = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadata.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        int startPos = lookupPositionForOffset(rlsMetadata.get(), offset);

Review Comment:
   We are calling `rlsMetadata.get()` multiple times. We could get it once and reuse the result?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,6 +622,176 @@ public String toString() {
         }
     }
 
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadata = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadata.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        int startPos = lookupPositionForOffset(rlsMetadata.get(), offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(rlsMetadata.get(), startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            int updatedFetchSize =
+                    remoteStorageFetchInfo.minOneMessage && firstBatch.sizeInBytes() > maxBytes
+                            ? firstBatch.sizeInBytes() : maxBytes;
+
+            ByteBuffer buffer = ByteBuffer.allocate(updatedFetchSize);
+            int remainingBytes = updatedFetchSize;
+
+            firstBatch.writeTo(buffer);
+            remainingBytes -= firstBatch.sizeInBytes();
+
+            if (remainingBytes > 0) {
+                // input stream is read till (startPos - 1) while getting the batch of records earlier.
+                // read the input stream until min of (EOF stream or buffer's remaining capacity).
+                Utils.readFully(remoteSegInputStream, buffer);
+            }
+            buffer.flip();
+
+            FetchDataInfo fetchDataInfo = new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.readableRecords(buffer));
+            if (includeAbortedTxns) {
+                fetchDataInfo = addAbortedTransactions(firstBatch.baseOffset(), rlsMetadata.get(), fetchDataInfo);
+            }
+
+            return fetchDataInfo;
+        } finally {
+            Utils.closeQuietly(remoteSegInputStream, "RemoteLogSegmentInputStream");
+        }
+    }
+
+    private int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        return indexCache.lookupOffset(remoteLogSegmentMetadata, offset);
+    }
+
+    private FetchDataInfo addAbortedTransactions(long startOffset,
+                                         RemoteLogSegmentMetadata segmentMetadata,
+                                         FetchDataInfo fetchInfo) throws RemoteStorageException {
+        int fetchSize = fetchInfo.records.sizeInBytes();
+        OffsetPosition startOffsetPosition = new OffsetPosition(fetchInfo.fetchOffsetMetadata.messageOffset,
+                fetchInfo.fetchOffsetMetadata.relativePositionInSegment);
+
+        OffsetIndex offsetIndex = indexCache.getIndexEntry(segmentMetadata).offsetIndex();
+        long upperBoundOffset = offsetIndex.fetchUpperBoundOffset(startOffsetPosition, fetchSize)
+                .map(x -> x.offset).orElse(segmentMetadata.endOffset() + 1);
+
+        final List<FetchResponseData.AbortedTransaction> abortedTransactions = new ArrayList<>();
+
+        Consumer<List<AbortedTxn>> accumulator =
+                abortedTxns -> abortedTransactions.addAll(abortedTxns.stream()
+                        .map(AbortedTxn::asAbortedTransaction).collect(Collectors.toList()));
+
+        collectAbortedTransactions(startOffset, upperBoundOffset, segmentMetadata, accumulator);
+
+        return new FetchDataInfo(fetchInfo.fetchOffsetMetadata,
+                fetchInfo.records,
+                fetchInfo.firstEntryIncomplete,
+                Optional.of(abortedTransactions));
+    }
+
+    private void collectAbortedTransactions(long startOffset,
+                                    long upperBoundOffset,
+                                    RemoteLogSegmentMetadata segmentMetadata,
+                                    Consumer<List<AbortedTxn>> accumulator) throws RemoteStorageException {
+        TopicPartition topicPartition = segmentMetadata.topicIdPartition().topicPartition();
+        Iterator<LogSegment> localLogSegments = fetchLog.apply(topicPartition)

Review Comment:
   We are calling `fetchLog.apply` multiple time when serving a single fetch request. Could we call it once and reuse the result?



##########
core/src/main/java/kafka/log/remote/RemoteLogReader.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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 org.apache.kafka.common.errors.OffsetOutOfRangeException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.storage.internals.log.FetchDataInfo;
+import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo;
+import org.slf4j.Logger;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.function.Consumer;
+
+public class RemoteLogReader implements Callable<Void> {
+

Review Comment:
   extra new line



##########
core/src/main/scala/kafka/server/DelayedRemoteFetch.scala:
##########
@@ -0,0 +1,121 @@
+/**
+ * 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 kafka.log.remote.RemoteLogReadResult
+import org.apache.kafka.common.TopicIdPartition
+import org.apache.kafka.common.errors._
+import org.apache.kafka.storage.internals.log.{FetchParams, FetchPartitionData, LogOffsetMetadata, RemoteStorageFetchInfo}
+
+import java.util.concurrent.{CompletableFuture, Future}
+import java.util.{Optional, OptionalInt, OptionalLong}
+import scala.collection._
+
+/**
+ * A remote fetch operation that can be created by the replica manager and watched
+ * in the remote fetch operation purgatory
+ */
+class DelayedRemoteFetch(remoteFetchTask: Future[Void],
+                         remoteFetchResult: CompletableFuture[RemoteLogReadResult],
+                         remoteFetchInfo: RemoteStorageFetchInfo,
+                         fetchPartitionStatus: Seq[(TopicIdPartition, FetchPartitionStatus)],
+                         fetchParams: FetchParams,
+                         localReadResults: Seq[(TopicIdPartition, LogReadResult)],
+                         replicaManager: ReplicaManager,
+                         quota: ReplicaQuota,
+                         responseCallback: Seq[(TopicIdPartition, FetchPartitionData)] => Unit)
+  extends DelayedOperation(fetchParams.maxWaitMs) {
+
+  /**
+   * The operation can be completed if:
+   *
+   * Case a: This broker is no longer the leader of the partition it tries to fetch
+   * Case b: This broker does not know the partition it tries to fetch
+   * Case c: The remote storage read request completed (succeeded or failed)
+   * Case d: The partition is in an offline log directory on this broker
+   * Case e: This broker is the leader, but the requested epoch is now fenced
+   *
+   * Upon completion, should return whatever data is available for each valid partition
+   */
+  override def tryComplete(): Boolean = {
+    fetchPartitionStatus.foreach {
+      case (topicPartition, fetchStatus) =>
+        val fetchOffset = fetchStatus.startOffsetMetadata
+        val fetchLeaderEpoch = fetchStatus.fetchInfo.currentLeaderEpoch
+        try {
+          if (fetchOffset != LogOffsetMetadata.UNKNOWN_OFFSET_METADATA) {
+            replicaManager.getPartitionOrException(topicPartition.topicPartition())
+          }
+        } catch {
+          case _: KafkaStorageException => // Case d
+            debug(s"Partition $topicPartition is in an offline log directory, satisfy $fetchParams immediately")
+            return forceComplete()
+          case _: UnknownTopicOrPartitionException => // Case b
+            debug(s"Broker no longer knows of partition $topicPartition, satisfy $fetchParams immediately")
+            return forceComplete()
+          case _: FencedLeaderEpochException => // Case e

Review Comment:
   Hmm, `replicaManager.getPartitionOrException` doesn't seem to check the epoch?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,6 +622,176 @@ public String toString() {
         }
     }
 
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadata = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadata.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        int startPos = lookupPositionForOffset(rlsMetadata.get(), offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(rlsMetadata.get(), startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            int updatedFetchSize =
+                    remoteStorageFetchInfo.minOneMessage && firstBatch.sizeInBytes() > maxBytes
+                            ? firstBatch.sizeInBytes() : maxBytes;
+
+            ByteBuffer buffer = ByteBuffer.allocate(updatedFetchSize);
+            int remainingBytes = updatedFetchSize;
+
+            firstBatch.writeTo(buffer);
+            remainingBytes -= firstBatch.sizeInBytes();
+
+            if (remainingBytes > 0) {
+                // input stream is read till (startPos - 1) while getting the batch of records earlier.
+                // read the input stream until min of (EOF stream or buffer's remaining capacity).
+                Utils.readFully(remoteSegInputStream, buffer);
+            }
+            buffer.flip();
+
+            FetchDataInfo fetchDataInfo = new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.readableRecords(buffer));

Review Comment:
   Hmm, `LogOffsetMetadata.position` is important for calculating `upperBoundOffset` in `addAbortedTransactions()`. So, we need to set it instead of relying on the default, right?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,6 +622,176 @@ public String toString() {
         }
     }
 
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();

Review Comment:
   Could we just use `fetchInfo.lastFetchedEpoch`?



##########
core/src/main/scala/kafka/server/DelayedRemoteFetch.scala:
##########
@@ -0,0 +1,121 @@
+/**
+ * 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 kafka.log.remote.RemoteLogReadResult
+import org.apache.kafka.common.TopicIdPartition
+import org.apache.kafka.common.errors._
+import org.apache.kafka.storage.internals.log.{FetchParams, FetchPartitionData, LogOffsetMetadata, RemoteStorageFetchInfo}
+
+import java.util.concurrent.{CompletableFuture, Future}
+import java.util.{Optional, OptionalInt, OptionalLong}
+import scala.collection._
+
+/**
+ * A remote fetch operation that can be created by the replica manager and watched
+ * in the remote fetch operation purgatory
+ */
+class DelayedRemoteFetch(remoteFetchTask: Future[Void],
+                         remoteFetchResult: CompletableFuture[RemoteLogReadResult],
+                         remoteFetchInfo: RemoteStorageFetchInfo,
+                         fetchPartitionStatus: Seq[(TopicIdPartition, FetchPartitionStatus)],
+                         fetchParams: FetchParams,
+                         localReadResults: Seq[(TopicIdPartition, LogReadResult)],
+                         replicaManager: ReplicaManager,
+                         quota: ReplicaQuota,

Review Comment:
   quota seems unused.



##########
core/src/main/java/kafka/log/remote/RemoteLogReadResult.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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 org.apache.kafka.storage.internals.log.FetchDataInfo;
+
+import java.util.Optional;
+
+public class RemoteLogReadResult {
+

Review Comment:
   extra new line



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1243,6 +1327,33 @@ class ReplicaManager(val config: KafkaConfig,
     result
   }
 
+  def createLogReadResult(highWatermark: Long,

Review Comment:
   Could this be private? Ditto below.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,6 +622,176 @@ public String toString() {
         }
     }
 
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadata = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadata.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        int startPos = lookupPositionForOffset(rlsMetadata.get(), offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(rlsMetadata.get(), startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            int updatedFetchSize =
+                    remoteStorageFetchInfo.minOneMessage && firstBatch.sizeInBytes() > maxBytes
+                            ? firstBatch.sizeInBytes() : maxBytes;
+
+            ByteBuffer buffer = ByteBuffer.allocate(updatedFetchSize);
+            int remainingBytes = updatedFetchSize;
+
+            firstBatch.writeTo(buffer);
+            remainingBytes -= firstBatch.sizeInBytes();
+
+            if (remainingBytes > 0) {
+                // input stream is read till (startPos - 1) while getting the batch of records earlier.

Review Comment:
   Hmm, not sure that I understand the comment. What is startPos?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,6 +622,176 @@ public String toString() {
         }
     }
 
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadata = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadata.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        int startPos = lookupPositionForOffset(rlsMetadata.get(), offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(rlsMetadata.get(), startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            int updatedFetchSize =
+                    remoteStorageFetchInfo.minOneMessage && firstBatch.sizeInBytes() > maxBytes
+                            ? firstBatch.sizeInBytes() : maxBytes;
+
+            ByteBuffer buffer = ByteBuffer.allocate(updatedFetchSize);
+            int remainingBytes = updatedFetchSize;
+
+            firstBatch.writeTo(buffer);
+            remainingBytes -= firstBatch.sizeInBytes();
+
+            if (remainingBytes > 0) {
+                // input stream is read till (startPos - 1) while getting the batch of records earlier.
+                // read the input stream until min of (EOF stream or buffer's remaining capacity).
+                Utils.readFully(remoteSegInputStream, buffer);
+            }
+            buffer.flip();
+
+            FetchDataInfo fetchDataInfo = new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.readableRecords(buffer));
+            if (includeAbortedTxns) {
+                fetchDataInfo = addAbortedTransactions(firstBatch.baseOffset(), rlsMetadata.get(), fetchDataInfo);
+            }
+
+            return fetchDataInfo;
+        } finally {
+            Utils.closeQuietly(remoteSegInputStream, "RemoteLogSegmentInputStream");
+        }
+    }
+
+    private int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        return indexCache.lookupOffset(remoteLogSegmentMetadata, offset);
+    }
+
+    private FetchDataInfo addAbortedTransactions(long startOffset,
+                                         RemoteLogSegmentMetadata segmentMetadata,
+                                         FetchDataInfo fetchInfo) throws RemoteStorageException {
+        int fetchSize = fetchInfo.records.sizeInBytes();
+        OffsetPosition startOffsetPosition = new OffsetPosition(fetchInfo.fetchOffsetMetadata.messageOffset,
+                fetchInfo.fetchOffsetMetadata.relativePositionInSegment);
+
+        OffsetIndex offsetIndex = indexCache.getIndexEntry(segmentMetadata).offsetIndex();
+        long upperBoundOffset = offsetIndex.fetchUpperBoundOffset(startOffsetPosition, fetchSize)
+                .map(x -> x.offset).orElse(segmentMetadata.endOffset() + 1);
+
+        final List<FetchResponseData.AbortedTransaction> abortedTransactions = new ArrayList<>();
+
+        Consumer<List<AbortedTxn>> accumulator =
+                abortedTxns -> abortedTransactions.addAll(abortedTxns.stream()
+                        .map(AbortedTxn::asAbortedTransaction).collect(Collectors.toList()));
+
+        collectAbortedTransactions(startOffset, upperBoundOffset, segmentMetadata, accumulator);
+
+        return new FetchDataInfo(fetchInfo.fetchOffsetMetadata,
+                fetchInfo.records,
+                fetchInfo.firstEntryIncomplete,
+                Optional.of(abortedTransactions));
+    }
+
+    private void collectAbortedTransactions(long startOffset,
+                                    long upperBoundOffset,
+                                    RemoteLogSegmentMetadata segmentMetadata,
+                                    Consumer<List<AbortedTxn>> accumulator) throws RemoteStorageException {
+        TopicPartition topicPartition = segmentMetadata.topicIdPartition().topicPartition();
+        Iterator<LogSegment> localLogSegments = fetchLog.apply(topicPartition)
+                .map(log -> JavaConverters.asJavaCollection(log.logSegments()))
+                .map(Collection::iterator)
+                .orElse(Collections.emptyIterator());
+
+        boolean searchInLocalLog = false;
+        Optional<RemoteLogSegmentMetadata> nextSegmentMetadataOpt = Optional.of(segmentMetadata);
+        Optional<TransactionIndex> txnIndexOpt = nextSegmentMetadataOpt.map(metadata -> indexCache.getIndexEntry(metadata).txnIndex());
+
+        while (txnIndexOpt.isPresent()) {
+            TxnIndexSearchResult searchResult = txnIndexOpt.get().collectAbortedTxns(startOffset, upperBoundOffset);
+            accumulator.accept(searchResult.abortedTransactions);
+            if (!searchResult.isComplete) {
+                if (!searchInLocalLog) {
+                    nextSegmentMetadataOpt = findNextSegmentMetadata(nextSegmentMetadataOpt.get());
+
+                    txnIndexOpt = nextSegmentMetadataOpt.map(x -> indexCache.getIndexEntry(x).txnIndex());
+                    if (!txnIndexOpt.isPresent()) {
+                        searchInLocalLog = true;
+                    }
+                }
+
+                if (searchInLocalLog) {
+                    txnIndexOpt = (localLogSegments.hasNext()) ? Optional.of(localLogSegments.next().txnIndex()) : Optional.empty();

Review Comment:
   localLogSegments could overlap with the remote segment. So, we could introduce duplicate aborted txn records here.



##########
core/src/main/java/kafka/log/remote/RemoteLogReader.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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 org.apache.kafka.common.errors.OffsetOutOfRangeException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.storage.internals.log.FetchDataInfo;
+import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo;
+import org.slf4j.Logger;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.function.Consumer;
+
+public class RemoteLogReader implements Callable<Void> {
+
+    private final Logger logger;
+    private final RemoteStorageFetchInfo fetchInfo;
+    private final RemoteLogManager rlm;
+    private final Consumer<RemoteLogReadResult> callback;
+
+    public RemoteLogReader(RemoteStorageFetchInfo fetchInfo,
+                           RemoteLogManager rlm,
+                           Consumer<RemoteLogReadResult> callback) {
+        this.fetchInfo = fetchInfo;
+        this.rlm = rlm;
+        this.callback = callback;
+        logger = new LogContext() {
+            @Override
+            public String logPrefix() {
+                return "[" + Thread.currentThread().getName() + "]";
+            }
+        }.logger(RemoteLogReader.class);
+    }
+
+    @Override
+    public Void call() {
+        RemoteLogReadResult result;
+        try {
+            logger.debug("Reading records from remote storage for topic partition {}", fetchInfo.topicPartition);
+
+            FetchDataInfo fetchDataInfo = rlm.read(fetchInfo);
+            result = new RemoteLogReadResult(Optional.of(fetchDataInfo), Optional.empty());
+        } catch (OffsetOutOfRangeException e) {
+            result = new RemoteLogReadResult(Optional.empty(), Optional.of(e));
+        } catch (Exception e) {
+            logger.error("Error occurred while reading the remote data for {}", fetchInfo.topicPartition, e);
+            result = new RemoteLogReadResult(Optional.empty(), Optional.of(e));
+        }
+
+        logger.debug("Finished reading records from remote storage for topic partition {}", fetchInfo.topicPartition);

Review Comment:
   Should we log fetchInfo too?



##########
core/src/main/java/kafka/log/remote/RemoteLogReadResult.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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 org.apache.kafka.storage.internals.log.FetchDataInfo;
+
+import java.util.Optional;
+
+public class RemoteLogReadResult {
+
+    public final Optional<FetchDataInfo> fetchDataInfo;
+    public final Optional<Throwable> error;
+
+    RemoteLogReadResult(Optional<FetchDataInfo> fetchDataInfo, Optional<Throwable> error) {
+        this.fetchDataInfo = fetchDataInfo;
+        this.error = error;
+    }
+

Review Comment:
   extra new line



##########
core/src/main/scala/kafka/server/DelayedRemoteFetch.scala:
##########
@@ -0,0 +1,121 @@
+/**
+ * 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 kafka.log.remote.RemoteLogReadResult
+import org.apache.kafka.common.TopicIdPartition
+import org.apache.kafka.common.errors._
+import org.apache.kafka.storage.internals.log.{FetchParams, FetchPartitionData, LogOffsetMetadata, RemoteStorageFetchInfo}
+
+import java.util.concurrent.{CompletableFuture, Future}
+import java.util.{Optional, OptionalInt, OptionalLong}
+import scala.collection._
+
+/**
+ * A remote fetch operation that can be created by the replica manager and watched
+ * in the remote fetch operation purgatory
+ */
+class DelayedRemoteFetch(remoteFetchTask: Future[Void],
+                         remoteFetchResult: CompletableFuture[RemoteLogReadResult],
+                         remoteFetchInfo: RemoteStorageFetchInfo,
+                         fetchPartitionStatus: Seq[(TopicIdPartition, FetchPartitionStatus)],
+                         fetchParams: FetchParams,
+                         localReadResults: Seq[(TopicIdPartition, LogReadResult)],
+                         replicaManager: ReplicaManager,
+                         quota: ReplicaQuota,
+                         responseCallback: Seq[(TopicIdPartition, FetchPartitionData)] => Unit)
+  extends DelayedOperation(fetchParams.maxWaitMs) {
+
+  /**
+   * The operation can be completed if:
+   *
+   * Case a: This broker is no longer the leader of the partition it tries to fetch
+   * Case b: This broker does not know the partition it tries to fetch
+   * Case c: The remote storage read request completed (succeeded or failed)
+   * Case d: The partition is in an offline log directory on this broker
+   * Case e: This broker is the leader, but the requested epoch is now fenced
+   *
+   * Upon completion, should return whatever data is available for each valid partition
+   */
+  override def tryComplete(): Boolean = {
+    fetchPartitionStatus.foreach {
+      case (topicPartition, fetchStatus) =>
+        val fetchOffset = fetchStatus.startOffsetMetadata
+        val fetchLeaderEpoch = fetchStatus.fetchInfo.currentLeaderEpoch
+        try {
+          if (fetchOffset != LogOffsetMetadata.UNKNOWN_OFFSET_METADATA) {
+            replicaManager.getPartitionOrException(topicPartition.topicPartition())
+          }
+        } catch {
+          case _: KafkaStorageException => // Case d
+            debug(s"Partition $topicPartition is in an offline log directory, satisfy $fetchParams immediately")
+            return forceComplete()
+          case _: UnknownTopicOrPartitionException => // Case b
+            debug(s"Broker no longer knows of partition $topicPartition, satisfy $fetchParams immediately")
+            return forceComplete()
+          case _: FencedLeaderEpochException => // Case e
+            debug(s"Broker is the leader of partition $topicPartition, but the requested epoch " +
+              s"$fetchLeaderEpoch is fenced by the latest leader epoch, satisfy $fetchParams immediately")
+            return forceComplete()
+          case _: NotLeaderOrFollowerException =>  // Case a

Review Comment:
   Should we add follower in the comment and the log below for case a?



##########
core/src/main/java/kafka/log/remote/RemoteLogReader.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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;

Review Comment:
   Could RemoteLogReader, RemoteLogReadResult and RemoteStorageThreadPool all be in the storage module?



##########
core/src/main/java/kafka/log/remote/RemoteStorageThreadPool.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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 org.apache.kafka.common.internals.FatalExitError;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class RemoteStorageThreadPool extends ThreadPoolExecutor {
+    private final Logger logger;
+
+    public RemoteStorageThreadPool(String name,
+                                   String threadNamePrefix,
+                                   int numThreads,
+                                   int maxPendingTasks,
+                                   Time time) {

Review Comment:
   unused param



##########
core/src/main/java/kafka/log/remote/RemoteStorageThreadPool.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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 org.apache.kafka.common.internals.FatalExitError;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class RemoteStorageThreadPool extends ThreadPoolExecutor {
+    private final Logger logger;
+
+    public RemoteStorageThreadPool(String name,

Review Comment:
   unused param



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1192183587


##########
core/src/test/java/kafka/log/remote/RemoteLogReaderTest.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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 org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.OffsetOutOfRangeException;
+import org.apache.kafka.common.record.Records;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.storage.internals.log.FetchDataInfo;
+import org.apache.kafka.storage.internals.log.LogOffsetMetadata;
+import org.apache.kafka.storage.internals.log.RemoteLogReadResult;
+import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo;
+import org.junit.jupiter.api.Test;
+import org.mockito.ArgumentCaptor;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+
+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.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class RemoteLogReaderTest {

Review Comment:
   Sure, you can add more unit tests. We have a few in Scala, which will be converted to Java and raised in a followup PR later. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] kamalcph commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "kamalcph (via GitHub)" <gi...@apache.org>.
kamalcph commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1190617224


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,25 +623,212 @@ public String toString() {
         }
     }
 
-    long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws RemoteStorageException {
-        Optional<Long> offset = Optional.empty();
-        Optional<UnifiedLog> maybeLog = fetchLog.apply(topicIdPartition.topicPartition());
-        if (maybeLog.isPresent()) {
-            UnifiedLog log = maybeLog.get();
-            Option<LeaderEpochFileCache> maybeLeaderEpochFileCache = log.leaderEpochCache();
-            if (maybeLeaderEpochFileCache.isDefined()) {
-                LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get();
-                OptionalInt epoch = cache.latestEpoch();
-                while (!offset.isPresent() && epoch.isPresent()) {
-                    offset = remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch.getAsInt());
-                    epoch = cache.previousEpoch(epoch.getAsInt());
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadataOptional = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadataOptional.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        RemoteLogSegmentMetadata remoteLogSegmentMetadata = rlsMetadataOptional.get();
+        int startPos = lookupPositionForOffset(remoteLogSegmentMetadata, offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            int firstBatchSize = firstBatch.sizeInBytes();
+            // An empty record is sent instead of an incomplete batch when
+            //  - there is no minimum-one-message constraint and
+            //  - the first batch size is more than maximum bytes that can be sent and
+            //  - for FetchRequest version 3 or above.
+            if (!remoteStorageFetchInfo.minOneMessage &&
+                    !remoteStorageFetchInfo.hardMaxBytesLimit &&
+                    firstBatchSize > maxBytes) {
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY);
+            }
+
+            int updatedFetchSize =
+                    remoteStorageFetchInfo.minOneMessage && firstBatchSize > maxBytes ? firstBatchSize : maxBytes;
+
+            ByteBuffer buffer = ByteBuffer.allocate(updatedFetchSize);
+            int remainingBytes = updatedFetchSize;
+
+            firstBatch.writeTo(buffer);
+            remainingBytes -= firstBatchSize;
+
+            if (remainingBytes > 0) {
+                // read the input stream until min of (EOF stream or buffer's remaining capacity).
+                Utils.readFully(remoteSegInputStream, buffer);
+            }
+            buffer.flip();
+
+            FetchDataInfo fetchDataInfo = new FetchDataInfo(
+                    new LogOffsetMetadata(offset, remoteLogSegmentMetadata.startOffset(), startPos),
+                    MemoryRecords.readableRecords(buffer));
+            if (includeAbortedTxns) {
+                fetchDataInfo = addAbortedTransactions(firstBatch.baseOffset(), remoteLogSegmentMetadata, fetchDataInfo, logOptional.get());
+            }
+
+            return fetchDataInfo;
+        } finally {
+            Utils.closeQuietly(remoteSegInputStream, "RemoteLogSegmentInputStream");
+        }
+    }
+
+    private int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        return indexCache.lookupOffset(remoteLogSegmentMetadata, offset);
+    }
+
+    private FetchDataInfo addAbortedTransactions(long startOffset,
+                                                 RemoteLogSegmentMetadata segmentMetadata,
+                                                 FetchDataInfo fetchInfo,
+                                                 UnifiedLog log) throws RemoteStorageException {
+        int fetchSize = fetchInfo.records.sizeInBytes();
+        OffsetPosition startOffsetPosition = new OffsetPosition(fetchInfo.fetchOffsetMetadata.messageOffset,
+                fetchInfo.fetchOffsetMetadata.relativePositionInSegment);
+
+        OffsetIndex offsetIndex = indexCache.getIndexEntry(segmentMetadata).offsetIndex();
+        long upperBoundOffset = offsetIndex.fetchUpperBoundOffset(startOffsetPosition, fetchSize)
+                .map(x -> x.offset).orElse(segmentMetadata.endOffset() + 1);
+
+        final Set<FetchResponseData.AbortedTransaction> abortedTransactions = new HashSet<>();
+
+        Consumer<List<AbortedTxn>> accumulator =
+                abortedTxns -> abortedTransactions.addAll(abortedTxns.stream()
+                        .map(AbortedTxn::asAbortedTransaction).collect(Collectors.toList()));
+
+        collectAbortedTransactions(startOffset, upperBoundOffset, segmentMetadata, accumulator, log);
+
+        return new FetchDataInfo(fetchInfo.fetchOffsetMetadata,
+                fetchInfo.records,
+                fetchInfo.firstEntryIncomplete,
+                Optional.of(abortedTransactions.isEmpty() ? Collections.emptyList() : new ArrayList<>(abortedTransactions)));
+    }
+
+    private void collectAbortedTransactions(long startOffset,
+                                            long upperBoundOffset,
+                                            RemoteLogSegmentMetadata segmentMetadata,
+                                            Consumer<List<AbortedTxn>> accumulator,
+                                            UnifiedLog log) throws RemoteStorageException {
+        // Search in remote segments first.
+        Optional<RemoteLogSegmentMetadata> nextSegmentMetadataOpt = Optional.of(segmentMetadata);
+        while (nextSegmentMetadataOpt.isPresent()) {
+            Optional<TransactionIndex> txnIndexOpt = nextSegmentMetadataOpt.map(metadata -> indexCache.getIndexEntry(metadata).txnIndex());
+            if (txnIndexOpt.isPresent()) {
+                TxnIndexSearchResult searchResult = txnIndexOpt.get().collectAbortedTxns(startOffset, upperBoundOffset);
+                accumulator.accept(searchResult.abortedTransactions);
+                if (searchResult.isComplete) {
+                    // Return immediately when the search result is complete, it does not need to go through local log segments.
+                    return;
                 }
             }
+
+            nextSegmentMetadataOpt = findNextSegmentMetadata(nextSegmentMetadataOpt.get(), log.leaderEpochCache());
+        }
+
+        // Search in local segments
+        collectAbortedTransactionInLocalSegments(startOffset, upperBoundOffset, accumulator, JavaConverters.asJavaIterator(log.logSegments().iterator()));
+    }
+
+    private void collectAbortedTransactionInLocalSegments(long startOffset,
+                                                          long upperBoundOffset,
+                                                          Consumer<List<AbortedTxn>> accumulator,
+                                                          Iterator<LogSegment> localLogSegments) {
+        while (localLogSegments.hasNext()) {
+            TransactionIndex txnIndexOpt = localLogSegments.next().txnIndex();

Review Comment:
   txnIndexOpt -> txnIndex



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1192195822


##########
core/src/test/scala/integration/kafka/server/DelayedRemoteFetchTest.scala:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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 kafka.cluster.Partition
+import org.apache.kafka.common.errors.NotLeaderOrFollowerException
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.record.MemoryRecords
+import org.apache.kafka.common.requests.FetchRequest
+import org.apache.kafka.common.{TopicIdPartition, Uuid}
+import org.apache.kafka.storage.internals.log._
+import org.junit.jupiter.api.Assertions._
+import org.junit.jupiter.api.Test
+import org.mockito.Mockito.{mock, when}
+
+import java.util.Optional
+import java.util.concurrent.CompletableFuture
+
+import scala.collection._

Review Comment:
   We have a few more tests that need to be pulled out from the existing repo with other changes, we can convert this test into Java in that PR.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1203673361


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1335,6 +1433,50 @@ class ReplicaManager(val config: KafkaConfig,
     result
   }
 
+  private def handleOffsetOutOfRangeError(tp: TopicIdPartition, params: FetchParams, fetchInfo: PartitionData,
+                                          adjustedMaxBytes: Int, minOneMessage:
+                                          Boolean, log: UnifiedLog, fetchTimeMs: Long,
+                                          exception: OffsetOutOfRangeException): LogReadResult = {
+    val offset = fetchInfo.fetchOffset
+    // In case of offset out of range errors, handle it for tiered storage only if all the below conditions are true.
+    //   1) remote log manager is enabled and it is available
+    //   2) `log` instance should not be null here as that would have been caught earlier with NotLeaderForPartitionException or ReplicaNotAvailableException.
+    //   3) fetch offset is within the offset range of the remote storage layer
+    if (remoteLogManager.isDefined && log != null && log.remoteLogEnabled() &&
+      log.logStartOffset <= offset && offset < log.localLogStartOffset())
+    {
+      val highWatermark = log.highWatermark
+      val leaderLogStartOffset = log.logStartOffset
+      val leaderLogEndOffset = log.logEndOffset
+
+      if (params.isFromFollower) {
+        // If it is from a follower then send the offset metadata only as the data is already available in remote
+        // storage and throw an error saying that this offset is moved to tiered storage.
+        createLogReadResult(highWatermark, leaderLogStartOffset, leaderLogEndOffset,
+          new OffsetMovedToTieredStorageException("Given offset" + offset + " is moved to tiered storage"))
+      } else {
+        // For consume fetch requests, create a dummy FetchDataInfo with the remote storage fetch information.
+        // For the first topic-partition that needs remote data, we will use this information to read the data in another thread.
+        val fetchDataInfo =
+        new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false, Optional.empty(),

Review Comment:
   nit: `new FetchDataInfo` should be on previous line or indented.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] kamalcph commented on pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "kamalcph (via GitHub)" <gi...@apache.org>.
kamalcph commented on PR #13535:
URL: https://github.com/apache/kafka/pull/13535#issuecomment-1549993361

   > @satishd : Thanks for the updated PR. Are the test failures related to this PR especially the following ones related to remote store?
   > 
   > ```
   > [Build / JDK 11 and Scala 2.13 / kafka.server.ListOffsetsRequestWithRemoteStoreTest.testResponseIncludesLeaderEpoch()](https://ci-builds.apache.org/job/Kafka/job/kafka-pr/job/PR-13535/30/testReport/junit/kafka.server/ListOffsetsRequestWithRemoteStoreTest/Build___JDK_11_and_Scala_2_13___testResponseIncludesLeaderEpoch___2/)
   > [Build / JDK 8 and Scala 2.12 / kafka.server.ListOffsetsRequestWithRemoteStoreTest.testResponseIncludesLeaderEpoch()](https://ci-builds.apache.org/job/Kafka/job/kafka-pr/job/PR-13535/30/testReport/junit/kafka.server/ListOffsetsRequestWithRemoteStoreTest/Build___JDK_8_and_Scala_2_12___testResponseIncludesLeaderEpoch__/)
   > [Build / JDK 8 and Scala 2.12 / kafka.server.ListOffsetsRequestWithRemoteStoreTest.testResponseIncludesLeaderEpoch()](https://ci-builds.apache.org/job/Kafka/job/kafka-pr/job/PR-13535/30/testReport/junit/kafka.server/ListOffsetsRequestWithRemoteStoreTest/Build___JDK_8_and_Scala_2_12___testResponseIncludesLeaderEpoch___2/)
   > ```
   
   It is not because of the changes in the PR. #10389 attempted to stabilize this test but it can still fail if the machine is slow.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] showuon commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1165342232


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,6 +622,176 @@ public String toString() {
         }
     }
 
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadata = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadata.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        int startPos = lookupPositionForOffset(rlsMetadata.get(), offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(rlsMetadata.get(), startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,

Review Comment:
   I think we need to log something 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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1171242181


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,6 +622,176 @@ public String toString() {
         }
     }
 
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadata = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadata.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        int startPos = lookupPositionForOffset(rlsMetadata.get(), offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(rlsMetadata.get(), startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            int updatedFetchSize =
+                    remoteStorageFetchInfo.minOneMessage && firstBatch.sizeInBytes() > maxBytes
+                            ? firstBatch.sizeInBytes() : maxBytes;
+
+            ByteBuffer buffer = ByteBuffer.allocate(updatedFetchSize);
+            int remainingBytes = updatedFetchSize;
+
+            firstBatch.writeTo(buffer);
+            remainingBytes -= firstBatch.sizeInBytes();
+
+            if (remainingBytes > 0) {
+                // input stream is read till (startPos - 1) while getting the batch of records earlier.
+                // read the input stream until min of (EOF stream or buffer's remaining capacity).
+                Utils.readFully(remoteSegInputStream, buffer);
+            }
+            buffer.flip();
+
+            FetchDataInfo fetchDataInfo = new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.readableRecords(buffer));
+            if (includeAbortedTxns) {
+                fetchDataInfo = addAbortedTransactions(firstBatch.baseOffset(), rlsMetadata.get(), fetchDataInfo);
+            }
+
+            return fetchDataInfo;
+        } finally {
+            Utils.closeQuietly(remoteSegInputStream, "RemoteLogSegmentInputStream");
+        }
+    }
+
+    private int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        return indexCache.lookupOffset(remoteLogSegmentMetadata, offset);
+    }
+
+    private FetchDataInfo addAbortedTransactions(long startOffset,
+                                         RemoteLogSegmentMetadata segmentMetadata,
+                                         FetchDataInfo fetchInfo) throws RemoteStorageException {
+        int fetchSize = fetchInfo.records.sizeInBytes();
+        OffsetPosition startOffsetPosition = new OffsetPosition(fetchInfo.fetchOffsetMetadata.messageOffset,
+                fetchInfo.fetchOffsetMetadata.relativePositionInSegment);
+
+        OffsetIndex offsetIndex = indexCache.getIndexEntry(segmentMetadata).offsetIndex();
+        long upperBoundOffset = offsetIndex.fetchUpperBoundOffset(startOffsetPosition, fetchSize)
+                .map(x -> x.offset).orElse(segmentMetadata.endOffset() + 1);
+
+        final List<FetchResponseData.AbortedTransaction> abortedTransactions = new ArrayList<>();
+
+        Consumer<List<AbortedTxn>> accumulator =
+                abortedTxns -> abortedTransactions.addAll(abortedTxns.stream()
+                        .map(AbortedTxn::asAbortedTransaction).collect(Collectors.toList()));
+
+        collectAbortedTransactions(startOffset, upperBoundOffset, segmentMetadata, accumulator);
+
+        return new FetchDataInfo(fetchInfo.fetchOffsetMetadata,
+                fetchInfo.records,
+                fetchInfo.firstEntryIncomplete,
+                Optional.of(abortedTransactions));
+    }
+
+    private void collectAbortedTransactions(long startOffset,
+                                    long upperBoundOffset,
+                                    RemoteLogSegmentMetadata segmentMetadata,
+                                    Consumer<List<AbortedTxn>> accumulator) throws RemoteStorageException {
+        TopicPartition topicPartition = segmentMetadata.topicIdPartition().topicPartition();
+        Iterator<LogSegment> localLogSegments = fetchLog.apply(topicPartition)
+                .map(log -> JavaConverters.asJavaCollection(log.logSegments()))
+                .map(Collection::iterator)
+                .orElse(Collections.emptyIterator());
+
+        boolean searchInLocalLog = false;
+        Optional<RemoteLogSegmentMetadata> nextSegmentMetadataOpt = Optional.of(segmentMetadata);
+        Optional<TransactionIndex> txnIndexOpt = nextSegmentMetadataOpt.map(metadata -> indexCache.getIndexEntry(metadata).txnIndex());
+
+        while (txnIndexOpt.isPresent()) {
+            TxnIndexSearchResult searchResult = txnIndexOpt.get().collectAbortedTxns(startOffset, upperBoundOffset);
+            accumulator.accept(searchResult.abortedTransactions);
+            if (!searchResult.isComplete) {
+                if (!searchInLocalLog) {
+                    nextSegmentMetadataOpt = findNextSegmentMetadata(nextSegmentMetadataOpt.get());
+
+                    txnIndexOpt = nextSegmentMetadataOpt.map(x -> indexCache.getIndexEntry(x).txnIndex());

Review Comment:
   1024 is the default value. Of course, this will be exposed as a config so that users can configure based on their workload patterns. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1186813113


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,25 +622,208 @@ public String toString() {
         }
     }
 
-    long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws RemoteStorageException {
-        Optional<Long> offset = Optional.empty();
-        Optional<UnifiedLog> maybeLog = fetchLog.apply(topicIdPartition.topicPartition());
-        if (maybeLog.isPresent()) {
-            UnifiedLog log = maybeLog.get();
-            Option<LeaderEpochFileCache> maybeLeaderEpochFileCache = log.leaderEpochCache();
-            if (maybeLeaderEpochFileCache.isDefined()) {
-                LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get();
-                OptionalInt epoch = cache.latestEpoch();
-                while (!offset.isPresent() && epoch.isPresent()) {
-                    offset = remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch.getAsInt());
-                    epoch = cache.previousEpoch(epoch.getAsInt());
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadataOptional = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadataOptional.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        RemoteLogSegmentMetadata remoteLogSegmentMetadata = rlsMetadataOptional.get();
+        int startPos = lookupPositionForOffset(remoteLogSegmentMetadata, offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            // An empty record is sent instead of an incomplete batch when there is no minimum-one-message constraint
+            // and for FetchRequest version 3 and above and the first batch size is more than maximum bytes that can be sent.
+            int firstBatchSize = firstBatch.sizeInBytes();
+            if (!remoteStorageFetchInfo.minOneMessage &&
+                    !remoteStorageFetchInfo.hardMaxBytesLimit &&
+                    firstBatchSize > maxBytes) {
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY);
+            }
+
+            int updatedFetchSize =
+                    remoteStorageFetchInfo.minOneMessage && firstBatchSize > maxBytes ? firstBatchSize : maxBytes;
+
+            ByteBuffer buffer = ByteBuffer.allocate(updatedFetchSize);
+            int remainingBytes = updatedFetchSize;
+
+            firstBatch.writeTo(buffer);
+            remainingBytes -= firstBatchSize;
+
+            if (remainingBytes > 0) {
+                // read the input stream until min of (EOF stream or buffer's remaining capacity).
+                Utils.readFully(remoteSegInputStream, buffer);
+            }
+            buffer.flip();
+
+            FetchDataInfo fetchDataInfo = new FetchDataInfo(
+                    new LogOffsetMetadata(offset, remoteLogSegmentMetadata.startOffset(), startPos),
+                    MemoryRecords.readableRecords(buffer));
+            if (includeAbortedTxns) {
+                fetchDataInfo = addAbortedTransactions(firstBatch.baseOffset(), remoteLogSegmentMetadata, fetchDataInfo, logOptional.get());
+            }
+
+            return fetchDataInfo;
+        } finally {
+            Utils.closeQuietly(remoteSegInputStream, "RemoteLogSegmentInputStream");
+        }
+    }
+
+    private int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        return indexCache.lookupOffset(remoteLogSegmentMetadata, offset);
+    }
+
+    private FetchDataInfo addAbortedTransactions(long startOffset,
+                                                 RemoteLogSegmentMetadata segmentMetadata,
+                                                 FetchDataInfo fetchInfo,
+                                                 UnifiedLog log) throws RemoteStorageException {
+        int fetchSize = fetchInfo.records.sizeInBytes();
+        OffsetPosition startOffsetPosition = new OffsetPosition(fetchInfo.fetchOffsetMetadata.messageOffset,
+                fetchInfo.fetchOffsetMetadata.relativePositionInSegment);
+
+        OffsetIndex offsetIndex = indexCache.getIndexEntry(segmentMetadata).offsetIndex();
+        long upperBoundOffset = offsetIndex.fetchUpperBoundOffset(startOffsetPosition, fetchSize)
+                .map(x -> x.offset).orElse(segmentMetadata.endOffset() + 1);
+
+        final Set<FetchResponseData.AbortedTransaction> abortedTransactions = new HashSet<>();
+
+        Consumer<List<AbortedTxn>> accumulator =
+                abortedTxns -> abortedTransactions.addAll(abortedTxns.stream()
+                        .map(AbortedTxn::asAbortedTransaction).collect(Collectors.toList()));
+
+        collectAbortedTransactions(startOffset, upperBoundOffset, segmentMetadata, accumulator, log);
+
+        return new FetchDataInfo(fetchInfo.fetchOffsetMetadata,
+                fetchInfo.records,
+                fetchInfo.firstEntryIncomplete,
+                Optional.of(abortedTransactions.isEmpty() ? Collections.emptyList() : new ArrayList<>(abortedTransactions)));
+    }
+
+    private void collectAbortedTransactions(long startOffset,
+                                            long upperBoundOffset,
+                                            RemoteLogSegmentMetadata segmentMetadata,
+                                            Consumer<List<AbortedTxn>> accumulator,
+                                            UnifiedLog log) throws RemoteStorageException {
+        Iterator<LogSegment> localLogSegments = JavaConverters.asJavaIterator(log.logSegments().iterator());
+
+        boolean searchInLocalLog = false;
+        Optional<RemoteLogSegmentMetadata> nextSegmentMetadataOpt = Optional.of(segmentMetadata);
+        Optional<TransactionIndex> txnIndexOpt = nextSegmentMetadataOpt.map(metadata -> indexCache.getIndexEntry(metadata).txnIndex());
+
+        while (txnIndexOpt.isPresent()) {
+            TxnIndexSearchResult searchResult = txnIndexOpt.get().collectAbortedTxns(startOffset, upperBoundOffset);
+            accumulator.accept(searchResult.abortedTransactions);
+            if (!searchResult.isComplete) {
+                if (!searchInLocalLog) {
+                    nextSegmentMetadataOpt = findNextSegmentMetadata(nextSegmentMetadataOpt.get(), log);
+
+                    txnIndexOpt = nextSegmentMetadataOpt.map(x -> indexCache.getIndexEntry(x).txnIndex());
+                    if (!txnIndexOpt.isPresent()) {
+                        searchInLocalLog = true;
+                    }
+                }
+
+                if (searchInLocalLog) {
+                    txnIndexOpt = (localLogSegments.hasNext()) ? Optional.of(localLogSegments.next().txnIndex()) : Optional.empty();
                 }
+            } else {
+                return;
+            }
+        }
+    }
+
+    private Optional<RemoteLogSegmentMetadata> findNextSegmentMetadata(RemoteLogSegmentMetadata segmentMetadata, UnifiedLog log) throws RemoteStorageException {
+        Option<LeaderEpochFileCache> leaderEpochFileCacheOption = log.leaderEpochCache();
+        if (leaderEpochFileCacheOption.isEmpty()) {
+            return Optional.empty();
+        }
+
+        TopicPartition topicPartition = segmentMetadata.topicIdPartition().topicPartition();
+        long nextSegmentBaseOffset = segmentMetadata.endOffset() + 1;
+        OptionalInt epoch = OptionalInt.of(segmentMetadata.segmentLeaderEpochs().lastEntry().getKey());

Review Comment:
   It initially tries to find a segment for that epoch and the offset. If that is not found, it gets the next available epoch by using `leaderEpochFileCache.nextEpoch(epoch.getAsInt())` and tries to find a segment for that epoch. It eventually tries to find it by going through the epoch sequence like below.
   
   ```
   while (!result.isPresent() && epoch.isPresent()) {
               result = fetchRemoteLogSegmentMetadata(topicPartition, epoch.getAsInt(), nextSegmentBaseOffset);
               epoch = leaderEpochFileCache.nextEpoch(epoch.getAsInt());
           }
   ``` 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1171246205


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,6 +622,176 @@ public String toString() {
         }
     }
 
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadata = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadata.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        int startPos = lookupPositionForOffset(rlsMetadata.get(), offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(rlsMetadata.get(), startPos);

Review Comment:
   We will look into it in a followup PR. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] Hangleton commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1180448490


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,25 +622,208 @@ public String toString() {
         }
     }
 
-    long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws RemoteStorageException {
-        Optional<Long> offset = Optional.empty();
-        Optional<UnifiedLog> maybeLog = fetchLog.apply(topicIdPartition.topicPartition());
-        if (maybeLog.isPresent()) {
-            UnifiedLog log = maybeLog.get();
-            Option<LeaderEpochFileCache> maybeLeaderEpochFileCache = log.leaderEpochCache();
-            if (maybeLeaderEpochFileCache.isDefined()) {
-                LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get();
-                OptionalInt epoch = cache.latestEpoch();
-                while (!offset.isPresent() && epoch.isPresent()) {
-                    offset = remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch.getAsInt());
-                    epoch = cache.previousEpoch(epoch.getAsInt());
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadataOptional = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadataOptional.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";

Review Comment:
   nit: instead of `NOT_AVAILABLE`, maybe the message could report that the log start offset is strictly greater than the fetch offset?



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1118,9 +1122,13 @@ class ReplicaManager(val config: KafkaConfig,
     responseCallback: Seq[(TopicIdPartition, FetchPartitionData)] => Unit
   ): Unit = {
     // check if this fetch request can be satisfied right away
-    val logReadResults = readFromLocalLog(params, fetchInfos, quota, readFromPurgatory = false)
+    val logReadResults = readFromLog(params, fetchInfos, quota, readFromPurgatory = false)
     var bytesReadable: Long = 0
     var errorReadingData = false
+
+    // The 1st topic-partition that has to be read from remote storage
+    var remoteFetchInfo: Optional[RemoteStorageFetchInfo] = Optional.empty()

Review Comment:
   Got it, thanks.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,6 +622,176 @@ public String toString() {
         }
     }
 
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadata = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadata.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        int startPos = lookupPositionForOffset(rlsMetadata.get(), offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(rlsMetadata.get(), startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            int updatedFetchSize =
+                    remoteStorageFetchInfo.minOneMessage && firstBatch.sizeInBytes() > maxBytes
+                            ? firstBatch.sizeInBytes() : maxBytes;
+
+            ByteBuffer buffer = ByteBuffer.allocate(updatedFetchSize);
+            int remainingBytes = updatedFetchSize;
+
+            firstBatch.writeTo(buffer);
+            remainingBytes -= firstBatch.sizeInBytes();
+
+            if (remainingBytes > 0) {
+                // input stream is read till (startPos - 1) while getting the batch of records earlier.
+                // read the input stream until min of (EOF stream or buffer's remaining capacity).
+                Utils.readFully(remoteSegInputStream, buffer);
+            }
+            buffer.flip();
+
+            FetchDataInfo fetchDataInfo = new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.readableRecords(buffer));
+            if (includeAbortedTxns) {
+                fetchDataInfo = addAbortedTransactions(firstBatch.baseOffset(), rlsMetadata.get(), fetchDataInfo);
+            }
+
+            return fetchDataInfo;
+        } finally {
+            Utils.closeQuietly(remoteSegInputStream, "RemoteLogSegmentInputStream");
+        }
+    }
+
+    private int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        return indexCache.lookupOffset(remoteLogSegmentMetadata, offset);
+    }
+
+    private FetchDataInfo addAbortedTransactions(long startOffset,
+                                         RemoteLogSegmentMetadata segmentMetadata,
+                                         FetchDataInfo fetchInfo) throws RemoteStorageException {
+        int fetchSize = fetchInfo.records.sizeInBytes();
+        OffsetPosition startOffsetPosition = new OffsetPosition(fetchInfo.fetchOffsetMetadata.messageOffset,
+                fetchInfo.fetchOffsetMetadata.relativePositionInSegment);
+
+        OffsetIndex offsetIndex = indexCache.getIndexEntry(segmentMetadata).offsetIndex();
+        long upperBoundOffset = offsetIndex.fetchUpperBoundOffset(startOffsetPosition, fetchSize)
+                .map(x -> x.offset).orElse(segmentMetadata.endOffset() + 1);
+
+        final List<FetchResponseData.AbortedTransaction> abortedTransactions = new ArrayList<>();
+
+        Consumer<List<AbortedTxn>> accumulator =
+                abortedTxns -> abortedTransactions.addAll(abortedTxns.stream()
+                        .map(AbortedTxn::asAbortedTransaction).collect(Collectors.toList()));
+
+        collectAbortedTransactions(startOffset, upperBoundOffset, segmentMetadata, accumulator);
+
+        return new FetchDataInfo(fetchInfo.fetchOffsetMetadata,
+                fetchInfo.records,
+                fetchInfo.firstEntryIncomplete,
+                Optional.of(abortedTransactions));
+    }
+
+    private void collectAbortedTransactions(long startOffset,
+                                    long upperBoundOffset,
+                                    RemoteLogSegmentMetadata segmentMetadata,
+                                    Consumer<List<AbortedTxn>> accumulator) throws RemoteStorageException {
+        TopicPartition topicPartition = segmentMetadata.topicIdPartition().topicPartition();
+        Iterator<LogSegment> localLogSegments = fetchLog.apply(topicPartition)
+                .map(log -> JavaConverters.asJavaCollection(log.logSegments()))
+                .map(Collection::iterator)
+                .orElse(Collections.emptyIterator());
+
+        boolean searchInLocalLog = false;
+        Optional<RemoteLogSegmentMetadata> nextSegmentMetadataOpt = Optional.of(segmentMetadata);
+        Optional<TransactionIndex> txnIndexOpt = nextSegmentMetadataOpt.map(metadata -> indexCache.getIndexEntry(metadata).txnIndex());
+
+        while (txnIndexOpt.isPresent()) {
+            TxnIndexSearchResult searchResult = txnIndexOpt.get().collectAbortedTxns(startOffset, upperBoundOffset);
+            accumulator.accept(searchResult.abortedTransactions);
+            if (!searchResult.isComplete) {
+                if (!searchInLocalLog) {
+                    nextSegmentMetadataOpt = findNextSegmentMetadata(nextSegmentMetadataOpt.get());
+
+                    txnIndexOpt = nextSegmentMetadataOpt.map(x -> indexCache.getIndexEntry(x).txnIndex());

Review Comment:
   Thanks, it makes sense to have it configurable. For the use case reported above, I think it's probably best to come up with a specific scenario and a test to reproduce. I can give it a shot in a couple of weeks, which will be after the PR is merged.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,25 +622,208 @@ public String toString() {
         }
     }
 
-    long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws RemoteStorageException {
-        Optional<Long> offset = Optional.empty();
-        Optional<UnifiedLog> maybeLog = fetchLog.apply(topicIdPartition.topicPartition());
-        if (maybeLog.isPresent()) {
-            UnifiedLog log = maybeLog.get();
-            Option<LeaderEpochFileCache> maybeLeaderEpochFileCache = log.leaderEpochCache();
-            if (maybeLeaderEpochFileCache.isDefined()) {
-                LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get();
-                OptionalInt epoch = cache.latestEpoch();
-                while (!offset.isPresent() && epoch.isPresent()) {
-                    offset = remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch.getAsInt());
-                    epoch = cache.previousEpoch(epoch.getAsInt());
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadataOptional = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadataOptional.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        RemoteLogSegmentMetadata remoteLogSegmentMetadata = rlsMetadataOptional.get();
+        int startPos = lookupPositionForOffset(remoteLogSegmentMetadata, offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            // An empty record is sent instead of an incomplete batch when there is no minimum-one-message constraint
+            // and for FetchRequest version 3 and above and the first batch size is more than maximum bytes that can be sent.
+            if (!remoteStorageFetchInfo.minOneMessage &&
+                    !remoteStorageFetchInfo.hardMaxBytesLimit &&

Review Comment:
   Since `remoteStorageFetchInfo.hardMaxBytesLimit` <=> Fetch request version <= 2, is there a possibility that Fetch requests v <= 2 are served from TS? If not, we can probably remove that condition on `hardMaxBytesLimit`. If yes, I think there may still be a risk of overflow for v <= 2.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] Hangleton commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1182493022


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1160,48 +1171,100 @@ class ReplicaManager(val config: KafkaConfig,
           fetchPartitionStatus += (topicIdPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
         })
       }
-      val delayedFetch = new DelayedFetch(
-        params = params,
-        fetchPartitionStatus = fetchPartitionStatus,
-        replicaManager = this,
-        quota = quota,
-        responseCallback = responseCallback
-      )
-
-      // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
-      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp) }
-
-      // try to complete the request immediately, otherwise put it into the purgatory;
-      // this is because while the delayed fetch operation is being created, new requests
-      // may arrive and hence make this operation completable.
-      delayedFetchPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys)
+
+      if (remoteFetchInfo.isPresent) {
+        val key = new TopicPartitionOperationKey(remoteFetchInfo.get.topicPartition.topic(), remoteFetchInfo.get.topicPartition.partition())
+        val remoteFetchResult = new CompletableFuture[RemoteLogReadResult]
+        var remoteFetchTask: Future[Void] = null
+        try {
+          remoteFetchTask = remoteLogManager.get.asyncRead(remoteFetchInfo.get, (result: RemoteLogReadResult) => {
+            remoteFetchResult.complete(result)
+            delayedRemoteFetchPurgatory.checkAndComplete(key)
+          })
+        } catch {
+          // if the task queue of remote storage reader thread pool is full, return what we currently have
+          // (the data read from local log segment for the other topic-partitions) and an error for the topic-partition that
+          // we couldn't read from remote storage
+          case e: RejectedExecutionException =>
+            val fetchPartitionData = logReadResults.map { case (tp, result) =>
+              val r = {
+                if (tp.topicPartition().equals(remoteFetchInfo.get.topicPartition))
+                  createLogReadResult(e)

Review Comment:
   Should we add a log if the nature of the error is not propagated?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] divijvaidya commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "divijvaidya (via GitHub)" <gi...@apache.org>.
divijvaidya commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1182279902


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1160,48 +1171,100 @@ class ReplicaManager(val config: KafkaConfig,
           fetchPartitionStatus += (topicIdPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
         })
       }
-      val delayedFetch = new DelayedFetch(
-        params = params,
-        fetchPartitionStatus = fetchPartitionStatus,
-        replicaManager = this,
-        quota = quota,
-        responseCallback = responseCallback
-      )
-
-      // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
-      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp) }
-
-      // try to complete the request immediately, otherwise put it into the purgatory;
-      // this is because while the delayed fetch operation is being created, new requests
-      // may arrive and hence make this operation completable.
-      delayedFetchPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys)
+
+      if (remoteFetchInfo.isPresent) {
+        val key = new TopicPartitionOperationKey(remoteFetchInfo.get.topicPartition.topic(), remoteFetchInfo.get.topicPartition.partition())
+        val remoteFetchResult = new CompletableFuture[RemoteLogReadResult]
+        var remoteFetchTask: Future[Void] = null
+        try {
+          remoteFetchTask = remoteLogManager.get.asyncRead(remoteFetchInfo.get, (result: RemoteLogReadResult) => {
+            remoteFetchResult.complete(result)
+            delayedRemoteFetchPurgatory.checkAndComplete(key)
+          })
+        } catch {
+          // if the task queue of remote storage reader thread pool is full, return what we currently have
+          // (the data read from local log segment for the other topic-partitions) and an error for the topic-partition that
+          // we couldn't read from remote storage
+          case e: RejectedExecutionException =>
+            val fetchPartitionData = logReadResults.map { case (tp, result) =>
+              val r = {
+                if (tp.topicPartition().equals(remoteFetchInfo.get.topicPartition))
+                  createLogReadResult(e)

Review Comment:
   Thank you. That answers my question.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] divijvaidya commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "divijvaidya (via GitHub)" <gi...@apache.org>.
divijvaidya commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1182326466


##########
core/src/main/scala/kafka/server/DelayedRemoteFetch.scala:
##########
@@ -0,0 +1,112 @@
+/**
+ * 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 org.apache.kafka.common.TopicIdPartition
+import org.apache.kafka.common.errors._
+import org.apache.kafka.storage.internals.log.{FetchParams, FetchPartitionData, LogOffsetMetadata, RemoteLogReadResult, RemoteStorageFetchInfo}
+
+import java.util.concurrent.{CompletableFuture, Future}
+import java.util.{Optional, OptionalInt, OptionalLong}
+import scala.collection._
+
+/**
+ * A remote fetch operation that can be created by the replica manager and watched
+ * in the remote fetch operation purgatory
+ */
+class DelayedRemoteFetch(remoteFetchTask: Future[Void],
+                         remoteFetchResult: CompletableFuture[RemoteLogReadResult],
+                         remoteFetchInfo: RemoteStorageFetchInfo,
+                         fetchPartitionStatus: Seq[(TopicIdPartition, FetchPartitionStatus)],
+                         fetchParams: FetchParams,
+                         localReadResults: Seq[(TopicIdPartition, LogReadResult)],
+                         replicaManager: ReplicaManager,
+                         responseCallback: Seq[(TopicIdPartition, FetchPartitionData)] => Unit)
+  extends DelayedOperation(fetchParams.maxWaitMs) {
+
+  /**
+   * The operation can be completed if:
+   *
+   * Case a: This broker is no longer the leader of the partition it tries to fetch
+   * Case b: This broker does not know the partition it tries to fetch
+   * Case c: The remote storage read request completed (succeeded or failed)
+   * Case d: The partition is in an offline log directory on this broker
+   *
+   * Upon completion, should return whatever data is available for each valid partition
+   */
+  override def tryComplete(): Boolean = {
+    fetchPartitionStatus.foreach {
+      case (topicPartition, fetchStatus) =>
+        val fetchOffset = fetchStatus.startOffsetMetadata
+        try {
+          if (fetchOffset != LogOffsetMetadata.UNKNOWN_OFFSET_METADATA) {
+            replicaManager.getPartitionOrException(topicPartition.topicPartition())
+          }
+        } catch {
+          case _: KafkaStorageException => // Case d
+            debug(s"Partition $topicPartition is in an offline log directory, satisfy $fetchParams immediately")
+            return forceComplete()
+          case _: UnknownTopicOrPartitionException => // Case b
+            debug(s"Broker no longer knows of partition $topicPartition, satisfy $fetchParams immediately")
+            return forceComplete()
+          case _: NotLeaderOrFollowerException =>  // Case a
+            debug("Broker is no longer the leader or follower of %s, satisfy %s immediately".format(topicPartition, fetchParams))
+            return forceComplete()
+        }
+    }
+    if (remoteFetchResult.isDone) // Case c
+      forceComplete()
+    else
+      false
+  }
+
+  override def onExpiration():Unit = {

Review Comment:
   Thanks, I added a JIRA https://issues.apache.org/jira/browse/KAFKA-14953 to ensure that we don't forget this and are tracking it somewhere.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1192210616


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,25 +623,210 @@ public String toString() {
         }
     }
 
-    long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws RemoteStorageException {
-        Optional<Long> offset = Optional.empty();
-        Optional<UnifiedLog> maybeLog = fetchLog.apply(topicIdPartition.topicPartition());
-        if (maybeLog.isPresent()) {
-            UnifiedLog log = maybeLog.get();
-            Option<LeaderEpochFileCache> maybeLeaderEpochFileCache = log.leaderEpochCache();
-            if (maybeLeaderEpochFileCache.isDefined()) {
-                LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get();
-                OptionalInt epoch = cache.latestEpoch();
-                while (!offset.isPresent() && epoch.isPresent()) {
-                    offset = remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch.getAsInt());
-                    epoch = cache.previousEpoch(epoch.getAsInt());
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadataOptional = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadataOptional.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        RemoteLogSegmentMetadata remoteLogSegmentMetadata = rlsMetadataOptional.get();
+        int startPos = lookupPositionForOffset(remoteLogSegmentMetadata, offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            int firstBatchSize = firstBatch.sizeInBytes();
+            // An empty record is sent instead of an incomplete batch when
+            //  - there is no minimum-one-message constraint and
+            //  - the first batch size is more than maximum bytes that can be sent.
+            //  - for FetchRequest version 3 or above and
+            if (!remoteStorageFetchInfo.minOneMessage &&
+                    !remoteStorageFetchInfo.hardMaxBytesLimit &&
+                    firstBatchSize > maxBytes) {
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY);
+            }
+
+            int updatedFetchSize =
+                    remoteStorageFetchInfo.minOneMessage && firstBatchSize > maxBytes ? firstBatchSize : maxBytes;
+
+            ByteBuffer buffer = ByteBuffer.allocate(updatedFetchSize);
+            int remainingBytes = updatedFetchSize;
+
+            firstBatch.writeTo(buffer);
+            remainingBytes -= firstBatchSize;
+
+            if (remainingBytes > 0) {
+                // read the input stream until min of (EOF stream or buffer's remaining capacity).
+                Utils.readFully(remoteSegInputStream, buffer);
+            }
+            buffer.flip();
+
+            FetchDataInfo fetchDataInfo = new FetchDataInfo(
+                    new LogOffsetMetadata(offset, remoteLogSegmentMetadata.startOffset(), startPos),
+                    MemoryRecords.readableRecords(buffer));
+            if (includeAbortedTxns) {
+                fetchDataInfo = addAbortedTransactions(firstBatch.baseOffset(), remoteLogSegmentMetadata, fetchDataInfo, logOptional.get());
+            }
+
+            return fetchDataInfo;
+        } finally {
+            Utils.closeQuietly(remoteSegInputStream, "RemoteLogSegmentInputStream");
+        }
+    }
+
+    private int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        return indexCache.lookupOffset(remoteLogSegmentMetadata, offset);
+    }
+
+    private FetchDataInfo addAbortedTransactions(long startOffset,

Review Comment:
   We have the below test scenarios in 2.8.x in Scala. We plan to raise these in a followup PR as it requires a rewrite in Java and refactor from EasyMock to Mockito.
   
   testAddAbortedTransactions
   testCollectAbortedTransactionsIteratesNextRemoteSegment
   testCollectAbortedTransactionsIteratesNextLocalSegment



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on PR #13535:
URL: https://github.com/apache/kafka/pull/13535#issuecomment-1557543152

   @satishd I see many failed tests [here](https://ci-builds.apache.org/blue/organizations/jenkins/Kafka%2Fkafka-pr/detail/PR-13535/33/tests). Are they related to changes made in this PR?


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1184873142


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -670,6 +875,14 @@ public void close() {
                 } catch (InterruptedException e) {
                     // ignore
                 }
+                remoteStorageReaderThreadPool.shutdownNow();
+                //waits for 2 mins to terminate the current tasks
+                try {
+                    remoteStorageReaderThreadPool.awaitTermination(2, TimeUnit.MINUTES);

Review Comment:
   It does not require that to be completed in 5 mins. `lifecycleManager.controlledShutdownFuture` is more about processing the controlled shutdown event to the controller for that broker. It will wait for 5 mins before proceeding with other sequence of actions. But that will not get affected because of the code introduced here. 
   Logging subsystem handles unclean shutdown for log segments and it would have been already finished before RemoteLogManager is closed. So, they will not get affected because of this timeout. But we can have a short duration here like 10 secs, we can revisit introducing a config if it is really needed for closing the remote log subsystem.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1194768781


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,25 +623,204 @@ public String toString() {
         }
     }
 
-    long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws RemoteStorageException {
-        Optional<Long> offset = Optional.empty();
-        Optional<UnifiedLog> maybeLog = fetchLog.apply(topicIdPartition.topicPartition());
-        if (maybeLog.isPresent()) {
-            UnifiedLog log = maybeLog.get();
-            Option<LeaderEpochFileCache> maybeLeaderEpochFileCache = log.leaderEpochCache();
-            if (maybeLeaderEpochFileCache.isDefined()) {
-                LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get();
-                OptionalInt epoch = cache.latestEpoch();
-                while (!offset.isPresent() && epoch.isPresent()) {
-                    offset = remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch.getAsInt());
-                    epoch = cache.previousEpoch(epoch.getAsInt());
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadataOptional = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadataOptional.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        RemoteLogSegmentMetadata remoteLogSegmentMetadata = rlsMetadataOptional.get();
+        int startPos = lookupPositionForOffset(remoteLogSegmentMetadata, offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            int firstBatchSize = firstBatch.sizeInBytes();
+            // An empty record is sent instead of an incomplete batch when
+            //  - there is no minimum-one-message constraint and
+            //  - the first batch size is more than maximum bytes that can be sent and
+            //  - for FetchRequest version 3 or above.
+            if (!remoteStorageFetchInfo.minOneMessage &&
+                    !remoteStorageFetchInfo.hardMaxBytesLimit &&
+                    firstBatchSize > maxBytes) {
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY);
+            }
+
+            int updatedFetchSize =
+                    remoteStorageFetchInfo.minOneMessage && firstBatchSize > maxBytes ? firstBatchSize : maxBytes;
+
+            ByteBuffer buffer = ByteBuffer.allocate(updatedFetchSize);
+            int remainingBytes = updatedFetchSize;
+
+            firstBatch.writeTo(buffer);
+            remainingBytes -= firstBatchSize;
+
+            if (remainingBytes > 0) {
+                // read the input stream until min of (EOF stream or buffer's remaining capacity).
+                Utils.readFully(remoteSegInputStream, buffer);
+            }
+            buffer.flip();
+
+            FetchDataInfo fetchDataInfo = new FetchDataInfo(
+                    new LogOffsetMetadata(offset, remoteLogSegmentMetadata.startOffset(), startPos),
+                    MemoryRecords.readableRecords(buffer));
+            if (includeAbortedTxns) {
+                fetchDataInfo = addAbortedTransactions(firstBatch.baseOffset(), remoteLogSegmentMetadata, fetchDataInfo, logOptional.get());
+            }
+
+            return fetchDataInfo;
+        } finally {
+            Utils.closeQuietly(remoteSegInputStream, "RemoteLogSegmentInputStream");
+        }
+    }
+
+    private int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        return indexCache.lookupOffset(remoteLogSegmentMetadata, offset);
+    }
+
+    private FetchDataInfo addAbortedTransactions(long startOffset,
+                                                 RemoteLogSegmentMetadata segmentMetadata,
+                                                 FetchDataInfo fetchInfo,
+                                                 UnifiedLog log) throws RemoteStorageException {
+        int fetchSize = fetchInfo.records.sizeInBytes();
+        OffsetPosition startOffsetPosition = new OffsetPosition(fetchInfo.fetchOffsetMetadata.messageOffset,
+                fetchInfo.fetchOffsetMetadata.relativePositionInSegment);
+
+        OffsetIndex offsetIndex = indexCache.getIndexEntry(segmentMetadata).offsetIndex();
+        long upperBoundOffset = offsetIndex.fetchUpperBoundOffset(startOffsetPosition, fetchSize)
+                .map(x -> x.offset).orElse(segmentMetadata.endOffset() + 1);
+
+        final Set<FetchResponseData.AbortedTransaction> abortedTransactions = new HashSet<>();
+
+        Consumer<List<AbortedTxn>> accumulator =
+                abortedTxns -> abortedTransactions.addAll(abortedTxns.stream()
+                        .map(AbortedTxn::asAbortedTransaction).collect(Collectors.toList()));
+
+        collectAbortedTransactions(startOffset, upperBoundOffset, segmentMetadata, accumulator, log);
+
+        return new FetchDataInfo(fetchInfo.fetchOffsetMetadata,
+                fetchInfo.records,
+                fetchInfo.firstEntryIncomplete,
+                Optional.of(abortedTransactions.isEmpty() ? Collections.emptyList() : new ArrayList<>(abortedTransactions)));
+    }
+
+    private void collectAbortedTransactions(long startOffset,
+                                            long upperBoundOffset,
+                                            RemoteLogSegmentMetadata segmentMetadata,
+                                            Consumer<List<AbortedTxn>> accumulator,
+                                            UnifiedLog log) throws RemoteStorageException {
+        Iterator<LogSegment> localLogSegments = JavaConverters.asJavaIterator(log.logSegments().iterator());
+
+        boolean searchInLocalLog = false;
+        Optional<RemoteLogSegmentMetadata> nextSegmentMetadataOpt = Optional.of(segmentMetadata);
+        Optional<TransactionIndex> txnIndexOpt = nextSegmentMetadataOpt.map(metadata -> indexCache.getIndexEntry(metadata).txnIndex());

Review Comment:
   This is answered in [JIRA](https://issues.apache.org/jira/browse/KAFKA-7739?focusedCommentId=17722119&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17722119).



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] junrao commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "junrao (via GitHub)" <gi...@apache.org>.
junrao commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1187994355


##########
core/src/test/scala/integration/kafka/server/DelayedRemoteFetchTest.scala:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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 kafka.cluster.Partition
+import org.apache.kafka.common.errors.NotLeaderOrFollowerException
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.record.MemoryRecords
+import org.apache.kafka.common.requests.FetchRequest
+import org.apache.kafka.common.{TopicIdPartition, Uuid}
+import org.apache.kafka.storage.internals.log._
+import org.junit.jupiter.api.Assertions._
+import org.junit.jupiter.api.Test
+import org.mockito.Mockito.{mock, when}
+
+import java.util.Optional
+import java.util.concurrent.CompletableFuture
+
+import scala.collection._
+
+class DelayedRemoteFetchTest {
+  private val maxBytes = 1024
+  private val replicaManager: ReplicaManager = mock(classOf[ReplicaManager])
+  private val topicIdPartition = new TopicIdPartition(Uuid.randomUuid(), 0, "topic")
+  private val fetchOffset = 500L
+  private val logStartOffset = 0L
+  private val currentLeaderEpoch = Optional.of[Integer](10)
+  private val replicaId = 1
+
+  private val fetchStatus = FetchPartitionStatus(
+    startOffsetMetadata = new LogOffsetMetadata(fetchOffset),
+    fetchInfo = new FetchRequest.PartitionData(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch))
+  private val fetchParams = buildFollowerFetchParams(replicaId, maxWaitMs = 500)
+
+  @Test
+  def testFetchWithFencedEpoch(): Unit = {

Review Comment:
   Hmm, where is logic to simulate a fenced epoch in this test?



##########
core/src/test/java/kafka/log/remote/RemoteLogReaderTest.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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 org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.OffsetOutOfRangeException;
+import org.apache.kafka.common.record.Records;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.storage.internals.log.FetchDataInfo;
+import org.apache.kafka.storage.internals.log.LogOffsetMetadata;
+import org.apache.kafka.storage.internals.log.RemoteLogReadResult;
+import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo;
+import org.junit.jupiter.api.Test;
+import org.mockito.ArgumentCaptor;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+
+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.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class RemoteLogReaderTest {
+    RemoteLogManager mockRLM = mock(RemoteLogManager.class);
+    LogOffsetMetadata logOffsetMetadata = new LogOffsetMetadata(100);
+    Records records = mock(Records.class);
+
+

Review Comment:
   extra new line.



##########
core/src/test/scala/integration/kafka/server/DelayedRemoteFetchTest.scala:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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 kafka.cluster.Partition
+import org.apache.kafka.common.errors.NotLeaderOrFollowerException
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.record.MemoryRecords
+import org.apache.kafka.common.requests.FetchRequest
+import org.apache.kafka.common.{TopicIdPartition, Uuid}
+import org.apache.kafka.storage.internals.log._
+import org.junit.jupiter.api.Assertions._
+import org.junit.jupiter.api.Test
+import org.mockito.Mockito.{mock, when}
+
+import java.util.Optional
+import java.util.concurrent.CompletableFuture
+
+import scala.collection._
+
+class DelayedRemoteFetchTest {
+  private val maxBytes = 1024
+  private val replicaManager: ReplicaManager = mock(classOf[ReplicaManager])
+  private val topicIdPartition = new TopicIdPartition(Uuid.randomUuid(), 0, "topic")
+  private val fetchOffset = 500L
+  private val logStartOffset = 0L
+  private val currentLeaderEpoch = Optional.of[Integer](10)
+  private val replicaId = 1
+
+  private val fetchStatus = FetchPartitionStatus(
+    startOffsetMetadata = new LogOffsetMetadata(fetchOffset),
+    fetchInfo = new FetchRequest.PartitionData(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch))
+  private val fetchParams = buildFollowerFetchParams(replicaId, maxWaitMs = 500)
+
+  @Test
+  def testFetchWithFencedEpoch(): Unit = {
+    var actualTopicPartition: Option[TopicIdPartition] = None
+    var fetchResultOpt: Option[FetchPartitionData] = None
+
+    def callback(responses: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = {
+      assertEquals(1, responses.size)
+      actualTopicPartition = Some(responses.head._1)
+      fetchResultOpt = Some(responses.head._2)
+    }
+
+    val future: CompletableFuture[RemoteLogReadResult] = new CompletableFuture[RemoteLogReadResult]()
+    future.complete(null)
+    val fetchInfo: RemoteStorageFetchInfo = new RemoteStorageFetchInfo(0, false, topicIdPartition.topicPartition(), null, null, false)
+    val highWatermark = 100
+    val leaderLogStartOffset = 10
+    val logReadInfo = buildReadResult(Errors.NONE, highWatermark, leaderLogStartOffset)
+
+    val delayedRemoteFetch = new DelayedRemoteFetch(null, future, fetchInfo, Seq(topicIdPartition -> fetchStatus), fetchParams,
+      Seq(topicIdPartition -> logReadInfo), replicaManager, callback)
+
+    when(replicaManager.getPartitionOrException(topicIdPartition.topicPartition))
+      .thenReturn(mock(classOf[Partition]))
+
+    assertTrue(delayedRemoteFetch.tryComplete())
+    assertTrue(delayedRemoteFetch.isCompleted)
+    assertTrue(actualTopicPartition.isDefined)
+    assertEquals(topicIdPartition, actualTopicPartition.get)
+    assertTrue(fetchResultOpt.isDefined)
+
+    val fetchResult = fetchResultOpt.get
+    assertEquals(Errors.NONE, fetchResult.error)
+    assertEquals(highWatermark, fetchResult.highWatermark)
+    assertEquals(leaderLogStartOffset, fetchResult.logStartOffset)
+  }
+
+  @Test
+  def testNotLeaderOrFollower(): Unit = {
+    var actualTopicPartition: Option[TopicIdPartition] = None
+    var fetchResultOpt: Option[FetchPartitionData] = None
+
+    def callback(responses: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = {
+      assertEquals(1, responses.size)
+      actualTopicPartition = Some(responses.head._1)
+      fetchResultOpt = Some(responses.head._2)
+    }
+
+    // throw exception while getPartition
+    when(replicaManager.getPartitionOrException(topicIdPartition.topicPartition))
+      .thenThrow(new NotLeaderOrFollowerException(s"Replica for $topicIdPartition not available"))
+
+    val future: CompletableFuture[RemoteLogReadResult] = new CompletableFuture[RemoteLogReadResult]()
+    val fetchInfo: RemoteStorageFetchInfo = new RemoteStorageFetchInfo(0, false, topicIdPartition.topicPartition(), null, null, false)
+
+    val logReadInfo = buildReadResult(Errors.NONE)
+
+    val delayedRemoteFetch = new DelayedRemoteFetch(null, future, fetchInfo, Seq(topicIdPartition -> fetchStatus), fetchParams,
+      Seq(topicIdPartition -> logReadInfo), replicaManager, callback)
+
+    // delayed remote fetch should still be able to complete
+    assertTrue(delayedRemoteFetch.tryComplete())
+    assertTrue(delayedRemoteFetch.isCompleted)
+    assertEquals(topicIdPartition, actualTopicPartition.get)
+    assertTrue(fetchResultOpt.isDefined)
+    assertTrue(fetchResultOpt.isDefined)
+  }
+
+  @Test
+  def testErrorLogReadInfo(): Unit = {
+    var actualTopicPartition: Option[TopicIdPartition] = None
+    var fetchResultOpt: Option[FetchPartitionData] = None
+
+    def callback(responses: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = {
+      assertEquals(1, responses.size)
+      actualTopicPartition = Some(responses.head._1)
+      fetchResultOpt = Some(responses.head._2)
+    }
+
+    when(replicaManager.getPartitionOrException(topicIdPartition.topicPartition))
+      .thenReturn(mock(classOf[Partition]))
+
+    val future: CompletableFuture[RemoteLogReadResult] = new CompletableFuture[RemoteLogReadResult]()
+    future.complete(null)
+    val fetchInfo: RemoteStorageFetchInfo = new RemoteStorageFetchInfo(0, false, topicIdPartition.topicPartition(), null, null, false)
+
+    // build a read result with error
+    val logReadInfo = buildReadResult(Errors.FENCED_LEADER_EPOCH)
+
+    val delayedRemoteFetch = new DelayedRemoteFetch(null, future, fetchInfo, Seq(topicIdPartition -> fetchStatus), fetchParams,
+      Seq(topicIdPartition -> logReadInfo), replicaManager, callback)
+
+    assertTrue(delayedRemoteFetch.tryComplete())
+    assertTrue(delayedRemoteFetch.isCompleted)
+    assertEquals(topicIdPartition, actualTopicPartition.get)
+    assertTrue(fetchResultOpt.isDefined)
+    assertTrue(fetchResultOpt.isDefined)

Review Comment:
   This line duplicates the line before.



##########
core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala:
##########
@@ -156,8 +156,7 @@ object AbstractCoordinatorConcurrencyTest {
   trait CoordinatorMember {
   }
 
-  class TestReplicaManager extends ReplicaManager(
-    null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, None, null) {
+  class TestReplicaManager extends ReplicaManager(null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, None, null) {

Review Comment:
   Do we need to change this since it makes the line quite long?



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1153,14 +1193,15 @@ class ReplicaManager(val config: KafkaConfig,
       logReadResultMap.put(topicIdPartition, logReadResult)
     }
 
-    // respond immediately if 1) fetch request does not want to wait
+    // Respond immediately if 1) no remote fetches are required
     //                        2) fetch request does not require any data
-    //                        3) has enough data to respond
-    //                        4) some error happens while reading data
-    //                        5) we found a diverging epoch
-    //                        6) has a preferred read replica
-    if (params.maxWaitMs <= 0 || fetchInfos.isEmpty || bytesReadable >= params.minBytes || errorReadingData ||
-      hasDivergingEpoch || hasPreferredReadReplica) {
+    //                        3) fetch request does not require any data

Review Comment:
   `2)` should be fetch request does not want to wait. Also, the comment is written as if `1)` is also disjunctive, but it is not.



##########
core/src/test/scala/integration/kafka/server/DelayedRemoteFetchTest.scala:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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 kafka.cluster.Partition
+import org.apache.kafka.common.errors.NotLeaderOrFollowerException
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.record.MemoryRecords
+import org.apache.kafka.common.requests.FetchRequest
+import org.apache.kafka.common.{TopicIdPartition, Uuid}
+import org.apache.kafka.storage.internals.log._
+import org.junit.jupiter.api.Assertions._
+import org.junit.jupiter.api.Test
+import org.mockito.Mockito.{mock, when}
+
+import java.util.Optional
+import java.util.concurrent.CompletableFuture
+
+import scala.collection._
+
+class DelayedRemoteFetchTest {
+  private val maxBytes = 1024
+  private val replicaManager: ReplicaManager = mock(classOf[ReplicaManager])
+  private val topicIdPartition = new TopicIdPartition(Uuid.randomUuid(), 0, "topic")
+  private val fetchOffset = 500L
+  private val logStartOffset = 0L
+  private val currentLeaderEpoch = Optional.of[Integer](10)
+  private val replicaId = 1
+
+  private val fetchStatus = FetchPartitionStatus(
+    startOffsetMetadata = new LogOffsetMetadata(fetchOffset),
+    fetchInfo = new FetchRequest.PartitionData(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch))
+  private val fetchParams = buildFollowerFetchParams(replicaId, maxWaitMs = 500)
+
+  @Test
+  def testFetchWithFencedEpoch(): Unit = {
+    var actualTopicPartition: Option[TopicIdPartition] = None
+    var fetchResultOpt: Option[FetchPartitionData] = None
+
+    def callback(responses: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = {
+      assertEquals(1, responses.size)
+      actualTopicPartition = Some(responses.head._1)
+      fetchResultOpt = Some(responses.head._2)
+    }
+
+    val future: CompletableFuture[RemoteLogReadResult] = new CompletableFuture[RemoteLogReadResult]()
+    future.complete(null)
+    val fetchInfo: RemoteStorageFetchInfo = new RemoteStorageFetchInfo(0, false, topicIdPartition.topicPartition(), null, null, false)
+    val highWatermark = 100
+    val leaderLogStartOffset = 10
+    val logReadInfo = buildReadResult(Errors.NONE, highWatermark, leaderLogStartOffset)
+
+    val delayedRemoteFetch = new DelayedRemoteFetch(null, future, fetchInfo, Seq(topicIdPartition -> fetchStatus), fetchParams,
+      Seq(topicIdPartition -> logReadInfo), replicaManager, callback)
+
+    when(replicaManager.getPartitionOrException(topicIdPartition.topicPartition))
+      .thenReturn(mock(classOf[Partition]))
+
+    assertTrue(delayedRemoteFetch.tryComplete())
+    assertTrue(delayedRemoteFetch.isCompleted)
+    assertTrue(actualTopicPartition.isDefined)
+    assertEquals(topicIdPartition, actualTopicPartition.get)
+    assertTrue(fetchResultOpt.isDefined)
+
+    val fetchResult = fetchResultOpt.get
+    assertEquals(Errors.NONE, fetchResult.error)
+    assertEquals(highWatermark, fetchResult.highWatermark)
+    assertEquals(leaderLogStartOffset, fetchResult.logStartOffset)
+  }
+
+  @Test
+  def testNotLeaderOrFollower(): Unit = {
+    var actualTopicPartition: Option[TopicIdPartition] = None
+    var fetchResultOpt: Option[FetchPartitionData] = None
+
+    def callback(responses: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = {
+      assertEquals(1, responses.size)
+      actualTopicPartition = Some(responses.head._1)
+      fetchResultOpt = Some(responses.head._2)
+    }
+
+    // throw exception while getPartition
+    when(replicaManager.getPartitionOrException(topicIdPartition.topicPartition))
+      .thenThrow(new NotLeaderOrFollowerException(s"Replica for $topicIdPartition not available"))
+
+    val future: CompletableFuture[RemoteLogReadResult] = new CompletableFuture[RemoteLogReadResult]()
+    val fetchInfo: RemoteStorageFetchInfo = new RemoteStorageFetchInfo(0, false, topicIdPartition.topicPartition(), null, null, false)
+
+    val logReadInfo = buildReadResult(Errors.NONE)
+
+    val delayedRemoteFetch = new DelayedRemoteFetch(null, future, fetchInfo, Seq(topicIdPartition -> fetchStatus), fetchParams,
+      Seq(topicIdPartition -> logReadInfo), replicaManager, callback)
+
+    // delayed remote fetch should still be able to complete
+    assertTrue(delayedRemoteFetch.tryComplete())
+    assertTrue(delayedRemoteFetch.isCompleted)
+    assertEquals(topicIdPartition, actualTopicPartition.get)
+    assertTrue(fetchResultOpt.isDefined)
+    assertTrue(fetchResultOpt.isDefined)

Review Comment:
   This line duplicates the line before.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,25 +623,210 @@ public String toString() {
         }
     }
 
-    long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws RemoteStorageException {
-        Optional<Long> offset = Optional.empty();
-        Optional<UnifiedLog> maybeLog = fetchLog.apply(topicIdPartition.topicPartition());
-        if (maybeLog.isPresent()) {
-            UnifiedLog log = maybeLog.get();
-            Option<LeaderEpochFileCache> maybeLeaderEpochFileCache = log.leaderEpochCache();
-            if (maybeLeaderEpochFileCache.isDefined()) {
-                LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get();
-                OptionalInt epoch = cache.latestEpoch();
-                while (!offset.isPresent() && epoch.isPresent()) {
-                    offset = remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch.getAsInt());
-                    epoch = cache.previousEpoch(epoch.getAsInt());
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadataOptional = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadataOptional.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        RemoteLogSegmentMetadata remoteLogSegmentMetadata = rlsMetadataOptional.get();
+        int startPos = lookupPositionForOffset(remoteLogSegmentMetadata, offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            int firstBatchSize = firstBatch.sizeInBytes();
+            // An empty record is sent instead of an incomplete batch when
+            //  - there is no minimum-one-message constraint and
+            //  - the first batch size is more than maximum bytes that can be sent.
+            //  - for FetchRequest version 3 or above and

Review Comment:
   remove "and" at the end



##########
core/src/test/scala/integration/kafka/server/DelayedRemoteFetchTest.scala:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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 kafka.cluster.Partition
+import org.apache.kafka.common.errors.NotLeaderOrFollowerException
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.record.MemoryRecords
+import org.apache.kafka.common.requests.FetchRequest
+import org.apache.kafka.common.{TopicIdPartition, Uuid}
+import org.apache.kafka.storage.internals.log._
+import org.junit.jupiter.api.Assertions._
+import org.junit.jupiter.api.Test
+import org.mockito.Mockito.{mock, when}
+
+import java.util.Optional
+import java.util.concurrent.CompletableFuture
+
+import scala.collection._

Review Comment:
   Could we write the new test in java?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] Hangleton commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1188387671


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,25 +623,210 @@ public String toString() {
         }
     }
 
-    long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws RemoteStorageException {
-        Optional<Long> offset = Optional.empty();
-        Optional<UnifiedLog> maybeLog = fetchLog.apply(topicIdPartition.topicPartition());
-        if (maybeLog.isPresent()) {
-            UnifiedLog log = maybeLog.get();
-            Option<LeaderEpochFileCache> maybeLeaderEpochFileCache = log.leaderEpochCache();
-            if (maybeLeaderEpochFileCache.isDefined()) {
-                LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get();
-                OptionalInt epoch = cache.latestEpoch();
-                while (!offset.isPresent() && epoch.isPresent()) {
-                    offset = remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch.getAsInt());
-                    epoch = cache.previousEpoch(epoch.getAsInt());
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadataOptional = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadataOptional.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        RemoteLogSegmentMetadata remoteLogSegmentMetadata = rlsMetadataOptional.get();
+        int startPos = lookupPositionForOffset(remoteLogSegmentMetadata, offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            int firstBatchSize = firstBatch.sizeInBytes();
+            // An empty record is sent instead of an incomplete batch when
+            //  - there is no minimum-one-message constraint and
+            //  - the first batch size is more than maximum bytes that can be sent.
+            //  - for FetchRequest version 3 or above and
+            if (!remoteStorageFetchInfo.minOneMessage &&
+                    !remoteStorageFetchInfo.hardMaxBytesLimit &&
+                    firstBatchSize > maxBytes) {
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY);
+            }
+
+            int updatedFetchSize =
+                    remoteStorageFetchInfo.minOneMessage && firstBatchSize > maxBytes ? firstBatchSize : maxBytes;
+
+            ByteBuffer buffer = ByteBuffer.allocate(updatedFetchSize);
+            int remainingBytes = updatedFetchSize;
+
+            firstBatch.writeTo(buffer);
+            remainingBytes -= firstBatchSize;
+
+            if (remainingBytes > 0) {
+                // read the input stream until min of (EOF stream or buffer's remaining capacity).
+                Utils.readFully(remoteSegInputStream, buffer);
+            }
+            buffer.flip();
+
+            FetchDataInfo fetchDataInfo = new FetchDataInfo(
+                    new LogOffsetMetadata(offset, remoteLogSegmentMetadata.startOffset(), startPos),
+                    MemoryRecords.readableRecords(buffer));
+            if (includeAbortedTxns) {
+                fetchDataInfo = addAbortedTransactions(firstBatch.baseOffset(), remoteLogSegmentMetadata, fetchDataInfo, logOptional.get());
+            }
+
+            return fetchDataInfo;
+        } finally {
+            Utils.closeQuietly(remoteSegInputStream, "RemoteLogSegmentInputStream");
+        }
+    }
+
+    private int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        return indexCache.lookupOffset(remoteLogSegmentMetadata, offset);
+    }
+
+    private FetchDataInfo addAbortedTransactions(long startOffset,

Review Comment:
   Do we have a unit test for this method and/or the associated functionality in isolation?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,25 +623,210 @@ public String toString() {
         }
     }
 
-    long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws RemoteStorageException {
-        Optional<Long> offset = Optional.empty();
-        Optional<UnifiedLog> maybeLog = fetchLog.apply(topicIdPartition.topicPartition());
-        if (maybeLog.isPresent()) {
-            UnifiedLog log = maybeLog.get();
-            Option<LeaderEpochFileCache> maybeLeaderEpochFileCache = log.leaderEpochCache();
-            if (maybeLeaderEpochFileCache.isDefined()) {
-                LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get();
-                OptionalInt epoch = cache.latestEpoch();
-                while (!offset.isPresent() && epoch.isPresent()) {
-                    offset = remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch.getAsInt());
-                    epoch = cache.previousEpoch(epoch.getAsInt());
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);

Review Comment:
   Would it be worth adding a `DEBUG` log here to help with debugging?



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1288,17 +1373,46 @@ class ReplicaManager(val config: KafkaConfig,
                  _: FencedLeaderEpochException |
                  _: ReplicaNotAvailableException |
                  _: KafkaStorageException |
-                 _: OffsetOutOfRangeException |
                  _: InconsistentTopicIdException) =>
-          LogReadResult(info = new FetchDataInfo(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, MemoryRecords.EMPTY),
-            divergingEpoch = None,
-            highWatermark = UnifiedLog.UnknownOffset,
-            leaderLogStartOffset = UnifiedLog.UnknownOffset,
-            leaderLogEndOffset = UnifiedLog.UnknownOffset,
-            followerLogStartOffset = UnifiedLog.UnknownOffset,
-            fetchTimeMs = -1L,
-            lastStableOffset = None,
-            exception = Some(e))
+          createLogReadResult(e)
+        case e: OffsetOutOfRangeException =>
+          // In case of offset out of range errors, check for remote log manager for non-compacted topics
+          // to fetch from remote storage. `log` instance should not be null here as that would have been caught earlier
+          // with NotLeaderForPartitionException or ReplicaNotAvailableException.
+          // If it is from a follower then send the offset metadata only as the data is already available in remote
+          // storage.
+          if (remoteLogManager.isDefined && log != null && log.remoteLogEnabled() &&
+            // Check that the fetch offset is within the offset range within the remote storage layer.
+            log.logStartOffset <= offset && offset < log.localLogStartOffset()) {
+            // For follower fetch requests, throw an error saying that this offset is moved to tiered storage.
+            val highWatermark = log.highWatermark
+            val leaderLogStartOffset = log.logStartOffset

Review Comment:
   Ditto.



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1288,17 +1373,46 @@ class ReplicaManager(val config: KafkaConfig,
                  _: FencedLeaderEpochException |
                  _: ReplicaNotAvailableException |
                  _: KafkaStorageException |
-                 _: OffsetOutOfRangeException |
                  _: InconsistentTopicIdException) =>
-          LogReadResult(info = new FetchDataInfo(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, MemoryRecords.EMPTY),
-            divergingEpoch = None,
-            highWatermark = UnifiedLog.UnknownOffset,
-            leaderLogStartOffset = UnifiedLog.UnknownOffset,
-            leaderLogEndOffset = UnifiedLog.UnknownOffset,
-            followerLogStartOffset = UnifiedLog.UnknownOffset,
-            fetchTimeMs = -1L,
-            lastStableOffset = None,
-            exception = Some(e))
+          createLogReadResult(e)
+        case e: OffsetOutOfRangeException =>
+          // In case of offset out of range errors, check for remote log manager for non-compacted topics
+          // to fetch from remote storage. `log` instance should not be null here as that would have been caught earlier
+          // with NotLeaderForPartitionException or ReplicaNotAvailableException.
+          // If it is from a follower then send the offset metadata only as the data is already available in remote
+          // storage.
+          if (remoteLogManager.isDefined && log != null && log.remoteLogEnabled() &&
+            // Check that the fetch offset is within the offset range within the remote storage layer.
+            log.logStartOffset <= offset && offset < log.localLogStartOffset()) {

Review Comment:
   The log start offset and local log start offset could be updated asynchronously and non-atomically. Could that be a problem?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -578,9 +595,15 @@ public void run() {
                 return;
 
             try {
+                Optional<UnifiedLog> unifiedLogOptional = fetchLog.apply(topicIdPartition.topicPartition());
+
+                if (!unifiedLogOptional.isPresent()) {
+                    return;

Review Comment:
   Should we log this case at `DEBUG` level?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,25 +622,208 @@ public String toString() {
         }
     }
 
-    long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws RemoteStorageException {
-        Optional<Long> offset = Optional.empty();
-        Optional<UnifiedLog> maybeLog = fetchLog.apply(topicIdPartition.topicPartition());
-        if (maybeLog.isPresent()) {
-            UnifiedLog log = maybeLog.get();
-            Option<LeaderEpochFileCache> maybeLeaderEpochFileCache = log.leaderEpochCache();
-            if (maybeLeaderEpochFileCache.isDefined()) {
-                LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get();
-                OptionalInt epoch = cache.latestEpoch();
-                while (!offset.isPresent() && epoch.isPresent()) {
-                    offset = remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch.getAsInt());
-                    epoch = cache.previousEpoch(epoch.getAsInt());
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadataOptional = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadataOptional.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        RemoteLogSegmentMetadata remoteLogSegmentMetadata = rlsMetadataOptional.get();
+        int startPos = lookupPositionForOffset(remoteLogSegmentMetadata, offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            // An empty record is sent instead of an incomplete batch when there is no minimum-one-message constraint
+            // and for FetchRequest version 3 and above and the first batch size is more than maximum bytes that can be sent.
+            if (!remoteStorageFetchInfo.minOneMessage &&
+                    !remoteStorageFetchInfo.hardMaxBytesLimit &&

Review Comment:
   Tagging this comment as not longer valid since a code change addressed it.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,25 +622,208 @@ public String toString() {
         }
     }
 
-    long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws RemoteStorageException {
-        Optional<Long> offset = Optional.empty();
-        Optional<UnifiedLog> maybeLog = fetchLog.apply(topicIdPartition.topicPartition());
-        if (maybeLog.isPresent()) {
-            UnifiedLog log = maybeLog.get();
-            Option<LeaderEpochFileCache> maybeLeaderEpochFileCache = log.leaderEpochCache();
-            if (maybeLeaderEpochFileCache.isDefined()) {
-                LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get();
-                OptionalInt epoch = cache.latestEpoch();
-                while (!offset.isPresent() && epoch.isPresent()) {
-                    offset = remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch.getAsInt());
-                    epoch = cache.previousEpoch(epoch.getAsInt());
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadataOptional = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadataOptional.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        RemoteLogSegmentMetadata remoteLogSegmentMetadata = rlsMetadataOptional.get();
+        int startPos = lookupPositionForOffset(remoteLogSegmentMetadata, offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            // An empty record is sent instead of an incomplete batch when there is no minimum-one-message constraint
+            // and for FetchRequest version 3 and above and the first batch size is more than maximum bytes that can be sent.
+            int firstBatchSize = firstBatch.sizeInBytes();
+            if (!remoteStorageFetchInfo.minOneMessage &&
+                    !remoteStorageFetchInfo.hardMaxBytesLimit &&
+                    firstBatchSize > maxBytes) {
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY);
+            }
+
+            int updatedFetchSize =
+                    remoteStorageFetchInfo.minOneMessage && firstBatchSize > maxBytes ? firstBatchSize : maxBytes;
+
+            ByteBuffer buffer = ByteBuffer.allocate(updatedFetchSize);
+            int remainingBytes = updatedFetchSize;
+
+            firstBatch.writeTo(buffer);
+            remainingBytes -= firstBatchSize;
+
+            if (remainingBytes > 0) {
+                // read the input stream until min of (EOF stream or buffer's remaining capacity).
+                Utils.readFully(remoteSegInputStream, buffer);
+            }
+            buffer.flip();
+
+            FetchDataInfo fetchDataInfo = new FetchDataInfo(
+                    new LogOffsetMetadata(offset, remoteLogSegmentMetadata.startOffset(), startPos),
+                    MemoryRecords.readableRecords(buffer));
+            if (includeAbortedTxns) {
+                fetchDataInfo = addAbortedTransactions(firstBatch.baseOffset(), remoteLogSegmentMetadata, fetchDataInfo, logOptional.get());
+            }
+
+            return fetchDataInfo;
+        } finally {
+            Utils.closeQuietly(remoteSegInputStream, "RemoteLogSegmentInputStream");
+        }
+    }
+
+    private int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        return indexCache.lookupOffset(remoteLogSegmentMetadata, offset);
+    }
+
+    private FetchDataInfo addAbortedTransactions(long startOffset,
+                                                 RemoteLogSegmentMetadata segmentMetadata,
+                                                 FetchDataInfo fetchInfo,
+                                                 UnifiedLog log) throws RemoteStorageException {
+        int fetchSize = fetchInfo.records.sizeInBytes();
+        OffsetPosition startOffsetPosition = new OffsetPosition(fetchInfo.fetchOffsetMetadata.messageOffset,
+                fetchInfo.fetchOffsetMetadata.relativePositionInSegment);
+
+        OffsetIndex offsetIndex = indexCache.getIndexEntry(segmentMetadata).offsetIndex();
+        long upperBoundOffset = offsetIndex.fetchUpperBoundOffset(startOffsetPosition, fetchSize)
+                .map(x -> x.offset).orElse(segmentMetadata.endOffset() + 1);
+
+        final Set<FetchResponseData.AbortedTransaction> abortedTransactions = new HashSet<>();
+
+        Consumer<List<AbortedTxn>> accumulator =
+                abortedTxns -> abortedTransactions.addAll(abortedTxns.stream()
+                        .map(AbortedTxn::asAbortedTransaction).collect(Collectors.toList()));
+
+        collectAbortedTransactions(startOffset, upperBoundOffset, segmentMetadata, accumulator, log);
+
+        return new FetchDataInfo(fetchInfo.fetchOffsetMetadata,
+                fetchInfo.records,
+                fetchInfo.firstEntryIncomplete,
+                Optional.of(abortedTransactions.isEmpty() ? Collections.emptyList() : new ArrayList<>(abortedTransactions)));
+    }
+
+    private void collectAbortedTransactions(long startOffset,
+                                            long upperBoundOffset,
+                                            RemoteLogSegmentMetadata segmentMetadata,
+                                            Consumer<List<AbortedTxn>> accumulator,
+                                            UnifiedLog log) throws RemoteStorageException {
+        Iterator<LogSegment> localLogSegments = JavaConverters.asJavaIterator(log.logSegments().iterator());
+
+        boolean searchInLocalLog = false;
+        Optional<RemoteLogSegmentMetadata> nextSegmentMetadataOpt = Optional.of(segmentMetadata);
+        Optional<TransactionIndex> txnIndexOpt = nextSegmentMetadataOpt.map(metadata -> indexCache.getIndexEntry(metadata).txnIndex());
+
+        while (txnIndexOpt.isPresent()) {
+            TxnIndexSearchResult searchResult = txnIndexOpt.get().collectAbortedTxns(startOffset, upperBoundOffset);
+            accumulator.accept(searchResult.abortedTransactions);
+            if (!searchResult.isComplete) {
+                if (!searchInLocalLog) {
+                    nextSegmentMetadataOpt = findNextSegmentMetadata(nextSegmentMetadataOpt.get(), log);
+
+                    txnIndexOpt = nextSegmentMetadataOpt.map(x -> indexCache.getIndexEntry(x).txnIndex());
+                    if (!txnIndexOpt.isPresent()) {
+                        searchInLocalLog = true;
+                    }
+                }
+
+                if (searchInLocalLog) {
+                    txnIndexOpt = (localLogSegments.hasNext()) ? Optional.of(localLogSegments.next().txnIndex()) : Optional.empty();
                 }
+            } else {
+                return;
+            }
+        }
+    }
+
+    private Optional<RemoteLogSegmentMetadata> findNextSegmentMetadata(RemoteLogSegmentMetadata segmentMetadata, UnifiedLog log) throws RemoteStorageException {
+        Option<LeaderEpochFileCache> leaderEpochFileCacheOption = log.leaderEpochCache();
+        if (leaderEpochFileCacheOption.isEmpty()) {
+            return Optional.empty();
+        }
+
+        TopicPartition topicPartition = segmentMetadata.topicIdPartition().topicPartition();
+        long nextSegmentBaseOffset = segmentMetadata.endOffset() + 1;
+        OptionalInt epoch = OptionalInt.of(segmentMetadata.segmentLeaderEpochs().lastEntry().getKey());

Review Comment:
   Wouldn't it be possible to resolve the leader epoch of the `nextSegmentBaseOffset` directly from the leader epoch cache of the partition, removing the need for the loop altogether?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,25 +623,210 @@ public String toString() {
         }
     }
 
-    long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws RemoteStorageException {
-        Optional<Long> offset = Optional.empty();
-        Optional<UnifiedLog> maybeLog = fetchLog.apply(topicIdPartition.topicPartition());
-        if (maybeLog.isPresent()) {
-            UnifiedLog log = maybeLog.get();
-            Option<LeaderEpochFileCache> maybeLeaderEpochFileCache = log.leaderEpochCache();
-            if (maybeLeaderEpochFileCache.isDefined()) {
-                LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get();
-                OptionalInt epoch = cache.latestEpoch();
-                while (!offset.isPresent() && epoch.isPresent()) {
-                    offset = remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch.getAsInt());
-                    epoch = cache.previousEpoch(epoch.getAsInt());
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadataOptional = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadataOptional.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        RemoteLogSegmentMetadata remoteLogSegmentMetadata = rlsMetadataOptional.get();
+        int startPos = lookupPositionForOffset(remoteLogSegmentMetadata, offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            int firstBatchSize = firstBatch.sizeInBytes();
+            // An empty record is sent instead of an incomplete batch when
+            //  - there is no minimum-one-message constraint and
+            //  - the first batch size is more than maximum bytes that can be sent.
+            //  - for FetchRequest version 3 or above and
+            if (!remoteStorageFetchInfo.minOneMessage &&
+                    !remoteStorageFetchInfo.hardMaxBytesLimit &&
+                    firstBatchSize > maxBytes) {
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY);
+            }
+
+            int updatedFetchSize =
+                    remoteStorageFetchInfo.minOneMessage && firstBatchSize > maxBytes ? firstBatchSize : maxBytes;
+
+            ByteBuffer buffer = ByteBuffer.allocate(updatedFetchSize);
+            int remainingBytes = updatedFetchSize;
+
+            firstBatch.writeTo(buffer);
+            remainingBytes -= firstBatchSize;
+
+            if (remainingBytes > 0) {
+                // read the input stream until min of (EOF stream or buffer's remaining capacity).
+                Utils.readFully(remoteSegInputStream, buffer);
+            }
+            buffer.flip();
+
+            FetchDataInfo fetchDataInfo = new FetchDataInfo(
+                    new LogOffsetMetadata(offset, remoteLogSegmentMetadata.startOffset(), startPos),
+                    MemoryRecords.readableRecords(buffer));
+            if (includeAbortedTxns) {
+                fetchDataInfo = addAbortedTransactions(firstBatch.baseOffset(), remoteLogSegmentMetadata, fetchDataInfo, logOptional.get());
+            }
+
+            return fetchDataInfo;
+        } finally {
+            Utils.closeQuietly(remoteSegInputStream, "RemoteLogSegmentInputStream");
+        }
+    }
+
+    private int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        return indexCache.lookupOffset(remoteLogSegmentMetadata, offset);
+    }
+
+    private FetchDataInfo addAbortedTransactions(long startOffset,
+                                                 RemoteLogSegmentMetadata segmentMetadata,
+                                                 FetchDataInfo fetchInfo,
+                                                 UnifiedLog log) throws RemoteStorageException {
+        int fetchSize = fetchInfo.records.sizeInBytes();
+        OffsetPosition startOffsetPosition = new OffsetPosition(fetchInfo.fetchOffsetMetadata.messageOffset,
+                fetchInfo.fetchOffsetMetadata.relativePositionInSegment);
+
+        OffsetIndex offsetIndex = indexCache.getIndexEntry(segmentMetadata).offsetIndex();
+        long upperBoundOffset = offsetIndex.fetchUpperBoundOffset(startOffsetPosition, fetchSize)
+                .map(x -> x.offset).orElse(segmentMetadata.endOffset() + 1);
+
+        final Set<FetchResponseData.AbortedTransaction> abortedTransactions = new HashSet<>();
+
+        Consumer<List<AbortedTxn>> accumulator =
+                abortedTxns -> abortedTransactions.addAll(abortedTxns.stream()
+                        .map(AbortedTxn::asAbortedTransaction).collect(Collectors.toList()));
+
+        collectAbortedTransactions(startOffset, upperBoundOffset, segmentMetadata, accumulator, log);
+
+        return new FetchDataInfo(fetchInfo.fetchOffsetMetadata,
+                fetchInfo.records,
+                fetchInfo.firstEntryIncomplete,
+                Optional.of(abortedTransactions.isEmpty() ? Collections.emptyList() : new ArrayList<>(abortedTransactions)));
+    }
+
+    private void collectAbortedTransactions(long startOffset,
+                                            long upperBoundOffset,
+                                            RemoteLogSegmentMetadata segmentMetadata,
+                                            Consumer<List<AbortedTxn>> accumulator,
+                                            UnifiedLog log) throws RemoteStorageException {
+        Iterator<LogSegment> localLogSegments = JavaConverters.asJavaIterator(log.logSegments().iterator());
+
+        boolean searchInLocalLog = false;
+        Optional<RemoteLogSegmentMetadata> nextSegmentMetadataOpt = Optional.of(segmentMetadata);
+        Optional<TransactionIndex> txnIndexOpt = nextSegmentMetadataOpt.map(metadata -> indexCache.getIndexEntry(metadata).txnIndex());
+
+        while (txnIndexOpt.isPresent()) {
+            TxnIndexSearchResult searchResult = txnIndexOpt.get().collectAbortedTxns(startOffset, upperBoundOffset);
+            accumulator.accept(searchResult.abortedTransactions);
+            if (!searchResult.isComplete) {
+                if (!searchInLocalLog) {

Review Comment:
   Maybe we could separate the local and remote accumulation from within the same loop and segregate the branches for local/remote from the core logic?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1187412666


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1083,48 +1095,100 @@ class ReplicaManager(val config: KafkaConfig,
           fetchPartitionStatus += (topicIdPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
         })
       }
-      val delayedFetch = new DelayedFetch(
-        params = params,
-        fetchPartitionStatus = fetchPartitionStatus,
-        replicaManager = this,
-        quota = quota,
-        responseCallback = responseCallback
-      )
-
-      // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
-      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp) }
-
-      // try to complete the request immediately, otherwise put it into the purgatory;
-      // this is because while the delayed fetch operation is being created, new requests
-      // may arrive and hence make this operation completable.
-      delayedFetchPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys)
+
+      if (remoteFetchInfo.isPresent) {

Review Comment:
   Do you mean to say that we should not return immediately if `remoteFetchInfo` exists because that should be served otherwise remote fetches may starve as long as there is enough data immediately available to be sent? So, the condition becomes 
   
   ```
       if (!remoteFetchInfo.isPresent && (params.maxWaitMs <= 0 || fetchInfos.isEmpty 
               || bytesReadable >= params.minBytes || errorReadingData || hasDivergingEpoch 
               || hasPreferredReadReplica))
   ```
   



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: [DRAFT] KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1164328159


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1196,17 +1252,45 @@ class ReplicaManager(val config: KafkaConfig,
                  _: FencedLeaderEpochException |
                  _: ReplicaNotAvailableException |
                  _: KafkaStorageException |
-                 _: OffsetOutOfRangeException |
                  _: InconsistentTopicIdException) =>
-          LogReadResult(info = new FetchDataInfo(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, MemoryRecords.EMPTY),
-            divergingEpoch = None,
-            highWatermark = UnifiedLog.UnknownOffset,
-            leaderLogStartOffset = UnifiedLog.UnknownOffset,
-            leaderLogEndOffset = UnifiedLog.UnknownOffset,
-            followerLogStartOffset = UnifiedLog.UnknownOffset,
-            fetchTimeMs = -1L,
-            lastStableOffset = None,
-            exception = Some(e))
+          createLogReadResult(e)
+        case e: OffsetOutOfRangeException =>
+          // In case of offset out of range errors, check for remote log manager for non-compacted topics
+          // to fetch from remote storage. `log` instance should not be null here as that would have caught earlier with
+          // NotLeaderForPartitionException or ReplicaNotAvailableException.
+          // If it is from a follower then send the offset metadata but not the records data as that can be fetched

Review Comment:
   I reworded that sentence to make it more clear. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeqo commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "jeqo (via GitHub)" <gi...@apache.org>.
jeqo commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1166318900


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,6 +622,176 @@ public String toString() {
         }
     }
 
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadata = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadata.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        int startPos = lookupPositionForOffset(rlsMetadata.get(), offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(rlsMetadata.get(), startPos);

Review Comment:
   Would be possible sending the endOffset as well? Without it, input stream will potentially contain the whole log and not be consumed til the end.
   In the case of S3, when inputstream is not consumed til the end HTTP connection is aborted.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1186812350


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,6 +622,176 @@ public String toString() {
         }
     }
 
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadata = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadata.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        int startPos = lookupPositionForOffset(rlsMetadata.get(), offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(rlsMetadata.get(), startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            int updatedFetchSize =
+                    remoteStorageFetchInfo.minOneMessage && firstBatch.sizeInBytes() > maxBytes
+                            ? firstBatch.sizeInBytes() : maxBytes;
+
+            ByteBuffer buffer = ByteBuffer.allocate(updatedFetchSize);
+            int remainingBytes = updatedFetchSize;
+
+            firstBatch.writeTo(buffer);
+            remainingBytes -= firstBatch.sizeInBytes();
+
+            if (remainingBytes > 0) {
+                // input stream is read till (startPos - 1) while getting the batch of records earlier.
+                // read the input stream until min of (EOF stream or buffer's remaining capacity).
+                Utils.readFully(remoteSegInputStream, buffer);
+            }
+            buffer.flip();
+
+            FetchDataInfo fetchDataInfo = new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.readableRecords(buffer));
+            if (includeAbortedTxns) {
+                fetchDataInfo = addAbortedTransactions(firstBatch.baseOffset(), rlsMetadata.get(), fetchDataInfo);
+            }
+
+            return fetchDataInfo;
+        } finally {
+            Utils.closeQuietly(remoteSegInputStream, "RemoteLogSegmentInputStream");
+        }
+    }
+
+    private int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        return indexCache.lookupOffset(remoteLogSegmentMetadata, offset);
+    }
+
+    private FetchDataInfo addAbortedTransactions(long startOffset,
+                                         RemoteLogSegmentMetadata segmentMetadata,
+                                         FetchDataInfo fetchInfo) throws RemoteStorageException {
+        int fetchSize = fetchInfo.records.sizeInBytes();
+        OffsetPosition startOffsetPosition = new OffsetPosition(fetchInfo.fetchOffsetMetadata.messageOffset,
+                fetchInfo.fetchOffsetMetadata.relativePositionInSegment);
+
+        OffsetIndex offsetIndex = indexCache.getIndexEntry(segmentMetadata).offsetIndex();
+        long upperBoundOffset = offsetIndex.fetchUpperBoundOffset(startOffsetPosition, fetchSize)
+                .map(x -> x.offset).orElse(segmentMetadata.endOffset() + 1);
+
+        final List<FetchResponseData.AbortedTransaction> abortedTransactions = new ArrayList<>();
+
+        Consumer<List<AbortedTxn>> accumulator =
+                abortedTxns -> abortedTransactions.addAll(abortedTxns.stream()
+                        .map(AbortedTxn::asAbortedTransaction).collect(Collectors.toList()));
+
+        collectAbortedTransactions(startOffset, upperBoundOffset, segmentMetadata, accumulator);
+
+        return new FetchDataInfo(fetchInfo.fetchOffsetMetadata,
+                fetchInfo.records,
+                fetchInfo.firstEntryIncomplete,
+                Optional.of(abortedTransactions));
+    }
+
+    private void collectAbortedTransactions(long startOffset,
+                                    long upperBoundOffset,
+                                    RemoteLogSegmentMetadata segmentMetadata,
+                                    Consumer<List<AbortedTxn>> accumulator) throws RemoteStorageException {
+        TopicPartition topicPartition = segmentMetadata.topicIdPartition().topicPartition();
+        Iterator<LogSegment> localLogSegments = fetchLog.apply(topicPartition)
+                .map(log -> JavaConverters.asJavaCollection(log.logSegments()))
+                .map(Collection::iterator)
+                .orElse(Collections.emptyIterator());
+
+        boolean searchInLocalLog = false;
+        Optional<RemoteLogSegmentMetadata> nextSegmentMetadataOpt = Optional.of(segmentMetadata);
+        Optional<TransactionIndex> txnIndexOpt = nextSegmentMetadataOpt.map(metadata -> indexCache.getIndexEntry(metadata).txnIndex());
+
+        while (txnIndexOpt.isPresent()) {
+            TxnIndexSearchResult searchResult = txnIndexOpt.get().collectAbortedTxns(startOffset, upperBoundOffset);
+            accumulator.accept(searchResult.abortedTransactions);
+            if (!searchResult.isComplete) {
+                if (!searchInLocalLog) {
+                    nextSegmentMetadataOpt = findNextSegmentMetadata(nextSegmentMetadataOpt.get());
+
+                    txnIndexOpt = nextSegmentMetadataOpt.map(x -> indexCache.getIndexEntry(x).txnIndex());
+                    if (!txnIndexOpt.isPresent()) {
+                        searchInLocalLog = true;
+                    }
+                }
+
+                if (searchInLocalLog) {
+                    txnIndexOpt = (localLogSegments.hasNext()) ? Optional.of(localLogSegments.next().txnIndex()) : Optional.empty();

Review Comment:
   `accumulator` collects these transactions and adds them to a set that removes the duplicates. You can find the accumulator [here](https://github.com/apache/kafka/pull/13535/files#diff-380e4d8859ea9148f21794c09039425c82d9012a392c2dbbe1ce2ec8677a1970R727).



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1192196398


##########
core/src/test/scala/integration/kafka/server/DelayedRemoteFetchTest.scala:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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 kafka.cluster.Partition
+import org.apache.kafka.common.errors.NotLeaderOrFollowerException
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.record.MemoryRecords
+import org.apache.kafka.common.requests.FetchRequest
+import org.apache.kafka.common.{TopicIdPartition, Uuid}
+import org.apache.kafka.storage.internals.log._
+import org.junit.jupiter.api.Assertions._
+import org.junit.jupiter.api.Test
+import org.mockito.Mockito.{mock, when}
+
+import java.util.Optional
+import java.util.concurrent.CompletableFuture
+
+import scala.collection._
+
+class DelayedRemoteFetchTest {
+  private val maxBytes = 1024
+  private val replicaManager: ReplicaManager = mock(classOf[ReplicaManager])
+  private val topicIdPartition = new TopicIdPartition(Uuid.randomUuid(), 0, "topic")
+  private val fetchOffset = 500L
+  private val logStartOffset = 0L
+  private val currentLeaderEpoch = Optional.of[Integer](10)
+  private val replicaId = 1
+
+  private val fetchStatus = FetchPartitionStatus(
+    startOffsetMetadata = new LogOffsetMetadata(fetchOffset),
+    fetchInfo = new FetchRequest.PartitionData(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch))
+  private val fetchParams = buildFollowerFetchParams(replicaId, maxWaitMs = 500)
+
+  @Test
+  def testFetchWithFencedEpoch(): Unit = {

Review Comment:
   This is not meant to be fenced epoch. It is covered in another test. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13535:
URL: https://github.com/apache/kafka/pull/13535#issuecomment-1547042358

   @junrao Looks like those test failures are related to this change. They do not fail on the laptop or other hosts. We are looking into them.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] Hangleton commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1166887367


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,6 +622,176 @@ public String toString() {
         }
     }
 
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadata = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadata.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        int startPos = lookupPositionForOffset(rlsMetadata.get(), offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(rlsMetadata.get(), startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            int updatedFetchSize =
+                    remoteStorageFetchInfo.minOneMessage && firstBatch.sizeInBytes() > maxBytes
+                            ? firstBatch.sizeInBytes() : maxBytes;
+
+            ByteBuffer buffer = ByteBuffer.allocate(updatedFetchSize);
+            int remainingBytes = updatedFetchSize;
+
+            firstBatch.writeTo(buffer);
+            remainingBytes -= firstBatch.sizeInBytes();
+
+            if (remainingBytes > 0) {
+                // input stream is read till (startPos - 1) while getting the batch of records earlier.
+                // read the input stream until min of (EOF stream or buffer's remaining capacity).
+                Utils.readFully(remoteSegInputStream, buffer);
+            }
+            buffer.flip();
+
+            FetchDataInfo fetchDataInfo = new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.readableRecords(buffer));
+            if (includeAbortedTxns) {
+                fetchDataInfo = addAbortedTransactions(firstBatch.baseOffset(), rlsMetadata.get(), fetchDataInfo);
+            }
+
+            return fetchDataInfo;
+        } finally {
+            Utils.closeQuietly(remoteSegInputStream, "RemoteLogSegmentInputStream");
+        }
+    }
+
+    private int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        return indexCache.lookupOffset(remoteLogSegmentMetadata, offset);
+    }
+
+    private FetchDataInfo addAbortedTransactions(long startOffset,
+                                         RemoteLogSegmentMetadata segmentMetadata,
+                                         FetchDataInfo fetchInfo) throws RemoteStorageException {
+        int fetchSize = fetchInfo.records.sizeInBytes();
+        OffsetPosition startOffsetPosition = new OffsetPosition(fetchInfo.fetchOffsetMetadata.messageOffset,
+                fetchInfo.fetchOffsetMetadata.relativePositionInSegment);
+
+        OffsetIndex offsetIndex = indexCache.getIndexEntry(segmentMetadata).offsetIndex();
+        long upperBoundOffset = offsetIndex.fetchUpperBoundOffset(startOffsetPosition, fetchSize)
+                .map(x -> x.offset).orElse(segmentMetadata.endOffset() + 1);
+
+        final List<FetchResponseData.AbortedTransaction> abortedTransactions = new ArrayList<>();
+
+        Consumer<List<AbortedTxn>> accumulator =
+                abortedTxns -> abortedTransactions.addAll(abortedTxns.stream()
+                        .map(AbortedTxn::asAbortedTransaction).collect(Collectors.toList()));
+
+        collectAbortedTransactions(startOffset, upperBoundOffset, segmentMetadata, accumulator);
+
+        return new FetchDataInfo(fetchInfo.fetchOffsetMetadata,
+                fetchInfo.records,
+                fetchInfo.firstEntryIncomplete,
+                Optional.of(abortedTransactions));
+    }
+
+    private void collectAbortedTransactions(long startOffset,
+                                    long upperBoundOffset,
+                                    RemoteLogSegmentMetadata segmentMetadata,
+                                    Consumer<List<AbortedTxn>> accumulator) throws RemoteStorageException {
+        TopicPartition topicPartition = segmentMetadata.topicIdPartition().topicPartition();
+        Iterator<LogSegment> localLogSegments = fetchLog.apply(topicPartition)
+                .map(log -> JavaConverters.asJavaCollection(log.logSegments()))
+                .map(Collection::iterator)
+                .orElse(Collections.emptyIterator());
+
+        boolean searchInLocalLog = false;
+        Optional<RemoteLogSegmentMetadata> nextSegmentMetadataOpt = Optional.of(segmentMetadata);
+        Optional<TransactionIndex> txnIndexOpt = nextSegmentMetadataOpt.map(metadata -> indexCache.getIndexEntry(metadata).txnIndex());
+
+        while (txnIndexOpt.isPresent()) {
+            TxnIndexSearchResult searchResult = txnIndexOpt.get().collectAbortedTxns(startOffset, upperBoundOffset);
+            accumulator.accept(searchResult.abortedTransactions);
+            if (!searchResult.isComplete) {
+                if (!searchInLocalLog) {
+                    nextSegmentMetadataOpt = findNextSegmentMetadata(nextSegmentMetadataOpt.get());
+
+                    txnIndexOpt = nextSegmentMetadataOpt.map(x -> indexCache.getIndexEntry(x).txnIndex());

Review Comment:
   There may be an availability risk here. The remote index cache allows up to 1,024 entries by default and it is possible to keep a transaction index entry permanently rotated in and out of the cache so that retries of Fetch requests for a partition at a given (constant) fetch offset permanently trigger a download of the index. This, plus the fact there is a global lock for read and write access on the remote index cache, can lead to lock contention on the remote log reader thread pool. In the worst case, it is possible for a partition to be prevented from making progress on the fetch path.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1170273242


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,6 +622,176 @@ public String toString() {
         }
     }
 
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadata = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadata.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        int startPos = lookupPositionForOffset(rlsMetadata.get(), offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(rlsMetadata.get(), startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            int updatedFetchSize =
+                    remoteStorageFetchInfo.minOneMessage && firstBatch.sizeInBytes() > maxBytes
+                            ? firstBatch.sizeInBytes() : maxBytes;
+
+            ByteBuffer buffer = ByteBuffer.allocate(updatedFetchSize);
+            int remainingBytes = updatedFetchSize;
+
+            firstBatch.writeTo(buffer);
+            remainingBytes -= firstBatch.sizeInBytes();
+
+            if (remainingBytes > 0) {
+                // input stream is read till (startPos - 1) while getting the batch of records earlier.
+                // read the input stream until min of (EOF stream or buffer's remaining capacity).
+                Utils.readFully(remoteSegInputStream, buffer);
+            }
+            buffer.flip();
+
+            FetchDataInfo fetchDataInfo = new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.readableRecords(buffer));

Review Comment:
   Good catch, addressed it in the latest commit. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeqo commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "jeqo (via GitHub)" <gi...@apache.org>.
jeqo commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1189509107


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,25 +623,204 @@ public String toString() {
         }
     }
 
-    long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws RemoteStorageException {
-        Optional<Long> offset = Optional.empty();
-        Optional<UnifiedLog> maybeLog = fetchLog.apply(topicIdPartition.topicPartition());
-        if (maybeLog.isPresent()) {
-            UnifiedLog log = maybeLog.get();
-            Option<LeaderEpochFileCache> maybeLeaderEpochFileCache = log.leaderEpochCache();
-            if (maybeLeaderEpochFileCache.isDefined()) {
-                LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get();
-                OptionalInt epoch = cache.latestEpoch();
-                while (!offset.isPresent() && epoch.isPresent()) {
-                    offset = remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch.getAsInt());
-                    epoch = cache.previousEpoch(epoch.getAsInt());
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadataOptional = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadataOptional.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        RemoteLogSegmentMetadata remoteLogSegmentMetadata = rlsMetadataOptional.get();
+        int startPos = lookupPositionForOffset(remoteLogSegmentMetadata, offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            int firstBatchSize = firstBatch.sizeInBytes();
+            // An empty record is sent instead of an incomplete batch when
+            //  - there is no minimum-one-message constraint and
+            //  - the first batch size is more than maximum bytes that can be sent and
+            //  - for FetchRequest version 3 or above.
+            if (!remoteStorageFetchInfo.minOneMessage &&
+                    !remoteStorageFetchInfo.hardMaxBytesLimit &&
+                    firstBatchSize > maxBytes) {
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY);
+            }
+
+            int updatedFetchSize =
+                    remoteStorageFetchInfo.minOneMessage && firstBatchSize > maxBytes ? firstBatchSize : maxBytes;
+
+            ByteBuffer buffer = ByteBuffer.allocate(updatedFetchSize);
+            int remainingBytes = updatedFetchSize;
+
+            firstBatch.writeTo(buffer);
+            remainingBytes -= firstBatchSize;
+
+            if (remainingBytes > 0) {
+                // read the input stream until min of (EOF stream or buffer's remaining capacity).
+                Utils.readFully(remoteSegInputStream, buffer);
+            }
+            buffer.flip();
+
+            FetchDataInfo fetchDataInfo = new FetchDataInfo(
+                    new LogOffsetMetadata(offset, remoteLogSegmentMetadata.startOffset(), startPos),
+                    MemoryRecords.readableRecords(buffer));
+            if (includeAbortedTxns) {
+                fetchDataInfo = addAbortedTransactions(firstBatch.baseOffset(), remoteLogSegmentMetadata, fetchDataInfo, logOptional.get());
+            }
+
+            return fetchDataInfo;
+        } finally {
+            Utils.closeQuietly(remoteSegInputStream, "RemoteLogSegmentInputStream");
+        }
+    }
+
+    private int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        return indexCache.lookupOffset(remoteLogSegmentMetadata, offset);
+    }
+
+    private FetchDataInfo addAbortedTransactions(long startOffset,
+                                                 RemoteLogSegmentMetadata segmentMetadata,
+                                                 FetchDataInfo fetchInfo,
+                                                 UnifiedLog log) throws RemoteStorageException {
+        int fetchSize = fetchInfo.records.sizeInBytes();
+        OffsetPosition startOffsetPosition = new OffsetPosition(fetchInfo.fetchOffsetMetadata.messageOffset,
+                fetchInfo.fetchOffsetMetadata.relativePositionInSegment);
+
+        OffsetIndex offsetIndex = indexCache.getIndexEntry(segmentMetadata).offsetIndex();
+        long upperBoundOffset = offsetIndex.fetchUpperBoundOffset(startOffsetPosition, fetchSize)
+                .map(x -> x.offset).orElse(segmentMetadata.endOffset() + 1);
+
+        final Set<FetchResponseData.AbortedTransaction> abortedTransactions = new HashSet<>();
+
+        Consumer<List<AbortedTxn>> accumulator =
+                abortedTxns -> abortedTransactions.addAll(abortedTxns.stream()
+                        .map(AbortedTxn::asAbortedTransaction).collect(Collectors.toList()));
+
+        collectAbortedTransactions(startOffset, upperBoundOffset, segmentMetadata, accumulator, log);
+
+        return new FetchDataInfo(fetchInfo.fetchOffsetMetadata,
+                fetchInfo.records,
+                fetchInfo.firstEntryIncomplete,
+                Optional.of(abortedTransactions.isEmpty() ? Collections.emptyList() : new ArrayList<>(abortedTransactions)));
+    }
+
+    private void collectAbortedTransactions(long startOffset,
+                                            long upperBoundOffset,
+                                            RemoteLogSegmentMetadata segmentMetadata,
+                                            Consumer<List<AbortedTxn>> accumulator,
+                                            UnifiedLog log) throws RemoteStorageException {
+        Iterator<LogSegment> localLogSegments = JavaConverters.asJavaIterator(log.logSegments().iterator());
+
+        boolean searchInLocalLog = false;
+        Optional<RemoteLogSegmentMetadata> nextSegmentMetadataOpt = Optional.of(segmentMetadata);
+        Optional<TransactionIndex> txnIndexOpt = nextSegmentMetadataOpt.map(metadata -> indexCache.getIndexEntry(metadata).txnIndex());

Review Comment:
   Related to this [question](https://issues.apache.org/jira/browse/KAFKA-7739?focusedCommentId=17720955&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17720955): 
   Does RLM expects to receive a null when this index is not available on RSM, or to throw `RemoteResourceNotFoundException`?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1188479417


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1288,17 +1373,46 @@ class ReplicaManager(val config: KafkaConfig,
                  _: FencedLeaderEpochException |
                  _: ReplicaNotAvailableException |
                  _: KafkaStorageException |
-                 _: OffsetOutOfRangeException |
                  _: InconsistentTopicIdException) =>
-          LogReadResult(info = new FetchDataInfo(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, MemoryRecords.EMPTY),
-            divergingEpoch = None,
-            highWatermark = UnifiedLog.UnknownOffset,
-            leaderLogStartOffset = UnifiedLog.UnknownOffset,
-            leaderLogEndOffset = UnifiedLog.UnknownOffset,
-            followerLogStartOffset = UnifiedLog.UnknownOffset,
-            fetchTimeMs = -1L,
-            lastStableOffset = None,
-            exception = Some(e))
+          createLogReadResult(e)
+        case e: OffsetOutOfRangeException =>
+          // In case of offset out of range errors, check for remote log manager for non-compacted topics
+          // to fetch from remote storage. `log` instance should not be null here as that would have been caught earlier
+          // with NotLeaderForPartitionException or ReplicaNotAvailableException.
+          // If it is from a follower then send the offset metadata only as the data is already available in remote
+          // storage.
+          if (remoteLogManager.isDefined && log != null && log.remoteLogEnabled() &&
+            // Check that the fetch offset is within the offset range within the remote storage layer.
+            log.logStartOffset <= offset && offset < log.localLogStartOffset()) {

Review Comment:
   That should work fine because it will eventually throw offset out-of-range error if the target offset does not exist. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1186813113


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,25 +622,208 @@ public String toString() {
         }
     }
 
-    long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws RemoteStorageException {
-        Optional<Long> offset = Optional.empty();
-        Optional<UnifiedLog> maybeLog = fetchLog.apply(topicIdPartition.topicPartition());
-        if (maybeLog.isPresent()) {
-            UnifiedLog log = maybeLog.get();
-            Option<LeaderEpochFileCache> maybeLeaderEpochFileCache = log.leaderEpochCache();
-            if (maybeLeaderEpochFileCache.isDefined()) {
-                LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get();
-                OptionalInt epoch = cache.latestEpoch();
-                while (!offset.isPresent() && epoch.isPresent()) {
-                    offset = remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch.getAsInt());
-                    epoch = cache.previousEpoch(epoch.getAsInt());
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadataOptional = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadataOptional.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        RemoteLogSegmentMetadata remoteLogSegmentMetadata = rlsMetadataOptional.get();
+        int startPos = lookupPositionForOffset(remoteLogSegmentMetadata, offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            // An empty record is sent instead of an incomplete batch when there is no minimum-one-message constraint
+            // and for FetchRequest version 3 and above and the first batch size is more than maximum bytes that can be sent.
+            int firstBatchSize = firstBatch.sizeInBytes();
+            if (!remoteStorageFetchInfo.minOneMessage &&
+                    !remoteStorageFetchInfo.hardMaxBytesLimit &&
+                    firstBatchSize > maxBytes) {
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY);
+            }
+
+            int updatedFetchSize =
+                    remoteStorageFetchInfo.minOneMessage && firstBatchSize > maxBytes ? firstBatchSize : maxBytes;
+
+            ByteBuffer buffer = ByteBuffer.allocate(updatedFetchSize);
+            int remainingBytes = updatedFetchSize;
+
+            firstBatch.writeTo(buffer);
+            remainingBytes -= firstBatchSize;
+
+            if (remainingBytes > 0) {
+                // read the input stream until min of (EOF stream or buffer's remaining capacity).
+                Utils.readFully(remoteSegInputStream, buffer);
+            }
+            buffer.flip();
+
+            FetchDataInfo fetchDataInfo = new FetchDataInfo(
+                    new LogOffsetMetadata(offset, remoteLogSegmentMetadata.startOffset(), startPos),
+                    MemoryRecords.readableRecords(buffer));
+            if (includeAbortedTxns) {
+                fetchDataInfo = addAbortedTransactions(firstBatch.baseOffset(), remoteLogSegmentMetadata, fetchDataInfo, logOptional.get());
+            }
+
+            return fetchDataInfo;
+        } finally {
+            Utils.closeQuietly(remoteSegInputStream, "RemoteLogSegmentInputStream");
+        }
+    }
+
+    private int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        return indexCache.lookupOffset(remoteLogSegmentMetadata, offset);
+    }
+
+    private FetchDataInfo addAbortedTransactions(long startOffset,
+                                                 RemoteLogSegmentMetadata segmentMetadata,
+                                                 FetchDataInfo fetchInfo,
+                                                 UnifiedLog log) throws RemoteStorageException {
+        int fetchSize = fetchInfo.records.sizeInBytes();
+        OffsetPosition startOffsetPosition = new OffsetPosition(fetchInfo.fetchOffsetMetadata.messageOffset,
+                fetchInfo.fetchOffsetMetadata.relativePositionInSegment);
+
+        OffsetIndex offsetIndex = indexCache.getIndexEntry(segmentMetadata).offsetIndex();
+        long upperBoundOffset = offsetIndex.fetchUpperBoundOffset(startOffsetPosition, fetchSize)
+                .map(x -> x.offset).orElse(segmentMetadata.endOffset() + 1);
+
+        final Set<FetchResponseData.AbortedTransaction> abortedTransactions = new HashSet<>();
+
+        Consumer<List<AbortedTxn>> accumulator =
+                abortedTxns -> abortedTransactions.addAll(abortedTxns.stream()
+                        .map(AbortedTxn::asAbortedTransaction).collect(Collectors.toList()));
+
+        collectAbortedTransactions(startOffset, upperBoundOffset, segmentMetadata, accumulator, log);
+
+        return new FetchDataInfo(fetchInfo.fetchOffsetMetadata,
+                fetchInfo.records,
+                fetchInfo.firstEntryIncomplete,
+                Optional.of(abortedTransactions.isEmpty() ? Collections.emptyList() : new ArrayList<>(abortedTransactions)));
+    }
+
+    private void collectAbortedTransactions(long startOffset,
+                                            long upperBoundOffset,
+                                            RemoteLogSegmentMetadata segmentMetadata,
+                                            Consumer<List<AbortedTxn>> accumulator,
+                                            UnifiedLog log) throws RemoteStorageException {
+        Iterator<LogSegment> localLogSegments = JavaConverters.asJavaIterator(log.logSegments().iterator());
+
+        boolean searchInLocalLog = false;
+        Optional<RemoteLogSegmentMetadata> nextSegmentMetadataOpt = Optional.of(segmentMetadata);
+        Optional<TransactionIndex> txnIndexOpt = nextSegmentMetadataOpt.map(metadata -> indexCache.getIndexEntry(metadata).txnIndex());
+
+        while (txnIndexOpt.isPresent()) {
+            TxnIndexSearchResult searchResult = txnIndexOpt.get().collectAbortedTxns(startOffset, upperBoundOffset);
+            accumulator.accept(searchResult.abortedTransactions);
+            if (!searchResult.isComplete) {
+                if (!searchInLocalLog) {
+                    nextSegmentMetadataOpt = findNextSegmentMetadata(nextSegmentMetadataOpt.get(), log);
+
+                    txnIndexOpt = nextSegmentMetadataOpt.map(x -> indexCache.getIndexEntry(x).txnIndex());
+                    if (!txnIndexOpt.isPresent()) {
+                        searchInLocalLog = true;
+                    }
+                }
+
+                if (searchInLocalLog) {
+                    txnIndexOpt = (localLogSegments.hasNext()) ? Optional.of(localLogSegments.next().txnIndex()) : Optional.empty();
                 }
+            } else {
+                return;
+            }
+        }
+    }
+
+    private Optional<RemoteLogSegmentMetadata> findNextSegmentMetadata(RemoteLogSegmentMetadata segmentMetadata, UnifiedLog log) throws RemoteStorageException {
+        Option<LeaderEpochFileCache> leaderEpochFileCacheOption = log.leaderEpochCache();
+        if (leaderEpochFileCacheOption.isEmpty()) {
+            return Optional.empty();
+        }
+
+        TopicPartition topicPartition = segmentMetadata.topicIdPartition().topicPartition();
+        long nextSegmentBaseOffset = segmentMetadata.endOffset() + 1;
+        OptionalInt epoch = OptionalInt.of(segmentMetadata.segmentLeaderEpochs().lastEntry().getKey());

Review Comment:
   It initially tries to find a segment for that epoch and the offset. If that is not found, it gets the next available epoch by using `leaderEpochFileCache.nextEpoch(epoch.getAsInt())` and tries to find a segment for that epoch. It eventually tries to find it by going through the epoch sequence like below.
   
   ```
           while (!result.isPresent() && epoch.isPresent()) {
               result = fetchRemoteLogSegmentMetadata(topicPartition, epoch.getAsInt(), nextSegmentBaseOffset);
               epoch = leaderEpochFileCache.nextEpoch(epoch.getAsInt());
           }
   ``` 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] junrao commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "junrao (via GitHub)" <gi...@apache.org>.
junrao commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1187906662


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1083,48 +1095,100 @@ class ReplicaManager(val config: KafkaConfig,
           fetchPartitionStatus += (topicIdPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
         })
       }
-      val delayedFetch = new DelayedFetch(
-        params = params,
-        fetchPartitionStatus = fetchPartitionStatus,
-        replicaManager = this,
-        quota = quota,
-        responseCallback = responseCallback
-      )
-
-      // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
-      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp) }
-
-      // try to complete the request immediately, otherwise put it into the purgatory;
-      // this is because while the delayed fetch operation is being created, new requests
-      // may arrive and hence make this operation completable.
-      delayedFetchPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys)
+
+      if (remoteFetchInfo.isPresent) {

Review Comment:
   Yes



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1170269949


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1083,48 +1095,100 @@ class ReplicaManager(val config: KafkaConfig,
           fetchPartitionStatus += (topicIdPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
         })
       }
-      val delayedFetch = new DelayedFetch(
-        params = params,
-        fetchPartitionStatus = fetchPartitionStatus,
-        replicaManager = this,
-        quota = quota,
-        responseCallback = responseCallback
-      )
-
-      // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
-      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp) }
-
-      // try to complete the request immediately, otherwise put it into the purgatory;
-      // this is because while the delayed fetch operation is being created, new requests
-      // may arrive and hence make this operation completable.
-      delayedFetchPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys)
+
+      if (remoteFetchInfo.isPresent) {

Review Comment:
   I am not sure line num:1082 is sane as you meant it to be as the file could have been updated. Please clarify. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] divijvaidya commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "divijvaidya (via GitHub)" <gi...@apache.org>.
divijvaidya commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1180144730


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,25 +622,208 @@ public String toString() {
         }
     }
 
-    long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws RemoteStorageException {
-        Optional<Long> offset = Optional.empty();
-        Optional<UnifiedLog> maybeLog = fetchLog.apply(topicIdPartition.topicPartition());
-        if (maybeLog.isPresent()) {
-            UnifiedLog log = maybeLog.get();
-            Option<LeaderEpochFileCache> maybeLeaderEpochFileCache = log.leaderEpochCache();
-            if (maybeLeaderEpochFileCache.isDefined()) {
-                LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get();
-                OptionalInt epoch = cache.latestEpoch();
-                while (!offset.isPresent() && epoch.isPresent()) {
-                    offset = remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch.getAsInt());
-                    epoch = cache.previousEpoch(epoch.getAsInt());
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadataOptional = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadataOptional.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        RemoteLogSegmentMetadata remoteLogSegmentMetadata = rlsMetadataOptional.get();
+        int startPos = lookupPositionForOffset(remoteLogSegmentMetadata, offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);

Review Comment:
   Perf comment: not blocking for this PR
   
   Note that `RemoteLogInputStream` performs a heap allocation here for each batch. This allocation will increase GC activity with the size of segment. Perhaps, we can use BufferSupplier (we allocate one pool per request handler thread, see `requestLocal.bufferSupplier` in KafkaAPI.scala) here in future.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,25 +622,208 @@ public String toString() {
         }
     }
 
-    long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws RemoteStorageException {
-        Optional<Long> offset = Optional.empty();
-        Optional<UnifiedLog> maybeLog = fetchLog.apply(topicIdPartition.topicPartition());
-        if (maybeLog.isPresent()) {
-            UnifiedLog log = maybeLog.get();
-            Option<LeaderEpochFileCache> maybeLeaderEpochFileCache = log.leaderEpochCache();
-            if (maybeLeaderEpochFileCache.isDefined()) {
-                LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get();
-                OptionalInt epoch = cache.latestEpoch();
-                while (!offset.isPresent() && epoch.isPresent()) {
-                    offset = remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch.getAsInt());
-                    epoch = cache.previousEpoch(epoch.getAsInt());
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadataOptional = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadataOptional.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        RemoteLogSegmentMetadata remoteLogSegmentMetadata = rlsMetadataOptional.get();
+        int startPos = lookupPositionForOffset(remoteLogSegmentMetadata, offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            // An empty record is sent instead of an incomplete batch when there is no minimum-one-message constraint
+            // and for FetchRequest version 3 and above and the first batch size is more than maximum bytes that can be sent.
+            if (!remoteStorageFetchInfo.minOneMessage &&
+                    !remoteStorageFetchInfo.hardMaxBytesLimit &&
+                    firstBatch.sizeInBytes() > maxBytes) {
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY);
+            }
+
+            int updatedFetchSize =
+                    remoteStorageFetchInfo.minOneMessage && firstBatch.sizeInBytes() > maxBytes
+                            ? firstBatch.sizeInBytes() : maxBytes;

Review Comment:
   Perf comment: not blocking for this PR
   
   Please calculate `firstBatch.sizeInBytes()` once and reuse. 



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1160,48 +1171,100 @@ class ReplicaManager(val config: KafkaConfig,
           fetchPartitionStatus += (topicIdPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
         })
       }
-      val delayedFetch = new DelayedFetch(
-        params = params,
-        fetchPartitionStatus = fetchPartitionStatus,
-        replicaManager = this,
-        quota = quota,
-        responseCallback = responseCallback
-      )
-
-      // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
-      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp) }
-
-      // try to complete the request immediately, otherwise put it into the purgatory;
-      // this is because while the delayed fetch operation is being created, new requests
-      // may arrive and hence make this operation completable.
-      delayedFetchPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys)
+
+      if (remoteFetchInfo.isPresent) {
+        val key = new TopicPartitionOperationKey(remoteFetchInfo.get.topicPartition.topic(), remoteFetchInfo.get.topicPartition.partition())
+        val remoteFetchResult = new CompletableFuture[RemoteLogReadResult]
+        var remoteFetchTask: Future[Void] = null
+        try {
+          remoteFetchTask = remoteLogManager.get.asyncRead(remoteFetchInfo.get, (result: RemoteLogReadResult) => {
+            remoteFetchResult.complete(result)
+            delayedRemoteFetchPurgatory.checkAndComplete(key)
+          })
+        } catch {
+          // if the task queue of remote storage reader thread pool is full, return what we currently have
+          // (the data read from local log segment for the other topic-partitions) and an error for the topic-partition that
+          // we couldn't read from remote storage
+          case e: RejectedExecutionException =>
+            val fetchPartitionData = logReadResults.map { case (tp, result) =>
+              val r = {
+                if (tp.topicPartition().equals(remoteFetchInfo.get.topicPartition))
+                  createLogReadResult(e)

Review Comment:
   Is this `RejectedExecutionException` propagated to the Consumer fetch? If yes, is this a change in the existing interface with the consumer? (please correct me if I am wrong but I am not aware of consumer handling or expecting `RejectedExecutionException` today.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -147,6 +164,11 @@ public RemoteLogManager(RemoteLogManagerConfig rlmConfig,
         indexCache = new RemoteIndexCache(1024, remoteLogStorageManager, logDir);
         delayInMs = rlmConfig.remoteLogManagerTaskIntervalMs();
         rlmScheduledThreadPool = new RLMScheduledThreadPool(rlmConfig.remoteLogManagerThreadPoolSize());
+        remoteStorageReaderThreadPool = new RemoteStorageThreadPool(

Review Comment:
   important
   
   We are not shutting this down in `RemoteLogManager#close()`



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,25 +622,208 @@ public String toString() {
         }
     }
 
-    long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws RemoteStorageException {
-        Optional<Long> offset = Optional.empty();
-        Optional<UnifiedLog> maybeLog = fetchLog.apply(topicIdPartition.topicPartition());
-        if (maybeLog.isPresent()) {
-            UnifiedLog log = maybeLog.get();
-            Option<LeaderEpochFileCache> maybeLeaderEpochFileCache = log.leaderEpochCache();
-            if (maybeLeaderEpochFileCache.isDefined()) {
-                LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get();
-                OptionalInt epoch = cache.latestEpoch();
-                while (!offset.isPresent() && epoch.isPresent()) {
-                    offset = remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch.getAsInt());
-                    epoch = cache.previousEpoch(epoch.getAsInt());
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadataOptional = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadataOptional.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        RemoteLogSegmentMetadata remoteLogSegmentMetadata = rlsMetadataOptional.get();
+        int startPos = lookupPositionForOffset(remoteLogSegmentMetadata, offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            // An empty record is sent instead of an incomplete batch when there is no minimum-one-message constraint
+            // and for FetchRequest version 3 and above and the first batch size is more than maximum bytes that can be sent.
+            if (!remoteStorageFetchInfo.minOneMessage &&
+                    !remoteStorageFetchInfo.hardMaxBytesLimit &&
+                    firstBatch.sizeInBytes() > maxBytes) {

Review Comment:
   This and the above `firstBatch == null` condition can be combined since they produce the same result.



##########
core/src/main/scala/kafka/server/DelayedRemoteFetch.scala:
##########
@@ -0,0 +1,112 @@
+/**
+ * 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 org.apache.kafka.common.TopicIdPartition
+import org.apache.kafka.common.errors._
+import org.apache.kafka.storage.internals.log.{FetchParams, FetchPartitionData, LogOffsetMetadata, RemoteLogReadResult, RemoteStorageFetchInfo}
+
+import java.util.concurrent.{CompletableFuture, Future}
+import java.util.{Optional, OptionalInt, OptionalLong}
+import scala.collection._
+
+/**
+ * A remote fetch operation that can be created by the replica manager and watched
+ * in the remote fetch operation purgatory
+ */
+class DelayedRemoteFetch(remoteFetchTask: Future[Void],
+                         remoteFetchResult: CompletableFuture[RemoteLogReadResult],
+                         remoteFetchInfo: RemoteStorageFetchInfo,
+                         fetchPartitionStatus: Seq[(TopicIdPartition, FetchPartitionStatus)],
+                         fetchParams: FetchParams,
+                         localReadResults: Seq[(TopicIdPartition, LogReadResult)],
+                         replicaManager: ReplicaManager,
+                         responseCallback: Seq[(TopicIdPartition, FetchPartitionData)] => Unit)
+  extends DelayedOperation(fetchParams.maxWaitMs) {
+
+  /**
+   * The operation can be completed if:
+   *
+   * Case a: This broker is no longer the leader of the partition it tries to fetch
+   * Case b: This broker does not know the partition it tries to fetch
+   * Case c: The remote storage read request completed (succeeded or failed)
+   * Case d: The partition is in an offline log directory on this broker
+   *
+   * Upon completion, should return whatever data is available for each valid partition
+   */
+  override def tryComplete(): Boolean = {
+    fetchPartitionStatus.foreach {
+      case (topicPartition, fetchStatus) =>
+        val fetchOffset = fetchStatus.startOffsetMetadata
+        try {
+          if (fetchOffset != LogOffsetMetadata.UNKNOWN_OFFSET_METADATA) {
+            replicaManager.getPartitionOrException(topicPartition.topicPartition())
+          }
+        } catch {
+          case _: KafkaStorageException => // Case d
+            debug(s"Partition $topicPartition is in an offline log directory, satisfy $fetchParams immediately")
+            return forceComplete()
+          case _: UnknownTopicOrPartitionException => // Case b
+            debug(s"Broker no longer knows of partition $topicPartition, satisfy $fetchParams immediately")
+            return forceComplete()
+          case _: NotLeaderOrFollowerException =>  // Case a
+            debug("Broker is no longer the leader or follower of %s, satisfy %s immediately".format(topicPartition, fetchParams))
+            return forceComplete()
+        }
+    }
+    if (remoteFetchResult.isDone) // Case c
+      forceComplete()
+    else
+      false
+  }
+
+  override def onExpiration():Unit = {
+    // cancel the remote storage read task, if it has not been executed yet
+    remoteFetchTask.cancel(false)

Review Comment:
   1. Why aren't we interrupting this with a true argument?
   2. We should probably check the return and if we are unable to cancel, we should log a debug statement here.



##########
core/src/main/scala/kafka/server/DelayedRemoteFetch.scala:
##########
@@ -0,0 +1,112 @@
+/**
+ * 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 org.apache.kafka.common.TopicIdPartition
+import org.apache.kafka.common.errors._
+import org.apache.kafka.storage.internals.log.{FetchParams, FetchPartitionData, LogOffsetMetadata, RemoteLogReadResult, RemoteStorageFetchInfo}
+
+import java.util.concurrent.{CompletableFuture, Future}
+import java.util.{Optional, OptionalInt, OptionalLong}
+import scala.collection._
+
+/**
+ * A remote fetch operation that can be created by the replica manager and watched
+ * in the remote fetch operation purgatory
+ */
+class DelayedRemoteFetch(remoteFetchTask: Future[Void],
+                         remoteFetchResult: CompletableFuture[RemoteLogReadResult],
+                         remoteFetchInfo: RemoteStorageFetchInfo,
+                         fetchPartitionStatus: Seq[(TopicIdPartition, FetchPartitionStatus)],
+                         fetchParams: FetchParams,
+                         localReadResults: Seq[(TopicIdPartition, LogReadResult)],
+                         replicaManager: ReplicaManager,
+                         responseCallback: Seq[(TopicIdPartition, FetchPartitionData)] => Unit)
+  extends DelayedOperation(fetchParams.maxWaitMs) {
+
+  /**
+   * The operation can be completed if:
+   *
+   * Case a: This broker is no longer the leader of the partition it tries to fetch
+   * Case b: This broker does not know the partition it tries to fetch
+   * Case c: The remote storage read request completed (succeeded or failed)
+   * Case d: The partition is in an offline log directory on this broker
+   *
+   * Upon completion, should return whatever data is available for each valid partition
+   */
+  override def tryComplete(): Boolean = {
+    fetchPartitionStatus.foreach {
+      case (topicPartition, fetchStatus) =>
+        val fetchOffset = fetchStatus.startOffsetMetadata
+        try {
+          if (fetchOffset != LogOffsetMetadata.UNKNOWN_OFFSET_METADATA) {
+            replicaManager.getPartitionOrException(topicPartition.topicPartition())
+          }
+        } catch {
+          case _: KafkaStorageException => // Case d
+            debug(s"Partition $topicPartition is in an offline log directory, satisfy $fetchParams immediately")
+            return forceComplete()
+          case _: UnknownTopicOrPartitionException => // Case b
+            debug(s"Broker no longer knows of partition $topicPartition, satisfy $fetchParams immediately")
+            return forceComplete()
+          case _: NotLeaderOrFollowerException =>  // Case a
+            debug("Broker is no longer the leader or follower of %s, satisfy %s immediately".format(topicPartition, fetchParams))
+            return forceComplete()
+        }
+    }
+    if (remoteFetchResult.isDone) // Case c
+      forceComplete()
+    else
+      false
+  }
+
+  override def onExpiration():Unit = {

Review Comment:
   non blocking comment
   
   we probably need metrics on expiration of delated remote fetch here. 



##########
core/src/main/scala/kafka/server/DelayedRemoteFetch.scala:
##########
@@ -0,0 +1,112 @@
+/**
+ * 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 org.apache.kafka.common.TopicIdPartition
+import org.apache.kafka.common.errors._
+import org.apache.kafka.storage.internals.log.{FetchParams, FetchPartitionData, LogOffsetMetadata, RemoteLogReadResult, RemoteStorageFetchInfo}
+
+import java.util.concurrent.{CompletableFuture, Future}
+import java.util.{Optional, OptionalInt, OptionalLong}
+import scala.collection._
+
+/**
+ * A remote fetch operation that can be created by the replica manager and watched
+ * in the remote fetch operation purgatory
+ */
+class DelayedRemoteFetch(remoteFetchTask: Future[Void],
+                         remoteFetchResult: CompletableFuture[RemoteLogReadResult],
+                         remoteFetchInfo: RemoteStorageFetchInfo,
+                         fetchPartitionStatus: Seq[(TopicIdPartition, FetchPartitionStatus)],
+                         fetchParams: FetchParams,
+                         localReadResults: Seq[(TopicIdPartition, LogReadResult)],
+                         replicaManager: ReplicaManager,
+                         responseCallback: Seq[(TopicIdPartition, FetchPartitionData)] => Unit)
+  extends DelayedOperation(fetchParams.maxWaitMs) {
+
+  /**
+   * The operation can be completed if:
+   *
+   * Case a: This broker is no longer the leader of the partition it tries to fetch
+   * Case b: This broker does not know the partition it tries to fetch
+   * Case c: The remote storage read request completed (succeeded or failed)
+   * Case d: The partition is in an offline log directory on this broker
+   *
+   * Upon completion, should return whatever data is available for each valid partition
+   */
+  override def tryComplete(): Boolean = {
+    fetchPartitionStatus.foreach {
+      case (topicPartition, fetchStatus) =>
+        val fetchOffset = fetchStatus.startOffsetMetadata
+        try {
+          if (fetchOffset != LogOffsetMetadata.UNKNOWN_OFFSET_METADATA) {
+            replicaManager.getPartitionOrException(topicPartition.topicPartition())
+          }
+        } catch {
+          case _: KafkaStorageException => // Case d
+            debug(s"Partition $topicPartition is in an offline log directory, satisfy $fetchParams immediately")
+            return forceComplete()
+          case _: UnknownTopicOrPartitionException => // Case b
+            debug(s"Broker no longer knows of partition $topicPartition, satisfy $fetchParams immediately")
+            return forceComplete()
+          case _: NotLeaderOrFollowerException =>  // Case a
+            debug("Broker is no longer the leader or follower of %s, satisfy %s immediately".format(topicPartition, fetchParams))
+            return forceComplete()
+        }
+    }
+    if (remoteFetchResult.isDone) // Case c
+      forceComplete()
+    else
+      false
+  }
+
+  override def onExpiration():Unit = {
+    // cancel the remote storage read task, if it has not been executed yet
+    remoteFetchTask.cancel(false)
+  }
+
+  /**
+   * Upon completion, read whatever data is available and pass to the complete callback
+   */
+  override def onComplete():Unit = {
+    val fetchPartitionData = localReadResults.map { case (tp, result) =>
+      if (tp.topicPartition().equals(remoteFetchInfo.topicPartition) && remoteFetchResult.isDone
+        && result.exception.isEmpty && result.info.delayedRemoteStorageFetch.isPresent) {

Review Comment:
   nit
   
   s/result.Exception.isEmpty/(result.error == Errors.NONE)
   
   Asking because it makes the code more clear since we are using result.error in the else to populate the reponse.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1181519032


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,25 +622,208 @@ public String toString() {
         }
     }
 
-    long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws RemoteStorageException {
-        Optional<Long> offset = Optional.empty();
-        Optional<UnifiedLog> maybeLog = fetchLog.apply(topicIdPartition.topicPartition());
-        if (maybeLog.isPresent()) {
-            UnifiedLog log = maybeLog.get();
-            Option<LeaderEpochFileCache> maybeLeaderEpochFileCache = log.leaderEpochCache();
-            if (maybeLeaderEpochFileCache.isDefined()) {
-                LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get();
-                OptionalInt epoch = cache.latestEpoch();
-                while (!offset.isPresent() && epoch.isPresent()) {
-                    offset = remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch.getAsInt());
-                    epoch = cache.previousEpoch(epoch.getAsInt());
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadataOptional = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadataOptional.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " + offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not exist in remote tier.");
+        }
+
+        RemoteLogSegmentMetadata remoteLogSegmentMetadata = rlsMetadataOptional.get();
+        int startPos = lookupPositionForOffset(remoteLogSegmentMetadata, offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is greater than or equal to the target offset
+            remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos);
+            RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? Optional.of(Collections.emptyList()) : Optional.empty());
+
+            // An empty record is sent instead of an incomplete batch when there is no minimum-one-message constraint
+            // and for FetchRequest version 3 and above and the first batch size is more than maximum bytes that can be sent.
+            if (!remoteStorageFetchInfo.minOneMessage &&
+                    !remoteStorageFetchInfo.hardMaxBytesLimit &&
+                    firstBatch.sizeInBytes() > maxBytes) {

Review Comment:
   After the recent change of extracting `firstBatch.sizeInBytes()`, the existing code looks simpler. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on PR #13535:
URL: https://github.com/apache/kafka/pull/13535#issuecomment-1558604752

   @satishd `testResponseIncludesLeaderEpoch` fails locally. Does it pass for you? It does not seem to be related to slow CI.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1203664360


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1121,21 +1154,69 @@ class ReplicaManager(val config: KafkaConfig,
     partition.legacyFetchOffsetsForTimestamp(timestamp, maxNumOffsets, isFromConsumer, fetchOnlyFromLeader)
   }
 
+  /**
+   * Returns [[LogReadResult]] with error if a task for RemoteStorageFetchInfo could not be scheduled successfully
+   * else returns [[None]].
+   */
+  private def processRemoteFetch(remoteFetchInfo: RemoteStorageFetchInfo,
+                                 params: FetchParams,
+                                 responseCallback: Seq[(TopicIdPartition, FetchPartitionData)] => Unit,
+                                 logReadResults: Seq[(TopicIdPartition, LogReadResult)],
+                                 fetchPartitionStatus: Seq[(TopicIdPartition, FetchPartitionStatus)]): Option[LogReadResult] = {
+    val key = new TopicPartitionOperationKey(remoteFetchInfo.topicPartition.topic(), remoteFetchInfo.topicPartition.partition())
+    val remoteFetchResult = new CompletableFuture[RemoteLogReadResult]
+    var remoteFetchTask: Future[Void] = null
+    try {
+      remoteFetchTask = remoteLogManager.get.asyncRead(remoteFetchInfo, (result: RemoteLogReadResult) => {
+        remoteFetchResult.complete(result)
+        delayedRemoteFetchPurgatory.checkAndComplete(key)
+      })
+    } catch {
+      case e: RejectedExecutionException =>
+        // Return the error if any in scheduling the remote fetch task
+        return Some(createLogReadResult(e))
+    }
+
+    val remoteFetch = new DelayedRemoteFetch(remoteFetchTask, remoteFetchResult, remoteFetchInfo,
+      fetchPartitionStatus, params, logReadResults, this, responseCallback)
+
+    delayedRemoteFetchPurgatory.tryCompleteElseWatch(remoteFetch, Seq(key))
+    None
+  }
+
+  private def buildPartitionToFetchPartitionData(logReadResults: Seq[(TopicIdPartition, LogReadResult)],
+                                                 remoteFetchTopicPartition: TopicPartition,
+                                                 error: LogReadResult): Seq[(TopicIdPartition, FetchPartitionData)] = {
+    logReadResults.map { case (tp, result) =>
+      val fetchPartitionData = {
+        if (tp.topicPartition().equals(remoteFetchTopicPartition))
+          error
+        else
+          result
+      }.toFetchPartitionData(false)
+
+      tp -> fetchPartitionData
+    }
+  }
+
   /**
    * Fetch messages from a replica, and wait until enough data can be fetched and return;
    * the callback function will be triggered either when timeout or required fetch info is satisfied.
    * Consumers may fetch from any replica, but followers can only fetch from the leader.
    */
-  def fetchMessages(
-    params: FetchParams,
-    fetchInfos: Seq[(TopicIdPartition, PartitionData)],
-    quota: ReplicaQuota,
-    responseCallback: Seq[(TopicIdPartition, FetchPartitionData)] => Unit
-  ): Unit = {
+  def fetchMessages(params: FetchParams,

Review Comment:
   A small comment. We should avoid completely changing the code style without reasons. The format of the method was not a mistake. It is the format that we mainly used in this class nowadays.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org