Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

feat(stream): add stream range continuous check for MemoryMetadataManager #860

Merged
merged 2 commits into from
Dec 27, 2023
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
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,21 @@ public synchronized CompletableFuture<CommitStreamSetObjectResponse> commitStrea
for (ObjectStreamRange range : request.getStreamRanges()) {
StreamMetadata stream = streams.get(range.getStreamId());
assert stream != null;
stream.setEndOffset(range.getEndOffset());
if (request.getCompactedObjectIds().isEmpty()) {
// Commit new object.
if (stream.getEndOffset() != range.getStartOffset()) {
throw new IllegalArgumentException("stream " + range.getStreamId() + " end offset " + stream.getEndOffset() + " is not equal to start offset of request " + range.getStartOffset());
}
stream.setEndOffset(range.getEndOffset());
} else {
// Compact old object.
if (stream.getEndOffset() < range.getEndOffset()) {
throw new IllegalArgumentException("stream " + range.getStreamId() + " end offset " + stream.getEndOffset() + " is lesser than request " + range.getEndOffset());
}
if (stream.getStartOffset() > range.getStartOffset()) {
throw new IllegalArgumentException("stream " + range.getStreamId() + " start offset " + stream.getStartOffset() + " is greater than request " + range.getStartOffset());
}
}
}

S3ObjectMetadata object = new S3ObjectMetadata(
Expand All @@ -95,7 +109,21 @@ public synchronized CompletableFuture<CommitStreamSetObjectResponse> commitStrea
long streamId = streamObject.getStreamId();
StreamMetadata stream = streams.get(streamId);
assert stream != null;
stream.setEndOffset(streamObject.getEndOffset());
if (request.getCompactedObjectIds().isEmpty()) {
// Commit new object.
if (stream.getEndOffset() != streamObject.getStartOffset()) {
throw new IllegalArgumentException("stream " + streamObject.getStreamId() + " end offset " + stream.getEndOffset() + " is not equal to start offset of request " + streamObject.getStartOffset());
}
stream.setEndOffset(streamObject.getEndOffset());
} else {
// Compact old object.
if (stream.getEndOffset() < streamObject.getEndOffset()) {
throw new IllegalArgumentException("stream " + streamObject.getStreamId() + " end offset " + stream.getEndOffset() + " is lesser than request " + streamObject.getEndOffset());
}
if (stream.getStartOffset() > streamObject.getStartOffset()) {
throw new IllegalArgumentException("stream " + streamObject.getStreamId() + " start offset " + stream.getStartOffset() + " is greater than request " + streamObject.getStartOffset());
}
}

List<S3ObjectMetadata> metadataList = streamObjects.computeIfAbsent(streamId, id -> new LinkedList<>());
metadataList.add(
Expand All @@ -112,12 +140,14 @@ public synchronized CompletableFuture<CommitStreamSetObjectResponse> commitStrea
@Override
public synchronized CompletableFuture<Void> compactStreamObject(CompactStreamObjectRequest request) {
long streamId = request.getStreamId();
StreamObject streamObject = new StreamObject();
streamObject.setStreamId(streamId);
streamObject.setStartOffset(request.getStartOffset());
streamObject.setEndOffset(request.getEndOffset());
streamObject.setObjectId(request.getObjectId());
streamObject.setObjectSize(request.getObjectSize());
StreamMetadata stream = streams.get(streamId);
assert stream != null;
if (stream.getEndOffset() < request.getEndOffset()) {
throw new IllegalArgumentException("stream " + streamId + " end offset " + stream.getEndOffset() + " is lesser than request " + request.getEndOffset());
}
if (stream.getStartOffset() > request.getStartOffset()) {
throw new IllegalArgumentException("stream " + streamId + " start offset " + stream.getStartOffset() + " is greater than request " + request.getStartOffset());
}

streamObjects.computeIfAbsent(streamId, id -> new LinkedList<>())
.add(new S3ObjectMetadata(
Expand All @@ -136,11 +166,11 @@ public synchronized CompletableFuture<List<S3ObjectMetadata>> getObjects(long st
List<S3ObjectMetadata> streamSetObjectList = streamSetObjects.values()
.stream()
.map(Pair::getRight)
.filter(o -> o.getOffsetRanges().stream().anyMatch(r -> r.getStreamId() == streamId && r.getEndOffset() > startOffset && (r.getStartOffset() <= endOffset || endOffset == -1)))
.filter(o -> o.getOffsetRanges().stream().anyMatch(r -> r.getStreamId() == streamId && r.getEndOffset() > startOffset && (r.getStartOffset() < endOffset || endOffset == -1)))
.toList();
List<S3ObjectMetadata> streamObjectList = streamObjects.computeIfAbsent(streamId, id -> new LinkedList<>())
.stream()
.filter(o -> o.getOffsetRanges().stream().anyMatch(r -> r.getStreamId() == streamId && r.getEndOffset() > startOffset && (r.getStartOffset() <= endOffset || endOffset == -1)))
.filter(o -> o.getOffsetRanges().stream().anyMatch(r -> r.getStreamId() == streamId && r.getEndOffset() > startOffset && (r.getStartOffset() < endOffset || endOffset == -1)))
.toList();

List<S3ObjectMetadata> result = new ArrayList<>();
Expand Down Expand Up @@ -169,15 +199,15 @@ public synchronized CompletableFuture<List<S3ObjectMetadata>> getStreamObjects(l
long endOffset, int limit) {
List<S3ObjectMetadata> streamObjectList = streamObjects.computeIfAbsent(streamId, id -> new LinkedList<>())
.stream()
.filter(o -> o.getOffsetRanges().stream().anyMatch(r -> r.getStreamId() == streamId && r.getEndOffset() > startOffset && (r.getStartOffset() <= endOffset || endOffset == -1)))
.filter(o -> o.getOffsetRanges().stream().anyMatch(r -> r.getStreamId() == streamId && r.getEndOffset() > startOffset && (r.getStartOffset() < endOffset || endOffset == -1)))
.limit(limit)
.toList();
return CompletableFuture.completedFuture(streamObjectList);
}

@Override
public synchronized CompletableFuture<List<StreamMetadata>> getOpeningStreams() {
return CompletableFuture.completedFuture(streams.values().stream().toList());
return CompletableFuture.completedFuture(streams.values().stream().filter(stream -> stream.getState() == StreamState.OPENED).toList());
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,14 +74,22 @@ void testCommitAndCompact() {
streamObject.setStartOffset(0);
streamObject.setEndOffset(10);
streamObjectList.add(streamObject);

streamObject = new StreamObject();
streamObject.setObjectId(4);
streamObject.setStreamId(2);
streamObject.setStartOffset(10);
streamObject.setEndOffset(20);
streamObjectList.add(streamObject);

request.setStreamObjects(streamObjectList);
objectManager.commitStreamSetObject(request).join();

List<StreamMetadata> streamMetadataList = streamManager.getStreams(List.of(0L, 1L, 2L)).join();
assertEquals(3, streamMetadataList.size());
assertEquals(3, streamMetadataList.get(0).getEndOffset());
assertEquals(5, streamMetadataList.get(1).getEndOffset());
assertEquals(10, streamMetadataList.get(2).getEndOffset());
assertEquals(20, streamMetadataList.get(2).getEndOffset());

List<S3ObjectMetadata> streamSetObjectMetadataList = objectManager.getServerObjects().join();
assertEquals(1, streamSetObjectMetadataList.size());
Expand All @@ -105,6 +113,9 @@ void testCommitAndCompact() {
assertEquals(0, ranges.get(0).getStartOffset());
assertEquals(10, ranges.get(0).getEndOffset());

streamObjectMetadataList = objectManager.getStreamObjects(2, 0, 20, 100).join();
assertEquals(2, streamObjectMetadataList.size());

// Compact stream set object and commit stream object.
request = new CommitStreamSetObjectRequest();
request.setObjectId(ObjectUtils.NOOP_OBJECT_ID);
Expand All @@ -125,13 +136,6 @@ void testCommitAndCompact() {
streamObject.setEndOffset(5);
streamObjectList.add(streamObject);

streamObject = new StreamObject();
streamObject.setObjectId(4);
streamObject.setStreamId(2);
streamObject.setStartOffset(10);
streamObject.setEndOffset(20);
streamObjectList.add(streamObject);

request.setStreamObjects(streamObjectList);
objectManager.commitStreamSetObject(request).join();

Expand All @@ -142,8 +146,6 @@ void testCommitAndCompact() {
assertEquals(1, streamObjectMetadataList.size());
streamObjectMetadataList = objectManager.getStreamObjects(1, 0, 10, 100).join();
assertEquals(1, streamObjectMetadataList.size());
streamObjectMetadataList = objectManager.getStreamObjects(2, 0, 10, 100).join();
assertEquals(2, streamObjectMetadataList.size());

// Compact stream object.
objectManager.compactStreamObject(new CompactStreamObjectRequest(5, 2000, 2, 0, 20, List.of(1L, 4L))).join();
Expand Down
Loading