Skip to content

Commit

Permalink
feat(s3stream): merge small data blocks during compaction
Browse files Browse the repository at this point in the history
Signed-off-by: Shichao Nie <[email protected]>
  • Loading branch information
SCNieh committed Jan 9, 2024
1 parent 7d7eb1b commit 6d83343
Show file tree
Hide file tree
Showing 12 changed files with 370 additions and 80 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import com.automq.stream.s3.compact.objects.CompactedObject;
import com.automq.stream.s3.compact.objects.CompactedObjectBuilder;
import com.automq.stream.s3.compact.objects.CompactionType;
import com.automq.stream.s3.compact.utils.CompactionUtils;
import com.automq.stream.utils.LogContext;
import java.util.ArrayList;
import java.util.Collection;
Expand All @@ -31,7 +32,6 @@
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import java.util.stream.Collectors;
import org.apache.commons.lang3.tuple.ImmutablePair;
import org.apache.commons.lang3.tuple.Pair;
Expand Down Expand Up @@ -87,7 +87,7 @@ public List<CompactionPlan> analyze(Map<Long, List<StreamDataBlock>> streamDataB
*/
List<CompactedObjectBuilder> groupObjectWithLimits(Map<Long, List<StreamDataBlock>> streamDataBlockMap,
Set<Long> excludedObjectIds) {
List<StreamDataBlock> sortedStreamDataBlocks = sortStreamRangePositions(streamDataBlockMap);
List<StreamDataBlock> sortedStreamDataBlocks = CompactionUtils.sortStreamRangePositions(streamDataBlockMap);
List<CompactedObjectBuilder> compactedObjectBuilders = new ArrayList<>();
CompactionStats stats = null;
int streamNumInStreamSet = -1;
Expand Down Expand Up @@ -354,24 +354,6 @@ private CompactedObjectBuilder splitObject(CompactedObjectBuilder builder,
return builder;
}

/**
* Sort stream data blocks by stream id and {@code startOffset).
*
* @param streamDataBlocksMap stream data blocks map, key: object id, value: stream data blocks
* @return sorted stream data blocks
*/
List<StreamDataBlock> sortStreamRangePositions(Map<Long, List<StreamDataBlock>> streamDataBlocksMap) {
//TODO: use merge sort
Map<Long, List<StreamDataBlock>> sortedStreamObjectMap = new TreeMap<>();
for (List<StreamDataBlock> streamDataBlocks : streamDataBlocksMap.values()) {
streamDataBlocks.forEach(e -> sortedStreamObjectMap.computeIfAbsent(e.getStreamId(), k -> new ArrayList<>()).add(e));
}
return sortedStreamObjectMap.values().stream().flatMap(list -> {
list.sort(StreamDataBlock.STREAM_OFFSET_COMPARATOR);
return list.stream();
}).collect(Collectors.toList());
}

private static abstract class AbstractCompactedObjectComparator implements Comparator<CompactedObjectBuilder> {
protected final Map<Long, Integer> objectStatsMap;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,8 @@
import com.automq.stream.s3.compact.objects.CompactionType;
import com.automq.stream.s3.compact.operator.DataBlockReader;
import com.automq.stream.s3.compact.operator.DataBlockWriter;
import com.automq.stream.s3.compact.utils.CompactionUtils;
import com.automq.stream.s3.compact.utils.GroupByOffsetPredicate;
import com.automq.stream.s3.metadata.S3ObjectMetadata;
import com.automq.stream.s3.metadata.StreamMetadata;
import com.automq.stream.s3.metadata.StreamOffsetRange;
Expand Down Expand Up @@ -338,7 +340,7 @@ private Collection<CompletableFuture<StreamObject>> splitStreamSetObject(List<St
Collection<CompletableFuture<StreamObject>> groupAndSplitStreamDataBlocks(S3ObjectMetadata objectMetadata,
List<StreamDataBlock> streamDataBlocks) {
List<Pair<List<StreamDataBlock>, CompletableFuture<StreamObject>>> groupedDataBlocks = new ArrayList<>();
List<List<StreamDataBlock>> groupedStreamDataBlocks = CompactionUtils.groupStreamDataBlocks(streamDataBlocks);
List<List<StreamDataBlock>> groupedStreamDataBlocks = CompactionUtils.groupStreamDataBlocks(streamDataBlocks, new GroupByOffsetPredicate());
for (List<StreamDataBlock> group : groupedStreamDataBlocks) {
groupedDataBlocks.add(new ImmutablePair<>(group, new CompletableFuture<>()));
}
Expand Down Expand Up @@ -611,7 +613,8 @@ void executeCompactionPlans(CommitStreamSetObjectRequest request, List<Compactio
}).join();
streamObjectCfList.stream().map(CompletableFuture::join).forEach(request::addStreamObject);
}
List<ObjectStreamRange> objectStreamRanges = CompactionUtils.buildObjectStreamRange(sortedStreamDataBlocks);
List<ObjectStreamRange> objectStreamRanges = CompactionUtils.buildObjectStreamRangeFromGroup(
CompactionUtils.groupStreamDataBlocks(sortedStreamDataBlocks, new GroupByOffsetPredicate()));
objectStreamRanges.forEach(request::addStreamRange);
request.setObjectId(uploader.getStreamSetObjectId());
// set stream set object id to be the first object id of compacted objects
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import com.automq.stream.s3.compact.objects.CompactedObject;
import com.automq.stream.s3.compact.objects.CompactionType;
import com.automq.stream.s3.compact.operator.DataBlockWriter;
import com.automq.stream.s3.compact.utils.CompactionUtils;
import com.automq.stream.s3.objects.ObjectManager;
import com.automq.stream.s3.objects.StreamObject;
import com.automq.stream.s3.operator.S3Operator;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,8 @@
import com.automq.stream.s3.DataBlockIndex;
import com.automq.stream.s3.DirectByteBufAlloc;
import com.automq.stream.s3.StreamDataBlock;
import com.automq.stream.s3.compact.utils.CompactionUtils;
import com.automq.stream.s3.compact.utils.GroupByLimitPredicate;
import com.automq.stream.s3.metadata.ObjectUtils;
import com.automq.stream.s3.metrics.MetricsLevel;
import com.automq.stream.s3.metrics.stats.CompactionStats;
Expand Down Expand Up @@ -147,22 +149,19 @@ public long size() {
}

class IndexBlock {
private static final int DEFAULT_DATA_BLOCK_GROUP_SIZE_THRESHOLD = 1024 * 1024; // 1MiB
private final ByteBuf buf;
private final long position;

public IndexBlock() {
position = nextDataBlockPosition;
buf = DirectByteBufAlloc.byteBuffer(calculateIndexBlockSize(), "write_index_block");
long nextPosition = 0;
for (StreamDataBlock block : completedBlocks) {
new DataBlockIndex(block.getStreamId(), block.getStartOffset(), (int) (block.getEndOffset() - block.getStartOffset()),
block.dataBlockIndex().recordCount(), nextPosition, block.getBlockSize()).encode(buf);
nextPosition += block.getBlockSize();
}
}

private int calculateIndexBlockSize() {
return completedBlocks.size() * DataBlockIndex.BLOCK_INDEX_SIZE;
List<DataBlockIndex> dataBlockIndices = CompactionUtils.buildDataBlockIndicesFromGroup(
CompactionUtils.groupStreamDataBlocks(completedBlocks, new GroupByLimitPredicate(DEFAULT_DATA_BLOCK_GROUP_SIZE_THRESHOLD)));
buf = DirectByteBufAlloc.byteBuffer(dataBlockIndices.size() * DataBlockIndex.BLOCK_INDEX_SIZE, "write_index_block");
for (DataBlockIndex dataBlockIndex : dataBlockIndices) {
dataBlockIndex.encode(buf);
}
}

public ByteBuf buffer() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,8 +15,9 @@
* limitations under the License.
*/

package com.automq.stream.s3.compact;
package com.automq.stream.s3.compact.utils;

import com.automq.stream.s3.DataBlockIndex;
import com.automq.stream.s3.StreamDataBlock;
import com.automq.stream.s3.compact.objects.CompactedObjectBuilder;
import com.automq.stream.s3.compact.operator.DataBlockReader;
Expand All @@ -31,8 +32,10 @@
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.TreeMap;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.function.Predicate;
import java.util.stream.Collectors;
import org.slf4j.Logger;

Expand Down Expand Up @@ -110,33 +113,88 @@ public static Map<Long, List<StreamDataBlock>> blockWaitObjectIndices(List<Strea
}

/**
* Group stream data blocks by stream id and offset.
* Sort stream data blocks by stream id and start offset.
*
* @param streamDataBlocksMap streamDataBlocksMap stream data blocks map, key: object id, value: stream data blocks
* @return sorted stream data blocks
*/
public static List<List<StreamDataBlock>> groupStreamDataBlocks(List<StreamDataBlock> streamDataBlocks) {
public static List<StreamDataBlock> sortStreamRangePositions(Map<Long, List<StreamDataBlock>> streamDataBlocksMap) {
//TODO: use merge sort
Map<Long, List<StreamDataBlock>> sortedStreamObjectMap = new TreeMap<>();
for (List<StreamDataBlock> streamDataBlocks : streamDataBlocksMap.values()) {
streamDataBlocks.forEach(e -> sortedStreamObjectMap.computeIfAbsent(e.getStreamId(), k -> new ArrayList<>()).add(e));
}
return sortedStreamObjectMap.values().stream().flatMap(list -> {
list.sort(StreamDataBlock.STREAM_OFFSET_COMPARATOR);
return list.stream();
}).collect(Collectors.toList());
}

/**
* Group stream data blocks by certain conditions.
*
* @param streamDataBlocks stream data blocks to be grouped
* @param predicate the predicate to check whether a stream data block should be grouped with the previous one
* @return grouped stream data blocks
*/
public static List<List<StreamDataBlock>> groupStreamDataBlocks(List<StreamDataBlock> streamDataBlocks,
Predicate<StreamDataBlock> predicate) {
List<List<StreamDataBlock>> groupedStreamDataBlocks = new ArrayList<>();
List<StreamDataBlock> currGroup = new ArrayList<>();
StreamDataBlock currStreamDataBlock = null;
for (StreamDataBlock streamDataBlock : streamDataBlocks) {
if (currGroup.isEmpty() || currStreamDataBlock == null) {
if (predicate.test(streamDataBlock)) {
currGroup.add(streamDataBlock);
} else if (!currGroup.isEmpty()) {
groupedStreamDataBlocks.add(currGroup);
currGroup = new ArrayList<>();
currGroup.add(streamDataBlock);
} else {
if (currStreamDataBlock.getStreamId() == streamDataBlock.getStreamId()
&& currStreamDataBlock.getEndOffset() == streamDataBlock.getStartOffset()) {
currGroup.add(streamDataBlock);
} else {
groupedStreamDataBlocks.add(currGroup);
currGroup = new ArrayList<>();
currGroup.add(streamDataBlock);
}
}
currStreamDataBlock = streamDataBlock;
}
if (!currGroup.isEmpty()) {
groupedStreamDataBlocks.add(currGroup);
}
return groupedStreamDataBlocks;
}

public static List<ObjectStreamRange> buildObjectStreamRangeFromGroup(List<List<StreamDataBlock>> streamDataBlockGroup) {
List<ObjectStreamRange> objectStreamRanges = new ArrayList<>();

for (List<StreamDataBlock> streamDataBlocks : streamDataBlockGroup) {
if (streamDataBlocks.isEmpty()) {
continue;
}
objectStreamRanges.add(new ObjectStreamRange(
streamDataBlocks.get(0).getStreamId(),
-1L,
streamDataBlocks.get(0).getStartOffset(),
streamDataBlocks.get(streamDataBlocks.size() - 1).getEndOffset(),
streamDataBlocks.stream().mapToInt(StreamDataBlock::getBlockSize).sum()));
}

return objectStreamRanges;
}

public static List<DataBlockIndex> buildDataBlockIndicesFromGroup(List<List<StreamDataBlock>> streamDataBlockGroup) {
List<DataBlockIndex> dataBlockIndices = new ArrayList<>();

long blockStartPosition = 0;
for (List<StreamDataBlock> streamDataBlocks : streamDataBlockGroup) {
if (streamDataBlocks.isEmpty()) {
continue;
}
dataBlockIndices.add(new DataBlockIndex(
streamDataBlocks.get(0).getStreamId(),
streamDataBlocks.get(0).getStartOffset(),
(int) (streamDataBlocks.get(streamDataBlocks.size() - 1).getEndOffset() - streamDataBlocks.get(0).getStartOffset()),
streamDataBlocks.stream().map(StreamDataBlock::dataBlockIndex).mapToInt(DataBlockIndex::recordCount).sum(),
blockStartPosition,
streamDataBlocks.stream().mapToInt(StreamDataBlock::getBlockSize).sum()));
blockStartPosition += streamDataBlocks.stream().mapToInt(StreamDataBlock::getBlockSize).sum();
}

return dataBlockIndices;
}

public static int getTotalObjectStats(CompactedObjectBuilder o, Map<Long, Integer> objectStatsMap) {
int totalCompactedObjects = 0;
for (Long objectId : o.uniqueObjectIds()) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,60 @@
/*
* 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 com.automq.stream.s3.compact.utils;

import com.automq.stream.s3.StreamDataBlock;
import java.util.function.Predicate;

public class GroupByLimitPredicate implements Predicate<StreamDataBlock> {
private final long blockSizeThreshold;
private long streamId = -1;
private long startOffset = 0;
private long nextStartOffset = 0;
private int blockSize = 0;
private int recordCnt = 0;

public GroupByLimitPredicate(long blockSizeThreshold) {
this.blockSizeThreshold = blockSizeThreshold;
}

@Override
public boolean test(StreamDataBlock block) {
boolean flag = true;
if (streamId == -1 // first block
|| block.getStreamId() != streamId // iterate to next stream
|| block.getStartOffset() != nextStartOffset // block start offset is not continuous for same stream (unlikely to happen)
|| (long) blockSize + block.getBlockSize() >= blockSizeThreshold // group size exceeds threshold
|| (long) recordCnt + block.dataBlockIndex().recordCount() > Integer.MAX_VALUE // group record count exceeds int32
|| (block.getEndOffset() - startOffset) > Integer.MAX_VALUE) { // group delta offset exceeds int32

if (streamId != -1) {
flag = false;
}

streamId = block.getStreamId();
startOffset = block.getStartOffset();
blockSize = 0;
recordCnt = 0;
}

nextStartOffset = block.getEndOffset();
blockSize += block.getBlockSize();
recordCnt += block.dataBlockIndex().recordCount();
return flag;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,45 @@
/*
* 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 com.automq.stream.s3.compact.utils;

import com.automq.stream.s3.StreamDataBlock;
import java.util.function.Predicate;

public class GroupByOffsetPredicate implements Predicate<StreamDataBlock> {

private long currStreamId = -1;
private long nextStartOffset = 0;

@Override
public boolean test(StreamDataBlock block) {
if (currStreamId == -1) {
currStreamId = block.getStreamId();
nextStartOffset = block.getEndOffset();
return true;
} else {
if (currStreamId == block.getStreamId() && nextStartOffset == block.getStartOffset()) {
nextStartOffset = block.getEndOffset();
return true;
} else {
currStreamId = block.getStreamId();
nextStartOffset = block.getEndOffset();
return false;
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import com.automq.stream.s3.compact.objects.CompactedObject;
import com.automq.stream.s3.compact.objects.CompactedObjectBuilder;
import com.automq.stream.s3.compact.objects.CompactionType;
import com.automq.stream.s3.compact.utils.CompactionUtils;
import com.automq.stream.s3.metadata.S3ObjectMetadata;
import com.automq.stream.s3.metadata.S3ObjectType;
import com.automq.stream.s3.metadata.StreamMetadata;
Expand Down Expand Up @@ -152,7 +153,7 @@ public void testSortStreamRangePositions() {
CompactionAnalyzer compactionAnalyzer = new CompactionAnalyzer(CACHE_SIZE, STREAM_SPLIT_SIZE, MAX_STREAM_NUM_IN_WAL, MAX_STREAM_OBJECT_NUM);
List<StreamMetadata> streamMetadataList = this.streamManager.getStreams(Collections.emptyList()).join();
Map<Long, List<StreamDataBlock>> streamDataBlocksMap = CompactionUtils.blockWaitObjectIndices(streamMetadataList, S3_WAL_OBJECT_METADATA_LIST, s3Operator);
List<StreamDataBlock> sortedStreamDataBlocks = compactionAnalyzer.sortStreamRangePositions(streamDataBlocksMap);
List<StreamDataBlock> sortedStreamDataBlocks = CompactionUtils.sortStreamRangePositions(streamDataBlocksMap);
List<StreamDataBlock> expectedBlocks = List.of(
new StreamDataBlock(STREAM_0, 0, 15, OBJECT_0, -1, -1, 1),
new StreamDataBlock(STREAM_0, 15, 20, OBJECT_1, -1, -1, 1),
Expand Down
Loading

0 comments on commit 6d83343

Please sign in to comment.