Skip to content

Commit 344d8a6

Browse files
authored
KAFKA-15859 Make RemoteListOffsets call an async operation (#16602)
This is the part-2 of the KIP-1075 To find the offset for a given timestamp, ListOffsets API is used by the client. When the topic is enabled with remote storage, then we have to fetch the remote indexes such as offset-index and time-index to serve the query. Also, the ListOffsets request can contain the query for multiple topics/partitions. The time taken to read the indexes from remote storage is non-deterministic and the query is handled by the request-handler threads. If there are multiple LIST_OFFSETS queries and most of the request-handler threads are busy in reading the data from remote storage, then the other high-priority requests such as FETCH and PRODUCE might starve and be queued. This can lead to higher latency in producing/consuming messages. In this patch, we have introduced a delayed operation for remote list-offsets call. If the timestamp need to be searched in the remote-storage, then the request-handler threads will pass-on the request to the remote-log-reader threads. And, the request gets handled in asynchronous fashion. Covered the patch with unit and integration tests. Reviewers: Satish Duggana <[email protected]>, Luke Chen <[email protected]>, Chia-Ping Tsai <[email protected]>
1 parent e1f11c6 commit 344d8a6

24 files changed

+1271
-202
lines changed

core/src/main/java/kafka/log/remote/RemoteLogManager.java

+28
Original file line numberDiff line numberDiff line change
@@ -18,8 +18,12 @@
1818

1919
import kafka.cluster.EndPoint;
2020
import kafka.cluster.Partition;
21+
import kafka.log.AsyncOffsetReadFutureHolder;
2122
import kafka.log.UnifiedLog;
23+
import kafka.server.DelayedOperationPurgatory;
24+
import kafka.server.DelayedRemoteListOffsets;
2225
import kafka.server.StopPartition;
26+
import kafka.server.TopicPartitionOperationKey;
2327

2428
import org.apache.kafka.common.KafkaException;
2529
import org.apache.kafka.common.TopicIdPartition;
@@ -132,11 +136,13 @@
132136
import java.util.function.Consumer;
133137
import java.util.function.Function;
134138
import java.util.function.Predicate;
139+
import java.util.function.Supplier;
135140
import java.util.stream.Collectors;
136141
import java.util.stream.Stream;
137142

138143
import scala.Option;
139144
import scala.collection.JavaConverters;
145+
import scala.util.Either;
140146

141147
import static org.apache.kafka.server.config.ServerLogConfigs.LOG_DIR_CONFIG;
142148
import static org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX;
@@ -200,6 +206,7 @@ public class RemoteLogManager implements Closeable {
200206

201207
private volatile boolean remoteLogManagerConfigured = false;
202208
private final Timer remoteReadTimer;
209+
private DelayedOperationPurgatory<DelayedRemoteListOffsets> delayedRemoteListOffsetsPurgatory;
203210

204211
/**
205212
* Creates RemoteLogManager instance with the given arguments.
@@ -263,6 +270,10 @@ public RemoteLogManager(RemoteLogManagerConfig rlmConfig,
263270
);
264271
}
265272

273+
public void setDelayedOperationPurgatory(DelayedOperationPurgatory<DelayedRemoteListOffsets> delayedRemoteListOffsetsPurgatory) {
274+
this.delayedRemoteListOffsetsPurgatory = delayedRemoteListOffsetsPurgatory;
275+
}
276+
266277
public void resizeCacheSize(long remoteLogIndexFileCacheSize) {
267278
indexCache.resizeCacheSize(remoteLogIndexFileCacheSize);
268279
}
@@ -620,6 +631,23 @@ private Optional<Integer> maybeLeaderEpoch(int leaderEpoch) {
620631
return leaderEpoch == RecordBatch.NO_PARTITION_LEADER_EPOCH ? Optional.empty() : Optional.of(leaderEpoch);
621632
}
622633

634+
public AsyncOffsetReadFutureHolder<Either<Exception, Option<FileRecords.TimestampAndOffset>>> asyncOffsetRead(
635+
TopicPartition topicPartition,
636+
Long timestamp,
637+
Long startingOffset,
638+
LeaderEpochFileCache leaderEpochCache,
639+
Supplier<Option<FileRecords.TimestampAndOffset>> searchLocalLog) {
640+
CompletableFuture<Either<Exception, Option<FileRecords.TimestampAndOffset>>> taskFuture = new CompletableFuture<>();
641+
Future<Void> jobFuture = remoteStorageReaderThreadPool.submit(
642+
new RemoteLogOffsetReader(this, topicPartition, timestamp, startingOffset, leaderEpochCache, searchLocalLog, result -> {
643+
TopicPartitionOperationKey key = new TopicPartitionOperationKey(topicPartition.topic(), topicPartition.partition());
644+
taskFuture.complete(result);
645+
delayedRemoteListOffsetsPurgatory.checkAndComplete(key);
646+
})
647+
);
648+
return new AsyncOffsetReadFutureHolder<>(jobFuture, taskFuture);
649+
}
650+
623651
/**
624652
* Search the message offset in the remote storage based on timestamp and offset.
625653
* <p>
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,79 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
package kafka.log.remote;
18+
19+
import org.apache.kafka.common.TopicPartition;
20+
import org.apache.kafka.common.record.FileRecords;
21+
import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache;
22+
23+
import org.slf4j.Logger;
24+
import org.slf4j.LoggerFactory;
25+
26+
import java.util.concurrent.Callable;
27+
import java.util.function.Consumer;
28+
import java.util.function.Supplier;
29+
30+
import scala.Option;
31+
import scala.compat.java8.OptionConverters;
32+
import scala.util.Either;
33+
import scala.util.Left;
34+
import scala.util.Right;
35+
36+
public class RemoteLogOffsetReader implements Callable<Void> {
37+
private static final Logger LOGGER = LoggerFactory.getLogger(RemoteLogOffsetReader.class);
38+
private final RemoteLogManager rlm;
39+
private final TopicPartition tp;
40+
private final long timestamp;
41+
private final long startingOffset;
42+
private final LeaderEpochFileCache leaderEpochCache;
43+
private final Supplier<Option<FileRecords.TimestampAndOffset>> searchInLocalLog;
44+
private final Consumer<Either<Exception, Option<FileRecords.TimestampAndOffset>>> callback;
45+
46+
public RemoteLogOffsetReader(RemoteLogManager rlm,
47+
TopicPartition tp,
48+
long timestamp,
49+
long startingOffset,
50+
LeaderEpochFileCache leaderEpochCache,
51+
Supplier<Option<FileRecords.TimestampAndOffset>> searchInLocalLog,
52+
Consumer<Either<Exception, Option<FileRecords.TimestampAndOffset>>> callback) {
53+
this.rlm = rlm;
54+
this.tp = tp;
55+
this.timestamp = timestamp;
56+
this.startingOffset = startingOffset;
57+
this.leaderEpochCache = leaderEpochCache;
58+
this.searchInLocalLog = searchInLocalLog;
59+
this.callback = callback;
60+
}
61+
62+
@Override
63+
public Void call() throws Exception {
64+
Either<Exception, Option<FileRecords.TimestampAndOffset>> result;
65+
try {
66+
// If it is not found in remote storage, then search in the local storage starting with local log start offset.
67+
Option<FileRecords.TimestampAndOffset> timestampAndOffsetOpt =
68+
OptionConverters.toScala(rlm.findOffsetByTimestamp(tp, timestamp, startingOffset, leaderEpochCache))
69+
.orElse(searchInLocalLog::get);
70+
result = Right.apply(timestampAndOffsetOpt);
71+
} catch (Exception e) {
72+
// NOTE: All the exceptions from the secondary storage are catched instead of only the KafkaException.
73+
LOGGER.error("Error occurred while reading the remote log offset for {}", tp, e);
74+
result = Left.apply(e);
75+
}
76+
callback.accept(result);
77+
return null;
78+
}
79+
}

core/src/main/java/kafka/server/builders/ReplicaManagerBuilder.java

+3
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@
2727
import kafka.server.DelayedOperationPurgatory;
2828
import kafka.server.DelayedProduce;
2929
import kafka.server.DelayedRemoteFetch;
30+
import kafka.server.DelayedRemoteListOffsets;
3031
import kafka.server.KafkaConfig;
3132
import kafka.server.MetadataCache;
3233
import kafka.server.QuotaFactory.QuotaManagers;
@@ -66,6 +67,7 @@ public class ReplicaManagerBuilder {
6667
private Optional<DelayedOperationPurgatory<DelayedDeleteRecords>> delayedDeleteRecordsPurgatory = Optional.empty();
6768
private Optional<DelayedOperationPurgatory<DelayedElectLeader>> delayedElectLeaderPurgatory = Optional.empty();
6869
private Optional<DelayedOperationPurgatory<DelayedRemoteFetch>> delayedRemoteFetchPurgatory = Optional.empty();
70+
private Optional<DelayedOperationPurgatory<DelayedRemoteListOffsets>> delayedRemoteListOffsetsPurgatory = Optional.empty();
6971
private Optional<String> threadNamePrefix = Optional.empty();
7072
private Long brokerEpoch = -1L;
7173
private Optional<AddPartitionsToTxnManager> addPartitionsToTxnManager = Optional.empty();
@@ -210,6 +212,7 @@ public ReplicaManager build() {
210212
OptionConverters.toScala(delayedDeleteRecordsPurgatory),
211213
OptionConverters.toScala(delayedElectLeaderPurgatory),
212214
OptionConverters.toScala(delayedRemoteFetchPurgatory),
215+
OptionConverters.toScala(delayedRemoteListOffsetsPurgatory),
213216
OptionConverters.toScala(threadNamePrefix),
214217
() -> brokerEpoch,
215218
OptionConverters.toScala(addPartitionsToTxnManager),

core/src/main/java/kafka/server/share/ShareFetchUtils.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -108,7 +108,7 @@ static long offsetForEarliestTimestamp(TopicIdPartition topicIdPartition, Replic
108108
// Isolation level is only required when reading from the latest offset hence use Option.empty() for now.
109109
Option<FileRecords.TimestampAndOffset> timestampAndOffset = replicaManager.fetchOffsetForTimestamp(
110110
topicIdPartition.topicPartition(), ListOffsetsRequest.EARLIEST_TIMESTAMP, Option.empty(),
111-
Optional.empty(), true);
111+
Optional.empty(), true).timestampAndOffsetOpt();
112112
return timestampAndOffset.isEmpty() ? (long) 0 : timestampAndOffset.get().offset;
113113
}
114114
}

core/src/main/scala/kafka/cluster/Partition.scala

+10-6
Original file line numberDiff line numberDiff line change
@@ -1576,7 +1576,7 @@ class Partition(val topicPartition: TopicPartition,
15761576
isolationLevel: Option[IsolationLevel],
15771577
currentLeaderEpoch: Optional[Integer],
15781578
fetchOnlyFromLeader: Boolean,
1579-
remoteLogManager: Option[RemoteLogManager] = None): Option[TimestampAndOffset] = inReadLock(leaderIsrUpdateLock) {
1579+
remoteLogManager: Option[RemoteLogManager] = None): OffsetResultHolder = inReadLock(leaderIsrUpdateLock) {
15801580
// decide whether to only fetch from leader
15811581
val localLog = localLogWithEpochOrThrow(currentLeaderEpoch, fetchOnlyFromLeader)
15821582

@@ -1601,21 +1601,25 @@ class Partition(val topicPartition: TopicPartition,
16011601
s"high watermark (${localLog.highWatermark}) is lagging behind the " +
16021602
s"start offset from the beginning of this epoch ($epochStart)."))
16031603

1604-
def getOffsetByTimestamp: Option[TimestampAndOffset] = {
1605-
logManager.getLog(topicPartition).flatMap(log => log.fetchOffsetByTimestamp(timestamp, remoteLogManager))
1604+
def getOffsetByTimestamp: OffsetResultHolder = {
1605+
logManager.getLog(topicPartition)
1606+
.map(log => log.fetchOffsetByTimestamp(timestamp, remoteLogManager))
1607+
.getOrElse(OffsetResultHolder(timestampAndOffsetOpt = None))
16061608
}
16071609

16081610
// If we're in the lagging HW state after a leader election, throw OffsetNotAvailable for "latest" offset
16091611
// or for a timestamp lookup that is beyond the last fetchable offset.
16101612
timestamp match {
16111613
case ListOffsetsRequest.LATEST_TIMESTAMP =>
16121614
maybeOffsetsError.map(e => throw e)
1613-
.orElse(Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, lastFetchableOffset, Optional.of(leaderEpoch))))
1615+
.getOrElse(OffsetResultHolder(Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, lastFetchableOffset, Optional.of(leaderEpoch)))))
16141616
case ListOffsetsRequest.EARLIEST_TIMESTAMP | ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP =>
16151617
getOffsetByTimestamp
16161618
case _ =>
1617-
getOffsetByTimestamp.filter(timestampAndOffset => timestampAndOffset.offset < lastFetchableOffset)
1618-
.orElse(maybeOffsetsError.map(e => throw e))
1619+
val offsetResultHolder = getOffsetByTimestamp
1620+
offsetResultHolder.maybeOffsetsError = maybeOffsetsError
1621+
offsetResultHolder.lastFetchableOffset = Some(lastFetchableOffset)
1622+
offsetResultHolder
16191623
}
16201624
}
16211625

Original file line numberDiff line numberDiff line change
@@ -0,0 +1,38 @@
1+
/**
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
package kafka.log
18+
19+
import org.apache.kafka.common.errors.ApiException
20+
import org.apache.kafka.common.record.FileRecords.TimestampAndOffset
21+
22+
import java.util.concurrent.{CompletableFuture, Future}
23+
24+
case class OffsetResultHolder(timestampAndOffsetOpt: Option[TimestampAndOffset],
25+
futureHolderOpt: Option[AsyncOffsetReadFutureHolder[Either[Exception, Option[TimestampAndOffset]]]] = None) {
26+
27+
var maybeOffsetsError: Option[ApiException] = None
28+
var lastFetchableOffset: Option[Long] = None
29+
}
30+
31+
/**
32+
* A remote log offset read task future holder. It contains two futures:
33+
* 1. JobFuture - Use this future to cancel the running job.
34+
* 2. TaskFuture - Use this future to get the result of the job/computation.
35+
*/
36+
case class AsyncOffsetReadFutureHolder[T](jobFuture: Future[Void], taskFuture: CompletableFuture[T]) {
37+
38+
}

core/src/main/scala/kafka/log/UnifiedLog.scala

+21-16
Original file line numberDiff line numberDiff line change
@@ -1263,7 +1263,7 @@ class UnifiedLog(@volatile var logStartOffset: Long,
12631263
* None if no such message is found.
12641264
*/
12651265
@nowarn("cat=deprecation")
1266-
def fetchOffsetByTimestamp(targetTimestamp: Long, remoteLogManager: Option[RemoteLogManager] = None): Option[TimestampAndOffset] = {
1266+
def fetchOffsetByTimestamp(targetTimestamp: Long, remoteLogManager: Option[RemoteLogManager] = None): OffsetResultHolder = {
12671267
maybeHandleIOException(s"Error while fetching offset by timestamp for $topicPartition in dir ${dir.getParent}") {
12681268
debug(s"Searching offset for timestamp $targetTimestamp")
12691269

@@ -1285,7 +1285,7 @@ class UnifiedLog(@volatile var logStartOffset: Long,
12851285
Optional.of[Integer](earliestEpochEntry.get().epoch)
12861286
} else Optional.empty[Integer]()
12871287

1288-
Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, logStartOffset, epochOpt))
1288+
OffsetResultHolder(Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, logStartOffset, epochOpt)))
12891289
} else if (targetTimestamp == ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP) {
12901290
val curLocalLogStartOffset = localLogStartOffset()
12911291

@@ -1297,15 +1297,15 @@ class UnifiedLog(@volatile var logStartOffset: Long,
12971297
Optional.empty()
12981298
}
12991299

1300-
Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, curLocalLogStartOffset, epochResult))
1300+
OffsetResultHolder(Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, curLocalLogStartOffset, epochResult)))
13011301
} else if (targetTimestamp == ListOffsetsRequest.LATEST_TIMESTAMP) {
13021302
val epoch = leaderEpochCache match {
13031303
case Some(cache) =>
13041304
val latestEpoch = cache.latestEpoch()
13051305
if (latestEpoch.isPresent) Optional.of[Integer](latestEpoch.getAsInt) else Optional.empty[Integer]()
13061306
case None => Optional.empty[Integer]()
13071307
}
1308-
Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, logEndOffset, epoch))
1308+
OffsetResultHolder(Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, logEndOffset, epoch)))
13091309
} else if (targetTimestamp == ListOffsetsRequest.LATEST_TIERED_TIMESTAMP) {
13101310
if (remoteLogEnabled()) {
13111311
val curHighestRemoteOffset = highestOffsetInRemoteStorage()
@@ -1324,9 +1324,9 @@ class UnifiedLog(@volatile var logStartOffset: Long,
13241324
Optional.empty()
13251325
}
13261326

1327-
Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, curHighestRemoteOffset, epochResult))
1327+
OffsetResultHolder(Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, curHighestRemoteOffset, epochResult)))
13281328
} else {
1329-
Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, -1L, Optional.of(-1)))
1329+
OffsetResultHolder(Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, -1L, Optional.of(-1))))
13301330
}
13311331
} else if (targetTimestamp == ListOffsetsRequest.MAX_TIMESTAMP) {
13321332
// Cache to avoid race conditions. `toBuffer` is faster than most alternatives and provides
@@ -1336,34 +1336,39 @@ class UnifiedLog(@volatile var logStartOffset: Long,
13361336
val maxTimestampSoFar = latestTimestampSegment.readMaxTimestampAndOffsetSoFar
13371337
// lookup the position of batch to avoid extra I/O
13381338
val position = latestTimestampSegment.offsetIndex.lookup(maxTimestampSoFar.offset)
1339-
latestTimestampSegment.log.batchesFrom(position.position).asScala
1339+
val timestampAndOffsetOpt = latestTimestampSegment.log.batchesFrom(position.position).asScala
13401340
.find(_.maxTimestamp() == maxTimestampSoFar.timestamp)
13411341
.flatMap(batch => batch.offsetOfMaxTimestamp().asScala.map(new TimestampAndOffset(batch.maxTimestamp(), _,
13421342
Optional.of[Integer](batch.partitionLeaderEpoch()).filter(_ >= 0))))
1343+
OffsetResultHolder(timestampAndOffsetOpt)
13431344
} else {
13441345
// We need to search the first segment whose largest timestamp is >= the target timestamp if there is one.
1345-
if (remoteLogEnabled()) {
1346+
if (remoteLogEnabled() && !isEmpty) {
13461347
if (remoteLogManager.isEmpty) {
13471348
throw new KafkaException("RemoteLogManager is empty even though the remote log storage is enabled.")
13481349
}
13491350
if (recordVersion.value < RecordVersion.V2.value) {
13501351
throw new KafkaException("Tiered storage is supported only with versions supporting leader epochs, that means RecordVersion must be >= 2.")
13511352
}
13521353

1353-
val remoteOffset = remoteLogManager.get.findOffsetByTimestamp(topicPartition, targetTimestamp, logStartOffset, leaderEpochCache.get)
1354-
if (remoteOffset.isPresent) {
1355-
remoteOffset.asScala
1356-
} else {
1357-
// If it is not found in remote log storage, search in the local log storage from local log start offset.
1358-
searchOffsetInLocalLog(targetTimestamp, localLogStartOffset())
1359-
}
1354+
val asyncOffsetReadFutureHolder = remoteLogManager.get.asyncOffsetRead(topicPartition, targetTimestamp,
1355+
logStartOffset, leaderEpochCache.get, () => searchOffsetInLocalLog(targetTimestamp, localLogStartOffset()))
1356+
OffsetResultHolder(None, Some(asyncOffsetReadFutureHolder))
13601357
} else {
1361-
searchOffsetInLocalLog(targetTimestamp, logStartOffset)
1358+
OffsetResultHolder(searchOffsetInLocalLog(targetTimestamp, logStartOffset))
13621359
}
13631360
}
13641361
}
13651362
}
13661363

1364+
/**
1365+
* Checks if the log is empty.
1366+
* @return Returns True when the log is empty. Otherwise, false.
1367+
*/
1368+
private[log] def isEmpty = {
1369+
logStartOffset == logEndOffset
1370+
}
1371+
13671372
private def searchOffsetInLocalLog(targetTimestamp: Long, startOffset: Long): Option[TimestampAndOffset] = {
13681373
// Cache to avoid race conditions. `toBuffer` is faster than most alternatives and provides
13691374
// constant time access while being safe to use with concurrent collections unlike `toArray`.

0 commit comments

Comments
 (0)