Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -189,6 +189,11 @@ public CompletableFuture<Stream> createAndOpenStream(CreateStreamOptions options
return failureHandle(streamClient.createAndOpenStream(options).thenApplyAsync(rst -> rst, streamManagerCallbackExecutors));
}

@Override
public CompletableFuture<Void> preWarmStream(long streamId) {
return streamClient.preWarmStream(streamId);
}

@Override
public CompletableFuture<Stream> openStream(long streamId, OpenStreamOptions options) {
return failureHandle(streamClient.openStream(streamId, options).thenApplyAsync(rst -> rst, streamManagerCallbackExecutors));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -709,6 +709,7 @@ object ElasticLog extends Logging {
stream
} else {
val metaStreamId = Unpooled.wrappedBuffer(value.get()).readLong()
client.streamClient().preWarmStream(metaStreamId)
val awaitCostMs = awaitStreamReadyForOpen(openStreamChecker, topicId.get, topicPartition.partition(), metaStreamId, leaderEpoch, logIdent = logIdent)
// open partition meta stream
val stream = client.streamClient().openStream(metaStreamId, OpenStreamOptions.builder().epoch(leaderEpoch).tags(streamTags).build())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@
@JsonIgnoreProperties(ignoreUnknown = true)
public class ElasticLogMeta {
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
private static final ObjectWriter WRITER = OBJECT_MAPPER.writer();
private static final ObjectWriter WRITER = OBJECT_MAPPER.writerFor(ElasticLogMeta.class);
private static final ObjectReader READER = OBJECT_MAPPER.readerFor(ElasticLogMeta.class);

private static final Logger LOGGER = LoggerFactory.getLogger(ElasticLogMeta.class);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -162,6 +162,11 @@ public CompletableFuture<Stream> createAndOpenStream(CreateStreamOptions createS
return CompletableFuture.completedFuture(new StreamImpl(streamIdAlloc.incrementAndGet()));
}

@Override
public CompletableFuture<Void> preWarmStream(long streamId) {
return CompletableFuture.completedFuture(null);
}

@Override
public CompletableFuture<Stream> openStream(long streamId, OpenStreamOptions openStreamOptions) {
return CompletableFuture.completedFuture(new StreamImpl(streamId));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,14 @@ public interface StreamClient {
*/
CompletableFuture<Stream> createAndOpenStream(CreateStreamOptions options);

/**
* PreWarmStream
* Expect to preWarm metaStream data cache before real open
* @param streamId metaStreamId
* @return preWarm result {@link CompletableFuture<Void>}
*/
CompletableFuture<Void> preWarmStream(long streamId);

/**
* Open stream.
*
Expand Down
19 changes: 19 additions & 0 deletions s3stream/src/main/java/com/automq/stream/s3/S3StreamClient.java
Original file line number Diff line number Diff line change
Expand Up @@ -23,16 +23,19 @@
import com.automq.stream.api.CreateStreamOptions;
import com.automq.stream.api.FetchResult;
import com.automq.stream.api.OpenStreamOptions;
import com.automq.stream.api.ReadOptions;
import com.automq.stream.api.RecordBatch;
import com.automq.stream.api.Stream;
import com.automq.stream.api.StreamClient;
import com.automq.stream.s3.cache.ReadDataBlock;
import com.automq.stream.s3.compact.StreamObjectCompactor;
import com.automq.stream.s3.context.AppendContext;
import com.automq.stream.s3.context.FetchContext;
import com.automq.stream.s3.metadata.StreamMetadata;
import com.automq.stream.s3.metadata.StreamState;
import com.automq.stream.s3.metrics.TimerUtil;
import com.automq.stream.s3.metrics.stats.StreamOperationStats;
import com.automq.stream.s3.model.StreamRecordBatch;
import com.automq.stream.s3.network.NetworkBandwidthLimiter;
import com.automq.stream.s3.objects.ObjectManager;
import com.automq.stream.s3.operator.ObjectStorage;
Expand Down Expand Up @@ -134,6 +137,22 @@ public CompletableFuture<Stream> createAndOpenStream(CreateStreamOptions options
});
}

public CompletableFuture<Void> preWarmStream(long streamId) {
return this.streamManager.getStreams(List.of(streamId)).thenCompose(meta -> {
if (meta.isEmpty()) {
return CompletableFuture.completedFuture(null);
}

FetchContext fetchContext = new FetchContext();
ReadOptions readOptions = ReadOptions.builder().prioritizedRead(true).build();
fetchContext.setReadOptions(readOptions);

StreamMetadata streamMetadata = meta.get(0);
CompletableFuture<ReadDataBlock> read = this.storage.read(fetchContext, streamId, streamMetadata.startOffset(), streamMetadata.endOffset(), 64 * 1024);
return read.thenAccept(readDataBlock -> readDataBlock.getRecords().forEach(StreamRecordBatch::release));
});
}

@Override
public CompletableFuture<Stream> openStream(long streamId, OpenStreamOptions openStreamOptions) {
return runInLock(() -> {
Expand Down
Loading