Skip to content
Merged
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
2 changes: 1 addition & 1 deletion core/src/main/scala/kafka/log/s3/S3Storage.java
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ public class S3Storage implements Storage {
private final ScheduledExecutorService mainExecutor = Executors.newSingleThreadScheduledExecutor(
ThreadUtils.createThreadFactory("s3-storage-main", false));
private final ScheduledExecutorService backgroundExecutor = Executors.newSingleThreadScheduledExecutor(
ThreadUtils.createThreadFactory("s3-storage-main", true));
ThreadUtils.createThreadFactory("s3-storage-background", true));
private final ObjectManager objectManager;
private final S3Operator s3Operator;
private final S3BlockCache blockCache;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,8 @@
import org.apache.kafka.metadata.stream.InRangeObjects;
import org.apache.kafka.metadata.stream.S3Object;
import org.apache.kafka.metadata.stream.S3ObjectMetadata;
import org.apache.kafka.metadata.stream.S3StreamObject;
import org.apache.kafka.metadata.stream.S3StreamObjectMetadata;
import org.apache.kafka.metadata.stream.StreamOffsetRange;
import org.apache.kafka.raft.OffsetAndEpoch;
import org.slf4j.Logger;
Expand Down Expand Up @@ -163,6 +165,24 @@ public CompletableFuture<InRangeObjects> fetch(long streamId, long startOffset,
}
}

public CompletableFuture<List<S3StreamObjectMetadata>> getStreamObjects(long streamId, long startOffset, long endOffset, int limit) {
synchronized (StreamMetadataManager.this) {
try {
List<S3StreamObject> streamObjects = streamsImage.getStreamObjects(streamId, startOffset, endOffset, limit);
List<S3StreamObjectMetadata> s3StreamObjectMetadataList = streamObjects.stream().map(object -> {
long committedTimeInMs = objectsImage.getObjectMetadata(object.objectId()).getCommittedTimeInMs();
return new S3StreamObjectMetadata(object, committedTimeInMs);
}).collect(Collectors.toList());
return CompletableFuture.completedFuture(s3StreamObjectMetadataList);
} catch (Exception e) {
LOGGER.warn(
"[GetStreamObjects]: stream: {}, startOffset: {}, endOffset: {}, limit: {}, and search in metadataCache failed with exception: {}",
streamId, startOffset, endOffset, limit, e.getMessage());
return CompletableFuture.failedFuture(e);
}
}
}

// must access thread safe
private CompletableFuture<InRangeObjects> pendingFetch(long streamId, long startOffset, long endOffset, int limit) {
GetObjectsTask task = GetObjectsTask.of(streamId, startOffset, endOffset, limit);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -98,6 +98,34 @@ public InRangeObjects getObjects(long streamId, long startOffset, long endOffset
return new InRangeObjects(streamId, startOffset, realEndOffset, objects);
}

/**
* Get stream objects in range [startOffset, endOffset) with limit.
* It will throw IllegalArgumentException if limit or streamId is invalid.
* @param streamId stream id
* @param startOffset inclusive start offset of the stream
* @param endOffset exclusive end offset of the stream
* @param limit max number of stream objects to return
* @return stream objects
*/
public List<S3StreamObject> getStreamObjects(long streamId, long startOffset, long endOffset, int limit) {
if (limit <= 0) {
throw new IllegalArgumentException("limit must be positive");
}
S3StreamMetadataImage stream = streamsMetadata.get(streamId);
if (stream == null) {
throw new IllegalArgumentException("stream not found");
}
Map<Long, S3StreamObject> streamObjectsMetadata = stream.getStreamObjects();
if (streamObjectsMetadata == null || streamObjectsMetadata.isEmpty()) {
return Collections.emptyList();
}
return streamObjectsMetadata.values().stream().filter(obj -> {
long objectStartOffset = obj.streamOffsetRange().getStartOffset();
long objectEndOffset = obj.streamOffsetRange().getEndOffset();
return objectStartOffset < endOffset && objectEndOffset > startOffset;
}).sorted(Comparator.comparing(S3StreamObject::streamOffsetRange)).limit(limit).collect(Collectors.toCollection(ArrayList::new));
}

private List<RangeSearcher> rangeSearchers(long streamId, long startOffset, long endOffset) {
S3StreamMetadataImage streamMetadata = streamsMetadata.get(streamId);
List<RangeSearcher> rangeSearchers = new ArrayList<>();
Expand Down Expand Up @@ -161,17 +189,12 @@ private Queue<ObjectStreamRange> rangeOfStreamObjects() {
// TODO: refactor to make stream objects in order
if (streamObjectsMetadata != null && !streamObjectsMetadata.isEmpty()) {
return streamObjectsMetadata.values().stream().filter(obj -> {
long objectStartOffset = obj.streamIndex().getStartOffset();
long objectEndOffset = obj.streamIndex().getEndOffset();
long objectStartOffset = obj.streamOffsetRange().getStartOffset();
long objectEndOffset = obj.streamOffsetRange().getEndOffset();
return objectStartOffset < endOffset && objectEndOffset > startOffset;
}).sorted(new Comparator<S3StreamObject>() {
@Override
public int compare(S3StreamObject o1, S3StreamObject o2) {
return o1.objectId() > o2.objectId() ? 1 : -1;
}
}).map(obj -> {
long startOffset = obj.streamIndex().getStartOffset();
long endOffset = obj.streamIndex().getEndOffset();
}).sorted(Comparator.comparingLong(S3StreamObject::objectId)).map(obj -> {
long startOffset = obj.streamOffsetRange().getStartOffset();
long endOffset = obj.streamOffsetRange().getEndOffset();
return new ObjectStreamRange(obj.objectId(), obj.objectType(), startOffset, endOffset);
}).collect(Collectors.toCollection(LinkedList::new));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,32 +27,36 @@ public class S3StreamObject {

private final long objectSize;

private final StreamOffsetRange streamIndex;
private final StreamOffsetRange streamOffsetRange;

public S3StreamObject(long objectId, long objectSize, long streamId, long startOffset, long endOffset) {
this.objectId = objectId;
this.objectSize = objectSize;
this.streamIndex = new StreamOffsetRange(streamId, startOffset, endOffset);
this.streamOffsetRange = new StreamOffsetRange(streamId, startOffset, endOffset);
}

public StreamOffsetRange streamIndex() {
return streamIndex;
public StreamOffsetRange streamOffsetRange() {
return streamOffsetRange;
}

public long objectId() {
return objectId;
}

public long objectSize() {
return objectSize;
}

public S3ObjectType objectType() {
return S3ObjectType.STREAM;
}

public ApiMessageAndVersion toRecord() {
return new ApiMessageAndVersion(new S3StreamObjectRecord()
.setObjectId(objectId)
.setStreamId(streamIndex.getStreamId())
.setStartOffset(streamIndex.getStartOffset())
.setEndOffset(streamIndex.getEndOffset()), (short) 0);
.setStreamId(streamOffsetRange.getStreamId())
.setStartOffset(streamOffsetRange.getStartOffset())
.setEndOffset(streamOffsetRange.getEndOffset()), (short) 0);
}

public static S3StreamObject of(S3StreamObjectRecord record) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
/*
* 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 org.apache.kafka.metadata.stream;

public class S3StreamObjectMetadata implements Comparable<S3StreamObjectMetadata> {
private final S3StreamObject s3StreamObject;
private final long timestamp;

public S3StreamObjectMetadata(S3StreamObject s3StreamObject, long timestamp) {
this.s3StreamObject = s3StreamObject;
this.timestamp = timestamp;
}

public long startOffset() {
return s3StreamObject.streamOffsetRange().getStartOffset();
}

public long endOffset() {
return s3StreamObject.streamOffsetRange().getEndOffset();
}

public long streamId() {
return s3StreamObject.streamOffsetRange().getStreamId();
}

public long objectId() {
return s3StreamObject.objectId();
}

public long timestamp() {
return timestamp;
}

@Override
public int compareTo(S3StreamObjectMetadata o) {
return s3StreamObject.streamOffsetRange().compareTo(o.s3StreamObject.streamOffsetRange());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,10 @@ public long getEndOffset() {
@Override
public int compareTo(StreamOffsetRange o) {
int res = Long.compare(this.streamId, o.streamId);
return res == 0 ? Long.compare(this.startOffset, o.startOffset) : res;
if (res != 0)
return res;
res = Long.compare(this.startOffset, o.startOffset);
return res == 0 ? Long.compare(this.endOffset, o.endOffset) : res;
}

public StreamIndex toRecordStreamIndex() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -624,8 +624,8 @@ public void testCommitStreamObject() {
// 6. verify stream objects
assertEquals(1, manager.streamsMetadata().get(STREAM1).streamObjects().size());
assertEquals(4L, manager.streamsMetadata().get(STREAM1).streamObjects().get(4L).objectId());
assertEquals(0L, manager.streamsMetadata().get(STREAM1).streamObjects().get(4L).streamIndex().getStartOffset());
assertEquals(400L, manager.streamsMetadata().get(STREAM1).streamObjects().get(4L).streamIndex().getEndOffset());
assertEquals(0L, manager.streamsMetadata().get(STREAM1).streamObjects().get(4L).streamOffsetRange().getStartOffset());
assertEquals(400L, manager.streamsMetadata().get(STREAM1).streamObjects().get(4L).streamOffsetRange().getEndOffset());
}

private void commitFirstLevelWalObject(long objectId, long orderId, long streamId, long startOffset, long endOffset, long epoch, int brokerId) {
Expand Down