From 4bacd09df586b28db5f729acc13708ffa7677109 Mon Sep 17 00:00:00 2001 From: SSpirits Date: Tue, 2 Jan 2024 20:13:23 +0800 Subject: [PATCH] refactor(stream): format and cleanup code (#874) Signed-off-by: SSpirits --- s3stream/pom.xml | 4 +- .../com/automq/stream/DefaultRecordBatch.java | 1 - .../stream/RecordBatchWithContextWrapper.java | 21 +- .../java/com/automq/stream/api/Client.java | 1 + .../com/automq/stream/api/FetchResult.java | 1 - .../java/com/automq/stream/api/KVClient.java | 1 + .../java/com/automq/stream/api/KeyValue.java | 13 +- .../com/automq/stream/api/ReadOptions.java | 8 +- .../java/com/automq/stream/api/Stream.java | 2 - .../automq/stream/s3/DeltaWALUploadTask.java | 40 +- .../automq/stream/s3/DirectByteBufAlloc.java | 5 +- .../com/automq/stream/s3/ObjectReader.java | 24 +- .../com/automq/stream/s3/ObjectWriter.java | 26 +- .../java/com/automq/stream/s3/S3Storage.java | 218 ++-- .../java/com/automq/stream/s3/S3Stream.java | 87 +- .../com/automq/stream/s3/S3StreamClient.java | 65 +- .../java/com/automq/stream/s3/Storage.java | 4 +- .../com/automq/stream/s3/StreamDataBlock.java | 26 +- .../automq/stream/s3/StreamObjectCopier.java | 71 +- .../s3/StreamObjectsCompactionTask.java | 415 +++--- .../stream/s3/StreamRecordBatchCodec.java | 2 +- .../com/automq/stream/s3/WalWriteRequest.java | 4 +- .../automq/stream/s3/cache/BlockCache.java | 37 +- .../s3/cache/DataBlockReadAccumulator.java | 7 +- .../stream/s3/cache/DataBlockRecords.java | 5 +- .../stream/s3/cache/DefaultS3BlockCache.java | 107 +- .../stream/s3/cache/InflightReadThrottle.java | 9 +- .../com/automq/stream/s3/cache/LogCache.java | 41 +- .../stream/s3/cache/ObjectReaderLRUCache.java | 15 +- .../stream/s3/cache/ReadAheadAgent.java | 33 +- .../stream/s3/cache/ReadAheadManager.java | 3 +- .../automq/stream/s3/cache/ReadDataBlock.java | 1 - .../automq/stream/s3/cache/S3BlockCache.java | 4 +- .../automq/stream/s3/cache/StreamReader.java | 233 ++-- .../stream/s3/compact/CompactionAnalyzer.java | 72 +- .../stream/s3/compact/CompactionManager.java | 259 ++-- .../stream/s3/compact/CompactionPlan.java | 5 +- .../stream/s3/compact/CompactionStats.java | 1 - .../stream/s3/compact/CompactionUploader.java | 58 +- .../stream/s3/compact/CompactionUtils.java | 69 +- .../s3/compact/objects/CompactedObject.java | 1 - .../objects/CompactedObjectBuilder.java | 3 +- .../s3/compact/operator/DataBlockReader.java | 93 +- .../s3/compact/operator/DataBlockWriter.java | 3 +- .../stream/s3/failover/DefaultServerless.java | 39 +- .../automq/stream/s3/failover/Failover.java | 13 +- .../stream/s3/failover/FailoverRequest.java | 16 +- .../automq/stream/s3/failover/Serverless.java | 7 +- .../s3/memory/MemoryMetadataManager.java | 8 +- .../stream/s3/metadata/S3ObjectMetadata.java | 36 +- .../stream/s3/metadata/StreamMetadata.java | 12 +- .../stream/s3/metadata/StreamOffsetRange.java | 4 +- .../stream/s3/metadata/StreamState.java | 8 +- .../stream/s3/metrics/AttributesCache.java | 31 +- .../s3/metrics/S3StreamMetricsConstant.java | 86 +- .../s3/metrics/S3StreamMetricsManager.java | 321 ++--- .../s3/metrics/operations/S3MetricsType.java | 12 +- .../s3/metrics/operations/S3Operation.java | 6 +- .../stream/s3/metrics/operations/S3Stage.java | 6 +- .../stream/s3/model/StreamRecordBatch.java | 10 +- .../network/AsyncNetworkBandwidthLimiter.java | 17 +- .../objects/CommitStreamSetObjectRequest.java | 22 +- .../objects/CompactStreamObjectRequest.java | 17 +- .../stream/s3/objects/ObjectManager.java | 18 +- .../stream/s3/objects/ObjectStreamRange.java | 32 +- .../stream/s3/objects/StreamObject.java | 15 +- .../stream/s3/operator/DefaultS3Operator.java | 34 +- .../stream/s3/operator/MemoryS3Operator.java | 9 +- .../stream/s3/operator/MultiPartWriter.java | 29 +- .../stream/s3/operator/ProxyWriter.java | 3 +- .../automq/stream/s3/operator/S3Operator.java | 27 +- .../com/automq/stream/s3/operator/Writer.java | 5 +- .../stream/s3/streams/StreamManager.java | 7 +- .../stream/s3/trace/AttributeBindings.java | 29 +- .../s3/trace/SpanAttributesExtractor.java | 9 +- .../automq/stream/s3/trace/TraceUtils.java | 28 +- .../stream/s3/trace/context/TraceContext.java | 5 +- .../java/com/automq/stream/s3/wal/Block.java | 5 +- .../com/automq/stream/s3/wal/BlockBatch.java | 18 +- .../com/automq/stream/s3/wal/BlockImpl.java | 4 +- .../automq/stream/s3/wal/BlockWALService.java | 152 +-- .../stream/s3/wal/MemoryWriteAheadLog.java | 1 - .../stream/s3/wal/SlidingWindowService.java | 33 +- .../com/automq/stream/s3/wal/WALHeader.java | 37 +- .../automq/stream/s3/wal/WriteAheadLog.java | 3 - .../stream/s3/wal/benchmark/WriteBench.java | 73 +- .../s3/wal/util/WALBlockDeviceChannel.java | 10 +- .../automq/stream/s3/wal/util/WALChannel.java | 3 +- .../stream/s3/wal/util/WALFileChannel.java | 1 - .../automq/stream/s3/wal/util/WALUtil.java | 7 +- .../moe/cnkirito/kdio/DirectChannel.java | 18 +- .../moe/cnkirito/kdio/DirectChannelImpl.java | 7 +- .../moe/cnkirito/kdio/DirectIOLib.java | 12 +- .../moe/cnkirito/kdio/DirectIOUtils.java | 3 +- .../cnkirito/kdio/DirectRandomAccessFile.java | 7 +- .../automq/stream/utils/AsyncRateLimiter.java | 5 +- .../stream/utils/ByteBufInputStream.java | 1 - .../stream/utils/CloseableIterator.java | 9 +- .../automq/stream/utils/CommandResult.java | 8 +- .../com/automq/stream/utils/FutureTicker.java | 8 +- .../com/automq/stream/utils/FutureUtil.java | 3 +- .../com/automq/stream/utils/LogContext.java | 40 +- .../java/com/automq/stream/utils/S3Utils.java | 194 +-- .../com/automq/stream/utils/ThreadUtils.java | 12 +- .../java/com/automq/stream/utils/Threads.java | 15 +- .../java/com/automq/stream/utils/Utils.java | 1 + .../utils/biniarysearch/ComparableItem.java | 1 + .../biniarysearch/IndexBlockOrderedBytes.java | 2 +- .../biniarysearch/StreamRecordBatchList.java | 1 - .../threads/S3StreamThreadPoolMonitor.java | 37 +- .../utils/threads/ThreadPoolWrapper.java | 87 +- .../s3/AsyncNetworkBandwidthLimiterTest.java | 3 +- .../automq/stream/s3/DefaultRecordBatch.java | 2 - .../s3/DefaultRecordBatchWithContext.java | 1 - .../stream/s3/DefaultS3BlockCacheTest.java | 32 +- .../stream/s3/DeltaWALUploadTaskTest.java | 58 +- .../automq/stream/s3/ObjectReaderTest.java | 6 +- .../automq/stream/s3/ObjectWriterTest.java | 10 +- .../com/automq/stream/s3/S3StorageTest.java | 44 +- .../automq/stream/s3/S3StreamMemoryTest.java | 2 +- .../com/automq/stream/s3/S3StreamTest.java | 11 +- .../stream/s3/StreamObjectCopierTest.java | 10 +- .../s3/StreamObjectsCompactionTaskTest.java | 211 ++- .../java/com/automq/stream/s3/TestUtils.java | 1 - .../stream/s3/cache/BlockCacheTest.java | 53 +- .../cache/DataBlockReadAccumulatorTest.java | 19 +- .../s3/cache/InflightReadThrottleTest.java | 5 +- .../automq/stream/s3/cache/LogCacheTest.java | 4 +- .../s3/cache/ObjectReaderLRUCacheTest.java | 5 +- .../stream/s3/cache/StreamCacheTest.java | 25 +- .../stream/s3/cache/StreamReaderTest.java | 55 +- .../s3/compact/CompactionAnalyzerTest.java | 419 +++--- .../s3/compact/CompactionManagerTest.java | 150 +-- .../stream/s3/compact/CompactionTestBase.java | 48 +- .../s3/compact/CompactionUploaderTest.java | 37 +- .../stream/s3/compact/CompactionUtilTest.java | 23 +- .../stream/s3/failover/FailoverTest.java | 11 +- .../s3/operator/DefaultS3OperatorTest.java | 37 +- .../s3/operator/MultiPartWriterTest.java | 62 +- .../stream/s3/operator/ProxyWriterTest.java | 6 +- .../stream/s3/utils/AsyncRateLimiterTest.java | 5 +- .../stream/s3/wal/BlockWALServiceTest.java | 1129 +++++++++-------- .../wal/util/WALBlockDeviceChannelTest.java | 11 +- .../stream/s3/wal/util/WALChannelTest.java | 5 +- .../automq/stream/utils/FutureTickerTest.java | 5 +- s3stream/src/test/resources/log4j.properties | 2 - 146 files changed, 3148 insertions(+), 3196 deletions(-) diff --git a/s3stream/pom.xml b/s3stream/pom.xml index 49b7c9e54..669e87e11 100644 --- a/s3stream/pom.xml +++ b/s3stream/pom.xml @@ -16,8 +16,8 @@ ~ limitations under the License. --> - 4.0.0 com.automq.elasticstream diff --git a/s3stream/src/main/java/com/automq/stream/DefaultRecordBatch.java b/s3stream/src/main/java/com/automq/stream/DefaultRecordBatch.java index 5991572d0..3e60bb546 100644 --- a/s3stream/src/main/java/com/automq/stream/DefaultRecordBatch.java +++ b/s3stream/src/main/java/com/automq/stream/DefaultRecordBatch.java @@ -18,7 +18,6 @@ package com.automq.stream; import com.automq.stream.api.RecordBatch; - import java.nio.ByteBuffer; import java.util.Collections; import java.util.Map; diff --git a/s3stream/src/main/java/com/automq/stream/RecordBatchWithContextWrapper.java b/s3stream/src/main/java/com/automq/stream/RecordBatchWithContextWrapper.java index bfa53947f..b3f0dd71c 100644 --- a/s3stream/src/main/java/com/automq/stream/RecordBatchWithContextWrapper.java +++ b/s3stream/src/main/java/com/automq/stream/RecordBatchWithContextWrapper.java @@ -19,7 +19,6 @@ import com.automq.stream.api.RecordBatch; import com.automq.stream.api.RecordBatchWithContext; - import java.nio.ByteBuffer; import java.util.Collections; import java.util.Map; @@ -33,6 +32,12 @@ public RecordBatchWithContextWrapper(RecordBatch recordBatch, long baseOffset) { this.baseOffset = baseOffset; } + public static RecordBatchWithContextWrapper decode(ByteBuffer buffer) { + long baseOffset = buffer.getLong(); + int count = buffer.getInt(); + return new RecordBatchWithContextWrapper(new DefaultRecordBatch(count, 0, Collections.emptyMap(), buffer), baseOffset); + } + @Override public long baseOffset() { return baseOffset; @@ -65,16 +70,10 @@ public ByteBuffer rawPayload() { public byte[] encode() { ByteBuffer buffer = ByteBuffer.allocate(8 + 4 + recordBatch.rawPayload().remaining()) - .putLong(baseOffset) - .putInt(recordBatch.count()) - .put(recordBatch.rawPayload().duplicate()) - .flip(); + .putLong(baseOffset) + .putInt(recordBatch.count()) + .put(recordBatch.rawPayload().duplicate()) + .flip(); return buffer.array(); } - - public static RecordBatchWithContextWrapper decode(ByteBuffer buffer) { - long baseOffset = buffer.getLong(); - int count = buffer.getInt(); - return new RecordBatchWithContextWrapper(new DefaultRecordBatch(count, 0, Collections.emptyMap(), buffer), baseOffset); - } } diff --git a/s3stream/src/main/java/com/automq/stream/api/Client.java b/s3stream/src/main/java/com/automq/stream/api/Client.java index 123713f55..9ca1f48a1 100644 --- a/s3stream/src/main/java/com/automq/stream/api/Client.java +++ b/s3stream/src/main/java/com/automq/stream/api/Client.java @@ -24,6 +24,7 @@ public interface Client { void start(); void shutdown(); + /** * Get stream client. * diff --git a/s3stream/src/main/java/com/automq/stream/api/FetchResult.java b/s3stream/src/main/java/com/automq/stream/api/FetchResult.java index e82cf145f..6832f7d3f 100644 --- a/s3stream/src/main/java/com/automq/stream/api/FetchResult.java +++ b/s3stream/src/main/java/com/automq/stream/api/FetchResult.java @@ -18,7 +18,6 @@ package com.automq.stream.api; import com.automq.stream.s3.cache.CacheAccessType; - import java.util.List; public interface FetchResult { diff --git a/s3stream/src/main/java/com/automq/stream/api/KVClient.java b/s3stream/src/main/java/com/automq/stream/api/KVClient.java index 1f6a82272..19b98bb2d 100644 --- a/s3stream/src/main/java/com/automq/stream/api/KVClient.java +++ b/s3stream/src/main/java/com/automq/stream/api/KVClient.java @@ -35,6 +35,7 @@ public interface KVClient { /** * Put key value, overwrite if key exist, return current key value after putting. + * * @param keyValue {@link KeyValue} k-v pair * @return async put result. {@link KeyValue} current value after putting. */ diff --git a/s3stream/src/main/java/com/automq/stream/api/KeyValue.java b/s3stream/src/main/java/com/automq/stream/api/KeyValue.java index d0e72f239..1d21f70a5 100644 --- a/s3stream/src/main/java/com/automq/stream/api/KeyValue.java +++ b/s3stream/src/main/java/com/automq/stream/api/KeyValue.java @@ -17,7 +17,6 @@ package com.automq.stream.api; - import java.nio.ByteBuffer; import java.util.Objects; @@ -44,8 +43,10 @@ public Value value() { @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; KeyValue keyValue = (KeyValue) o; return Objects.equals(key, keyValue.key) && Objects.equals(value, keyValue.value); } @@ -131,8 +132,10 @@ public boolean isNull() { @Override public boolean equals(Object o) { - if (this == o) return true; - if (!(o instanceof Value)) return false; + if (this == o) + return true; + if (!(o instanceof Value)) + return false; Value value1 = (Value) o; return Objects.equals(value, value1.value); } diff --git a/s3stream/src/main/java/com/automq/stream/api/ReadOptions.java b/s3stream/src/main/java/com/automq/stream/api/ReadOptions.java index 52bdaaf4d..45c99c733 100644 --- a/s3stream/src/main/java/com/automq/stream/api/ReadOptions.java +++ b/s3stream/src/main/java/com/automq/stream/api/ReadOptions.java @@ -25,6 +25,10 @@ public class ReadOptions { private boolean fastRead; private boolean pooledBuf; + public static Builder builder() { + return new Builder(); + } + public boolean fastRead() { return fastRead; } @@ -33,10 +37,6 @@ public boolean pooledBuf() { return pooledBuf; } - public static Builder builder() { - return new Builder(); - } - public static class Builder { private final ReadOptions options = new ReadOptions(); diff --git a/s3stream/src/main/java/com/automq/stream/api/Stream.java b/s3stream/src/main/java/com/automq/stream/api/Stream.java index c82deb9e3..c6ad7940f 100644 --- a/s3stream/src/main/java/com/automq/stream/api/Stream.java +++ b/s3stream/src/main/java/com/automq/stream/api/Stream.java @@ -20,7 +20,6 @@ import com.automq.stream.api.exceptions.StreamClientException; import com.automq.stream.s3.context.AppendContext; import com.automq.stream.s3.context.FetchContext; - import java.util.concurrent.CompletableFuture; /** @@ -48,7 +47,6 @@ public interface Stream { */ long nextOffset(); - /** * Append recordBatch to stream. * diff --git a/s3stream/src/main/java/com/automq/stream/s3/DeltaWALUploadTask.java b/s3stream/src/main/java/com/automq/stream/s3/DeltaWALUploadTask.java index 1b8b132fa..cd45f4a05 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/DeltaWALUploadTask.java +++ b/s3stream/src/main/java/com/automq/stream/s3/DeltaWALUploadTask.java @@ -25,9 +25,6 @@ import com.automq.stream.s3.operator.S3Operator; import com.automq.stream.utils.AsyncRateLimiter; import com.automq.stream.utils.FutureUtil; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.ArrayList; import java.util.Collections; import java.util.LinkedList; @@ -37,12 +34,14 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static com.automq.stream.s3.metadata.ObjectUtils.NOOP_OBJECT_ID; public class DeltaWALUploadTask { private static final Logger LOGGER = LoggerFactory.getLogger(DeltaWALUploadTask.class); - private long startTimestamp; + final boolean forceSplit; private final Logger s3ObjectLogger; private final Map> streamRecordsMap; private final int objectBlockSize; @@ -50,17 +49,18 @@ public class DeltaWALUploadTask { private final int streamSplitSizeThreshold; private final ObjectManager objectManager; private final S3Operator s3Operator; - final boolean forceSplit; private final boolean s3ObjectLogEnable; private final CompletableFuture prepareCf = new CompletableFuture<>(); - private volatile CommitStreamSetObjectRequest commitStreamSetObjectRequest; private final CompletableFuture uploadCf = new CompletableFuture<>(); private final ExecutorService executor; private final double rate; private final AsyncRateLimiter limiter; + private long startTimestamp; + private volatile CommitStreamSetObjectRequest commitStreamSetObjectRequest; - public DeltaWALUploadTask(Config config, Map> streamRecordsMap, ObjectManager objectManager, S3Operator s3Operator, - ExecutorService executor, boolean forceSplit, double rate) { + public DeltaWALUploadTask(Config config, Map> streamRecordsMap, + ObjectManager objectManager, S3Operator s3Operator, + ExecutorService executor, boolean forceSplit, double rate) { this.s3ObjectLogger = S3ObjectLogger.logger(String.format("[DeltaWALUploadTask id=%d] ", config.nodeId())); this.streamRecordsMap = streamRecordsMap; this.objectBlockSize = config.objectBlockSize(); @@ -85,12 +85,12 @@ public CompletableFuture prepare() { prepareCf.complete(NOOP_OBJECT_ID); } else { objectManager - .prepareObject(1, TimeUnit.MINUTES.toMillis(60)) - .thenAcceptAsync(prepareCf::complete, executor) - .exceptionally(ex -> { - prepareCf.completeExceptionally(ex); - return null; - }); + .prepareObject(1, TimeUnit.MINUTES.toMillis(60)) + .thenAcceptAsync(prepareCf::complete, executor) + .exceptionally(ex -> { + prepareCf.completeExceptionally(ex); + return null; + }); } return prepareCf; } @@ -135,7 +135,7 @@ private void upload0(long objectId) { request.setObjectId(objectId); request.setOrderId(objectId); CompletableFuture streamSetObjectCf = CompletableFuture.allOf(streamSetWriteCfList.toArray(new CompletableFuture[0])) - .thenCompose(nil -> streamSetObject.close().thenAccept(nil2 -> request.setObjectSize(streamSetObject.size()))); + .thenCompose(nil -> streamSetObject.close().thenAccept(nil2 -> request.setObjectSize(streamSetObject.size()))); List> allCf = new LinkedList<>(streamObjectCfList); allCf.add(streamSetObjectCf); CompletableFuture.allOf(allCf.toArray(new CompletableFuture[0])).thenAccept(nil -> { @@ -150,7 +150,7 @@ private void upload0(long objectId) { public CompletableFuture commit() { return uploadCf.thenCompose(request -> objectManager.commitStreamSetObject(request).thenAccept(resp -> { LOGGER.info("Upload delta WAL {}, cost {}ms, rate limiter {}bytes/s", commitStreamSetObjectRequest, - System.currentTimeMillis() - startTimestamp, rate); + System.currentTimeMillis() - startTimestamp, rate); if (s3ObjectLogEnable) { s3ObjectLogger.trace("{}", commitStreamSetObjectRequest); } @@ -227,10 +227,10 @@ public DeltaWALUploadTask build() { boolean forceSplit = streamRecordsMap.size() == 1; if (!forceSplit) { Optional hasStreamSetData = streamRecordsMap.values() - .stream() - .map(records -> records.stream().mapToLong(StreamRecordBatch::size).sum() >= config.streamSplitSize()) - .filter(split -> !split) - .findAny(); + .stream() + .map(records -> records.stream().mapToLong(StreamRecordBatch::size).sum() >= config.streamSplitSize()) + .filter(split -> !split) + .findAny(); if (hasStreamSetData.isEmpty()) { forceSplit = true; } diff --git a/s3stream/src/main/java/com/automq/stream/s3/DirectByteBufAlloc.java b/s3stream/src/main/java/com/automq/stream/s3/DirectByteBufAlloc.java index 44af1eff2..29488ffa0 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/DirectByteBufAlloc.java +++ b/s3stream/src/main/java/com/automq/stream/s3/DirectByteBufAlloc.java @@ -23,11 +23,10 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.CompositeByteBuf; import io.netty.buffer.PooledByteBufAllocator; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.ArrayList; import java.util.List; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class DirectByteBufAlloc { private static final Logger LOGGER = LoggerFactory.getLogger(DirectByteBufAlloc.class); diff --git a/s3stream/src/main/java/com/automq/stream/s3/ObjectReader.java b/s3stream/src/main/java/com/automq/stream/s3/ObjectReader.java index 40d717eec..f2adcdcaf 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/ObjectReader.java +++ b/s3stream/src/main/java/com/automq/stream/s3/ObjectReader.java @@ -24,14 +24,13 @@ import com.automq.stream.utils.CloseableIterator; import com.automq.stream.utils.biniarysearch.IndexBlockOrderedBytes; import io.netty.buffer.ByteBuf; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.LinkedList; import java.util.List; import java.util.NoSuchElementException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicInteger; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static com.automq.stream.s3.ObjectWriter.Footer.FOOTER_SIZE; import static com.automq.stream.s3.metadata.ObjectUtils.NOOP_OFFSET; @@ -128,7 +127,8 @@ public void close0() { */ public record BasicObjectInfo(long dataBlockSize, IndexBlock indexBlock, int blockCount, int indexBlockSize) { - public static BasicObjectInfo parse(ByteBuf objectTailBuf, S3ObjectMetadata s3ObjectMetadata) throws IndexBlockParseException { + public static BasicObjectInfo parse(ByteBuf objectTailBuf, + S3ObjectMetadata s3ObjectMetadata) throws IndexBlockParseException { long indexBlockPosition = objectTailBuf.getLong(objectTailBuf.readableBytes() - FOOTER_SIZE); int indexBlockSize = objectTailBuf.getInt(objectTailBuf.readableBytes() - 40); if (indexBlockPosition + objectTailBuf.readableBytes() < s3ObjectMetadata.objectSize()) { @@ -217,14 +217,14 @@ public FindIndexResult find(long streamId, long startOffset, long endOffset, int int blockSize = blocks.getInt(rangeBlockId * 16 + 8); int recordCount = blocks.getInt(rangeBlockId * 16 + 12); rst.add(new StreamDataBlock(streamId, rangeStartOffset, rangeEndOffset, s3ObjectMetadata.objectId(), - new DataBlockIndex(rangeBlockId, blockPosition, blockSize, recordCount))); + new DataBlockIndex(rangeBlockId, blockPosition, blockSize, recordCount))); // we consider first block as not matched because we do not know exactly how many bytes are within // the range in first block, as a result we may read one more block than expected. if (matched) { int recordPayloadSize = blockSize - - recordCount * StreamRecordBatchCodec.HEADER_SIZE // sum of encoded record header size - - ObjectWriter.DataBlock.BLOCK_HEADER_SIZE; // block header size + - recordCount * StreamRecordBatchCodec.HEADER_SIZE // sum of encoded record header size + - ObjectWriter.DataBlock.BLOCK_HEADER_SIZE; // block header size nextMaxBytes -= Math.min(nextMaxBytes, recordPayloadSize); } if ((endOffset != NOOP_OFFSET && nextStartOffset >= endOffset) || nextMaxBytes == 0) { @@ -272,11 +272,11 @@ public long endPosition() { @Override public String toString() { return "DataBlockIndex{" + - "blockId=" + blockId + - ", startPosition=" + startPosition + - ", size=" + size + - ", recordCount=" + recordCount + - '}'; + "blockId=" + blockId + + ", startPosition=" + startPosition + + ", size=" + size + + ", recordCount=" + recordCount + + '}'; } } diff --git a/s3stream/src/main/java/com/automq/stream/s3/ObjectWriter.java b/s3stream/src/main/java/com/automq/stream/s3/ObjectWriter.java index f86c82ffa..7753188f7 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/ObjectWriter.java +++ b/s3stream/src/main/java/com/automq/stream/s3/ObjectWriter.java @@ -24,7 +24,6 @@ import com.automq.stream.s3.operator.Writer; import io.netty.buffer.ByteBuf; import io.netty.buffer.CompositeByteBuf; - import java.util.ArrayList; import java.util.Collections; import java.util.LinkedList; @@ -42,6 +41,14 @@ public interface ObjectWriter { // TODO: first n bit is the compressed flag byte DATA_BLOCK_DEFAULT_FLAG = 0x02; + static ObjectWriter writer(long objectId, S3Operator s3Operator, int blockSizeThreshold, int partSizeThreshold) { + return new DefaultObjectWriter(objectId, s3Operator, blockSizeThreshold, partSizeThreshold); + } + + static ObjectWriter noop(long objectId) { + return new NoopObjectWriter(objectId); + } + void write(long streamId, List records); CompletableFuture close(); @@ -52,25 +59,16 @@ public interface ObjectWriter { long size(); - static ObjectWriter writer(long objectId, S3Operator s3Operator, int blockSizeThreshold, int partSizeThreshold) { - return new DefaultObjectWriter(objectId, s3Operator, blockSizeThreshold, partSizeThreshold); - } - - static ObjectWriter noop(long objectId) { - return new NoopObjectWriter(objectId); - } - class DefaultObjectWriter implements ObjectWriter { private final int blockSizeThreshold; private final int partSizeThreshold; private final List waitingUploadBlocks; - private int waitingUploadBlocksSize; private final List completedBlocks; - private IndexBlock indexBlock; private final Writer writer; private final long objectId; - + private int waitingUploadBlocksSize; + private IndexBlock indexBlock; private long size; /** @@ -81,7 +79,8 @@ class DefaultObjectWriter implements ObjectWriter { * @param blockSizeThreshold the max size of a block * @param partSizeThreshold the max size of a part. If it is smaller than {@link Writer#MIN_PART_SIZE}, it will be set to {@link Writer#MIN_PART_SIZE}. */ - public DefaultObjectWriter(long objectId, S3Operator s3Operator, int blockSizeThreshold, int partSizeThreshold) { + public DefaultObjectWriter(long objectId, S3Operator s3Operator, int blockSizeThreshold, + int partSizeThreshold) { this.objectId = objectId; String objectKey = ObjectUtils.genKey(0, objectId); this.blockSizeThreshold = blockSizeThreshold; @@ -195,7 +194,6 @@ public long size() { return size; } - class IndexBlock { private final ByteBuf buf; private final long position; diff --git a/s3stream/src/main/java/com/automq/stream/s3/S3Storage.java b/s3stream/src/main/java/com/automq/stream/s3/S3Storage.java index dd32878c0..294afd4de 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/S3Storage.java +++ b/s3stream/src/main/java/com/automq/stream/s3/S3Storage.java @@ -45,9 +45,6 @@ import io.netty.util.Timeout; import io.opentelemetry.instrumentation.annotations.SpanAttribute; import io.opentelemetry.instrumentation.annotations.WithSpan; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -75,13 +72,15 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.stream.Collectors; import java.util.stream.IntStream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static com.automq.stream.utils.FutureUtil.suppress; - public class S3Storage implements Storage { private static final Logger LOGGER = LoggerFactory.getLogger(S3Storage.class); private static final FastReadFailFastException FAST_READ_FAIL_FAST_EXCEPTION = new FastReadFailFastException(); + private static final int NUM_STREAM_CALLBACK_LOCKS = 128; private final long maxDeltaWALCacheSize; private final Config config; private final WriteAheadLog deltaWAL; @@ -97,40 +96,35 @@ public class S3Storage implements Storage { private final Queue walPrepareQueue = new LinkedList<>(); private final Queue walCommitQueue = new LinkedList<>(); private final List inflightWALUploadTasks = new CopyOnWriteArrayList<>(); - private final ScheduledExecutorService backgroundExecutor = Threads.newSingleThreadScheduledExecutor( - ThreadUtils.createThreadFactory("s3-storage-background", true), LOGGER); + ThreadUtils.createThreadFactory("s3-storage-background", true), LOGGER); private final ExecutorService uploadWALExecutor = Threads.newFixedThreadPoolWithMonitor( - 4, "s3-storage-upload-wal", true, LOGGER); - + 4, "s3-storage-upload-wal", true, LOGGER); /** * A ticker used for batching force upload WAL. + * * @see #forceUpload */ private final FutureTicker forceUploadTicker = new FutureTicker(500, TimeUnit.MILLISECONDS, backgroundExecutor); - private final Queue backoffRecords = new LinkedBlockingQueue<>(); private final ScheduledFuture drainBackoffTask; - private long lastLogTimestamp = 0L; - private final StreamManager streamManager; private final ObjectManager objectManager; private final S3Operator s3Operator; private final S3BlockCache blockCache; - private static final int NUM_STREAM_CALLBACK_LOCKS = 128; /** * Stream callback locks. Used to ensure the stream callbacks will not be called concurrently. * * @see #handleAppendCallback */ private final Lock[] streamCallbackLocks = IntStream.range(0, NUM_STREAM_CALLBACK_LOCKS).mapToObj(i -> new ReentrantLock()).toArray(Lock[]::new); - private final HashedWheelTimer timeoutDetect = new HashedWheelTimer( - ThreadUtils.createThreadFactory("storage-timeout-detect", true), 1, TimeUnit.SECONDS, 100); + ThreadUtils.createThreadFactory("storage-timeout-detect", true), 1, TimeUnit.SECONDS, 100); + private long lastLogTimestamp = 0L; private volatile double maxDataWriteRate = 0.0; public S3Storage(Config config, WriteAheadLog deltaWAL, StreamManager streamManager, ObjectManager objectManager, - S3BlockCache blockCache, S3Operator s3Operator) { + S3BlockCache blockCache, S3Operator s3Operator) { this.config = config; this.maxDeltaWALCacheSize = config.walCacheSize(); this.deltaWAL = deltaWAL; @@ -144,66 +138,13 @@ public S3Storage(Config config, WriteAheadLog deltaWAL, StreamManager streamMana S3StreamMetricsManager.registerInflightWALUploadTasksCountSupplier(this.inflightWALUploadTasks::size); } - @Override - public void startup() { - try { - LOGGER.info("S3Storage starting"); - recover(); - LOGGER.info("S3Storage start completed"); - } catch (Throwable e) { - LOGGER.error("S3Storage start fail", e); - throw new RuntimeException(e); - } - } - - /** - * Upload WAL to S3 and close opening streams. - */ - public void recover() throws Throwable { - recover0(this.deltaWAL, this.streamManager, this.objectManager, LOGGER); - } - - public void recover(WriteAheadLog deltaWAL, StreamManager streamManager, ObjectManager objectManager, Logger logger) throws Throwable { - recover0(deltaWAL, streamManager, objectManager, logger); - } - - void recover0(WriteAheadLog deltaWAL, StreamManager streamManager, ObjectManager objectManager, Logger logger) throws Throwable { - deltaWAL.start(); - List streams = streamManager.getOpeningStreams().get(); - - LogCache.LogCacheBlock cacheBlock = recoverContinuousRecords(deltaWAL.recover(), streams, logger); - Map streamEndOffsets = new HashMap<>(); - cacheBlock.records().forEach((streamId, records) -> { - if (!records.isEmpty()) { - streamEndOffsets.put(streamId, records.get(records.size() - 1).getLastOffset()); - } - }); - - if (cacheBlock.size() != 0) { - logger.info("try recover from crash, recover records bytes size {}", cacheBlock.size()); - DeltaWALUploadTask task = DeltaWALUploadTask.builder().config(config).streamRecordsMap(cacheBlock.records()) - .objectManager(objectManager).s3Operator(s3Operator).executor(uploadWALExecutor).build(); - task.prepare().thenCompose(nil -> task.upload()).thenCompose(nil -> task.commit()).get(); - cacheBlock.records().forEach((streamId, records) -> records.forEach(StreamRecordBatch::release)); - } - deltaWAL.reset().get(); - for (StreamMetadata stream : streams) { - long newEndOffset = streamEndOffsets.getOrDefault(stream.streamId(), stream.endOffset()); - logger.info("recover try close stream {} with new end offset {}", stream, newEndOffset); - } - CompletableFuture.allOf( - streams - .stream() - .map(s -> streamManager.closeStream(s.streamId(), s.epoch())) - .toArray(CompletableFuture[]::new) - ).get(); - } - - static LogCache.LogCacheBlock recoverContinuousRecords(Iterator it, List openingStreams) { + static LogCache.LogCacheBlock recoverContinuousRecords(Iterator it, + List openingStreams) { return recoverContinuousRecords(it, openingStreams, LOGGER); } - static LogCache.LogCacheBlock recoverContinuousRecords(Iterator it, List openingStreams, Logger logger) { + static LogCache.LogCacheBlock recoverContinuousRecords(Iterator it, + List openingStreams, Logger logger) { Map openingStreamEndOffsets = openingStreams.stream().collect(Collectors.toMap(StreamMetadata::streamId, StreamMetadata::endOffset)); LogCache.LogCacheBlock cacheBlock = new LogCache.LogCacheBlock(1024L * 1024 * 1024); long logEndOffset = -1L; @@ -243,7 +184,7 @@ static LogCache.LogCacheBlock recoverContinuousRecords(Iterator streams = streamManager.getOpeningStreams().get(); + + LogCache.LogCacheBlock cacheBlock = recoverContinuousRecords(deltaWAL.recover(), streams, logger); + Map streamEndOffsets = new HashMap<>(); + cacheBlock.records().forEach((streamId, records) -> { + if (!records.isEmpty()) { + streamEndOffsets.put(streamId, records.get(records.size() - 1).getLastOffset()); + } + }); + + if (cacheBlock.size() != 0) { + logger.info("try recover from crash, recover records bytes size {}", cacheBlock.size()); + DeltaWALUploadTask task = DeltaWALUploadTask.builder().config(config).streamRecordsMap(cacheBlock.records()) + .objectManager(objectManager).s3Operator(s3Operator).executor(uploadWALExecutor).build(); + task.prepare().thenCompose(nil -> task.upload()).thenCompose(nil -> task.commit()).get(); + cacheBlock.records().forEach((streamId, records) -> records.forEach(StreamRecordBatch::release)); + } + deltaWAL.reset().get(); + for (StreamMetadata stream : streams) { + long newEndOffset = streamEndOffsets.getOrDefault(stream.streamId(), stream.endOffset()); + logger.info("recover try close stream {} with new end offset {}", stream, newEndOffset); + } + CompletableFuture.allOf( + streams + .stream() + .map(s -> streamManager.closeStream(s.streamId(), s.epoch())) + .toArray(CompletableFuture[]::new) + ).get(); + } + @Override public void shutdown() { drainBackoffTask.cancel(false); @@ -262,7 +260,6 @@ public void shutdown() { backgroundExecutor.shutdown(); } - @Override @WithSpan public CompletableFuture append(AppendContext context, StreamRecordBatch streamRecord) { @@ -364,10 +361,10 @@ private void tryDrainBackoffRecords() { @Override @WithSpan public CompletableFuture read(FetchContext context, - @SpanAttribute long streamId, - @SpanAttribute long startOffset, - @SpanAttribute long endOffset, - @SpanAttribute int maxBytes) { + @SpanAttribute long streamId, + @SpanAttribute long startOffset, + @SpanAttribute long endOffset, + @SpanAttribute int maxBytes) { TimerUtil timerUtil = new TimerUtil(); CompletableFuture cf = new CompletableFuture<>(); FutureUtil.propagate(read0(context, streamId, startOffset, endOffset, maxBytes), cf); @@ -377,10 +374,10 @@ public CompletableFuture read(FetchContext context, @WithSpan private CompletableFuture read0(FetchContext context, - @SpanAttribute long streamId, - @SpanAttribute long startOffset, - @SpanAttribute long endOffset, - @SpanAttribute int maxBytes) { + @SpanAttribute long streamId, + @SpanAttribute long startOffset, + @SpanAttribute long endOffset, + @SpanAttribute int maxBytes) { List logCacheRecords = deltaWALCache.get(context, streamId, startOffset, endOffset, maxBytes); if (!logCacheRecords.isEmpty() && logCacheRecords.get(0).getBaseOffset() <= startOffset) { return CompletableFuture.completedFuture(new ReadDataBlock(logCacheRecords, CacheAccessType.DELTA_WAL_CACHE_HIT)); @@ -421,7 +418,7 @@ private CompletableFuture read0(FetchContext context, timeout.cancel(); if (ex != null) { LOGGER.error("read from block cache failed, stream={}, {}-{}, maxBytes: {}", - streamId, startOffset, finalEndOffset, maxBytes, ex); + streamId, startOffset, finalEndOffset, maxBytes, ex); logCacheRecords.forEach(StreamRecordBatch::release); } }); @@ -434,7 +431,7 @@ private void continuousCheck(List records) { expectStartOffset = record.getLastOffset(); } else { throw new IllegalArgumentException(String.format("Continuous check failed, expect offset: %d," + - " actual: %d, records: %s", expectStartOffset, record.getBaseOffset(), records)); + " actual: %d, records: %s", expectStartOffset, record.getBaseOffset(), records)); } } } @@ -453,9 +450,9 @@ public CompletableFuture forceUpload(long streamId) { uploadDeltaWAL(streamId, true); // Wait for all tasks contains streamId complete. List> tasksContainsStream = this.inflightWALUploadTasks.stream() - .filter(it -> it.cache.containsStream(streamId)) - .map(it -> it.cf) - .toList(); + .filter(it -> it.cache.containsStream(streamId)) + .map(it -> it.cf) + .toList(); FutureUtil.propagate(CompletableFuture.allOf(tasksContainsStream.toArray(new CompletableFuture[0])), cf); if (LogCache.MATCH_ALL_STREAMS != streamId) { callbackSequencer.tryFree(streamId); @@ -547,7 +544,6 @@ CompletableFuture uploadDeltaWAL(DeltaWALUploadTaskContext context) { return cf; } - private void uploadDeltaWAL0(DeltaWALUploadTaskContext context) { // calculate upload rate long elapsed = System.currentTimeMillis() - context.cache.createdTimestamp(); @@ -562,13 +558,13 @@ private void uploadDeltaWAL0(DeltaWALUploadTaskContext context) { rate = maxDataWriteRate; } context.task = DeltaWALUploadTask.builder() - .config(config) - .streamRecordsMap(context.cache.records()) - .objectManager(objectManager) - .s3Operator(s3Operator) - .executor(uploadWALExecutor) - .rate(rate) - .build(); + .config(config) + .streamRecordsMap(context.cache.records()) + .objectManager(objectManager) + .s3Operator(s3Operator) + .executor(uploadWALExecutor) + .rate(rate) + .build(); boolean walObjectPrepareQueueEmpty = walPrepareQueue.isEmpty(); walPrepareQueue.add(context); if (!walObjectPrepareQueueEmpty) { @@ -584,7 +580,7 @@ private void prepareDeltaWALUpload(DeltaWALUploadTaskContext context) { // 1. poll out current task and trigger upload. DeltaWALUploadTaskContext peek = walPrepareQueue.poll(); Objects.requireNonNull(peek).task.upload().thenAccept(nil2 -> S3StreamMetricsManager.recordStageLatency( - MetricsLevel.DEBUG, context.timer.elapsedAs(TimeUnit.NANOSECONDS), S3Stage.UPLOAD_WAL_UPLOAD)); + MetricsLevel.DEBUG, context.timer.elapsedAs(TimeUnit.NANOSECONDS), S3Stage.UPLOAD_WAL_UPLOAD)); // 2. add task to commit queue. boolean walObjectCommitQueueEmpty = walCommitQueue.isEmpty(); walCommitQueue.add(peek); @@ -644,7 +640,7 @@ static class WALConfirmOffsetCalculator { public WALConfirmOffsetCalculator() { // Update the confirmed offset periodically. Threads.newSingleThreadScheduledExecutor(ThreadUtils.createThreadFactory("wal-calculator-update-confirm-offset", true), LOGGER) - .scheduleAtFixedRate(this::update, 100, 100, TimeUnit.MILLISECONDS); + .scheduleAtFixedRate(this::update, 100, 100, TimeUnit.MILLISECONDS); } /** @@ -760,7 +756,7 @@ static class WALCallbackSequencer { public void before(WalWriteRequest request) { try { Queue streamRequests = stream2requests.computeIfAbsent(request.record.getStreamId(), - s -> new ConcurrentLinkedQueue<>()); + s -> new ConcurrentLinkedQueue<>()); streamRequests.add(request); } catch (Throwable ex) { request.cf.completeExceptionally(ex); @@ -814,19 +810,6 @@ public void tryFree(long streamId) { } } - class LogCacheEvictOOMHandler implements DirectByteBufAlloc.OOMHandler { - @Override - public int handle(int memoryRequired) { - try { - CompletableFuture cf = new CompletableFuture<>(); - FutureUtil.exec(() -> cf.complete(deltaWALCache.forceFree(memoryRequired)), cf, LOGGER, "handleOOM"); - return cf.get(); - } catch (Throwable e) { - return 0; - } - } - } - public static class DeltaWALUploadTaskContext { TimerUtil timer; LogCache.LogCacheBlock cache; @@ -843,4 +826,17 @@ public DeltaWALUploadTaskContext(LogCache.LogCacheBlock cache) { this.cache = cache; } } + + class LogCacheEvictOOMHandler implements DirectByteBufAlloc.OOMHandler { + @Override + public int handle(int memoryRequired) { + try { + CompletableFuture cf = new CompletableFuture<>(); + FutureUtil.exec(() -> cf.complete(deltaWALCache.forceFree(memoryRequired)), cf, LOGGER, "handleOOM"); + return cf.get(); + } catch (Throwable e) { + return 0; + } + } + } } diff --git a/s3stream/src/main/java/com/automq/stream/s3/S3Stream.java b/s3stream/src/main/java/com/automq/stream/s3/S3Stream.java index 1ad1e03bb..310a75d30 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/S3Stream.java +++ b/s3stream/src/main/java/com/automq/stream/s3/S3Stream.java @@ -42,9 +42,6 @@ import io.netty.buffer.Unpooled; import io.opentelemetry.instrumentation.annotations.SpanAttribute; import io.opentelemetry.instrumentation.annotations.WithSpan; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; @@ -61,42 +58,44 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Function; import java.util.stream.Collectors; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static com.automq.stream.utils.FutureUtil.exec; import static com.automq.stream.utils.FutureUtil.propagate; public class S3Stream implements Stream { private static final Logger LOGGER = LoggerFactory.getLogger(S3Stream.class); + final AtomicLong confirmOffset; private final String logIdent; private final long streamId; private final long epoch; - private long startOffset; - final AtomicLong confirmOffset; private final AtomicLong nextOffset; private final Storage storage; private final StreamManager streamManager; private final Status status; private final Function closeHook; private final StreamObjectsCompactionTask streamObjectsCompactionTask; - private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); private final ReentrantReadWriteLock.WriteLock writeLock = lock.writeLock(); private final ReentrantReadWriteLock.ReadLock readLock = lock.readLock(); - private final Set> pendingAppends = ConcurrentHashMap.newKeySet(); private final Set> pendingFetches = ConcurrentHashMap.newKeySet(); private final AsyncNetworkBandwidthLimiter networkInboundLimiter; private final AsyncNetworkBandwidthLimiter networkOutboundLimiter; + private long startOffset; private CompletableFuture lastPendingTrim = CompletableFuture.completedFuture(null); - public S3Stream(long streamId, long epoch, long startOffset, long nextOffset, Storage storage, StreamManager streamManager, - StreamObjectsCompactionTask.Builder compactionTaskBuilder, Function closeHook) { + public S3Stream(long streamId, long epoch, long startOffset, long nextOffset, Storage storage, + StreamManager streamManager, + StreamObjectsCompactionTask.Builder compactionTaskBuilder, Function closeHook) { this(streamId, epoch, startOffset, nextOffset, storage, streamManager, compactionTaskBuilder, closeHook, null, null); } - public S3Stream(long streamId, long epoch, long startOffset, long nextOffset, Storage storage, StreamManager streamManager, - StreamObjectsCompactionTask.Builder compactionTaskBuilder, Function closeHook, - AsyncNetworkBandwidthLimiter networkInboundLimiter, AsyncNetworkBandwidthLimiter networkOutboundLimiter) { + public S3Stream(long streamId, long epoch, long startOffset, long nextOffset, Storage storage, + StreamManager streamManager, + StreamObjectsCompactionTask.Builder compactionTaskBuilder, Function closeHook, + AsyncNetworkBandwidthLimiter networkInboundLimiter, AsyncNetworkBandwidthLimiter networkOutboundLimiter) { this.streamId = streamId; this.epoch = epoch; this.startOffset = startOffset; @@ -194,9 +193,9 @@ private CompletableFuture append0(AppendContext context, RecordBat @Override @WithSpan public CompletableFuture fetch(FetchContext context, - @SpanAttribute long startOffset, - @SpanAttribute long endOffset, - @SpanAttribute int maxBytes) { + @SpanAttribute long startOffset, + @SpanAttribute long endOffset, + @SpanAttribute int maxBytes) { TimerUtil timerUtil = new TimerUtil(); readLock.lock(); S3StreamMetricsManager.recordOperationLatency(MetricsLevel.DEBUG, timerUtil.elapsedAs(TimeUnit.NANOSECONDS), S3Operation.FETCH_STREAM_READ_LOCK); @@ -215,7 +214,7 @@ public CompletableFuture fetch(FetchContext context, networkOutboundLimiter.forceConsume(totalSize); if (LOGGER.isDebugEnabled()) { LOGGER.debug("[S3BlockCache] fetch data, stream={}, {}-{}, total bytes: {}, cost={}ms", streamId, - startOffset, endOffset, totalSize, timerUtil.elapsedAs(TimeUnit.MILLISECONDS)); + startOffset, endOffset, totalSize, timerUtil.elapsedAs(TimeUnit.MILLISECONDS)); } } pendingFetches.remove(cf); @@ -227,7 +226,8 @@ public CompletableFuture fetch(FetchContext context, } @WithSpan - private CompletableFuture fetch0(FetchContext context, long startOffset, long endOffset, int maxBytes) { + private CompletableFuture fetch0(FetchContext context, long startOffset, long endOffset, + int maxBytes) { if (!status.isReadable()) { return FutureUtil.failedFuture(new StreamClientException(ErrorCode.STREAM_ALREADY_CLOSED, logIdent + " stream is already closed")); } @@ -237,10 +237,10 @@ private CompletableFuture fetch0(FetchContext context, long startOf long confirmOffset = this.confirmOffset.get(); if (startOffset < startOffset() || endOffset > confirmOffset) { return FutureUtil.failedFuture( - new StreamClientException( - ErrorCode.OFFSET_OUT_OF_RANGE_BOUNDS, - String.format("fetch range[%s, %s) is out of stream bound [%s, %s)", startOffset, endOffset, startOffset(), confirmOffset) - )); + new StreamClientException( + ErrorCode.OFFSET_OUT_OF_RANGE_BOUNDS, + String.format("fetch range[%s, %s) is out of stream bound [%s, %s)", startOffset, endOffset, startOffset(), confirmOffset) + )); } if (startOffset > endOffset) { return FutureUtil.failedFuture(new IllegalArgumentException(String.format("fetch startOffset %s is greater than endOffset %s", startOffset, endOffset))); @@ -330,8 +330,8 @@ public CompletableFuture close() { private CompletableFuture close0() { streamObjectsCompactionTask.close(); return storage.forceUpload(streamId) - .thenCompose(nil -> streamManager.closeStream(streamId, epoch)) - .whenComplete((nil, ex) -> closeHook.apply(streamId)); + .thenCompose(nil -> streamManager.closeStream(streamId, epoch)) + .whenComplete((nil, ex) -> closeHook.apply(streamId)); } @Override @@ -381,7 +381,8 @@ static class DefaultFetchResult implements FetchResult { private final boolean pooledBuf; private volatile boolean freed = false; - public DefaultFetchResult(List streamRecords, CacheAccessType cacheAccessType, boolean pooledBuf) { + public DefaultFetchResult(List streamRecords, CacheAccessType cacheAccessType, + boolean pooledBuf) { this.pooledRecords = streamRecords; this.pooledBuf = pooledBuf; this.records = streamRecords.stream().map(r -> new RecordBatchWithContextWrapper(covert(r, pooledBuf), r.getBaseOffset())).collect(Collectors.toList()); @@ -393,25 +394,6 @@ public DefaultFetchResult(List streamRecords, CacheAccessType } } - @Override - public List recordBatchList() { - return records; - } - - @Override - public CacheAccessType getCacheAccessType() { - return cacheAccessType; - } - - @Override - public void free() { - if (!freed && pooledBuf) { - pooledRecords.forEach(StreamRecordBatch::release); - INFLIGHT.decrement(); - } - freed = true; - } - private static RecordBatch covert(StreamRecordBatch streamRecordBatch, boolean pooledBuf) { ByteBuffer buf; if (pooledBuf) { @@ -443,6 +425,25 @@ public ByteBuffer rawPayload() { } }; } + + @Override + public List recordBatchList() { + return records; + } + + @Override + public CacheAccessType getCacheAccessType() { + return cacheAccessType; + } + + @Override + public void free() { + if (!freed && pooledBuf) { + pooledRecords.forEach(StreamRecordBatch::release); + INFLIGHT.decrement(); + } + freed = true; + } } static class Status { diff --git a/s3stream/src/main/java/com/automq/stream/s3/S3StreamClient.java b/s3stream/src/main/java/com/automq/stream/s3/S3StreamClient.java index 629a87c38..c7cfc462c 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/S3StreamClient.java +++ b/s3stream/src/main/java/com/automq/stream/s3/S3StreamClient.java @@ -49,12 +49,10 @@ public class S3StreamClient implements StreamClient { private static final Logger LOGGER = LoggerFactory.getLogger(S3StreamClient.class); private final ScheduledThreadPoolExecutor streamObjectCompactionScheduler = Threads.newSingleThreadScheduledExecutor( - ThreadUtils.createThreadFactory("stream-object-compaction-scheduler", true), LOGGER, true); + ThreadUtils.createThreadFactory("stream-object-compaction-scheduler", true), LOGGER, true); private final ExecutorService streamCompactionExecutor = Threads.newFixedThreadPool(1, - ThreadUtils.createThreadFactory("stream-object-compaction-background", true), LOGGER); - private ScheduledFuture scheduledCompactionTaskFuture; + ThreadUtils.createThreadFactory("stream-object-compaction-background", true), LOGGER); private final Map openedStreams; - private final StreamManager streamManager; private final Storage storage; private final ObjectManager objectManager; @@ -62,13 +60,16 @@ public class S3StreamClient implements StreamClient { private final Config config; private final AsyncNetworkBandwidthLimiter networkInboundBucket; private final AsyncNetworkBandwidthLimiter networkOutboundBucket; + private ScheduledFuture scheduledCompactionTaskFuture; - public S3StreamClient(StreamManager streamManager, Storage storage, ObjectManager objectManager, S3Operator s3Operator, Config config) { + public S3StreamClient(StreamManager streamManager, Storage storage, ObjectManager objectManager, + S3Operator s3Operator, Config config) { this(streamManager, storage, objectManager, s3Operator, config, null, null); } - public S3StreamClient(StreamManager streamManager, Storage storage, ObjectManager objectManager, S3Operator s3Operator, Config config, - AsyncNetworkBandwidthLimiter networkInboundBucket, AsyncNetworkBandwidthLimiter networkOutboundBucket) { + public S3StreamClient(StreamManager streamManager, Storage storage, ObjectManager objectManager, + S3Operator s3Operator, Config config, + AsyncNetworkBandwidthLimiter networkInboundBucket, AsyncNetworkBandwidthLimiter networkOutboundBucket) { this.streamManager = streamManager; this.storage = storage; this.openedStreams = new ConcurrentHashMap<>(); @@ -138,22 +139,22 @@ private void startStreamObjectsCompactions() { private CompletableFuture openStream0(long streamId, long epoch) { TimerUtil timerUtil = new TimerUtil(); return streamManager.openStream(streamId, epoch). - thenApply(metadata -> { - S3StreamMetricsManager.recordOperationLatency(MetricsLevel.INFO, timerUtil.elapsedAs(TimeUnit.NANOSECONDS), S3Operation.OPEN_STREAM); - StreamObjectsCompactionTask.Builder builder = new StreamObjectsCompactionTask.Builder(objectManager, s3Operator) - .compactedStreamObjectMaxSizeInBytes(config.streamObjectCompactionMaxSizeBytes()) - .eligibleStreamObjectLivingTimeInMs(config.streamObjectCompactionLivingTimeMinutes() * 60L * 1000) - .s3ObjectLogEnabled(config.objectLogEnable()).executor(streamCompactionExecutor); - S3Stream stream = new S3Stream( - metadata.streamId(), metadata.epoch(), - metadata.startOffset(), metadata.endOffset(), - storage, streamManager, builder, id -> { - openedStreams.remove(id); - return null; - }, networkInboundBucket, networkOutboundBucket); - openedStreams.put(streamId, stream); - return stream; - }); + thenApply(metadata -> { + S3StreamMetricsManager.recordOperationLatency(MetricsLevel.INFO, timerUtil.elapsedAs(TimeUnit.NANOSECONDS), S3Operation.OPEN_STREAM); + StreamObjectsCompactionTask.Builder builder = new StreamObjectsCompactionTask.Builder(objectManager, s3Operator) + .compactedStreamObjectMaxSizeInBytes(config.streamObjectCompactionMaxSizeBytes()) + .eligibleStreamObjectLivingTimeInMs(config.streamObjectCompactionLivingTimeMinutes() * 60L * 1000) + .s3ObjectLogEnabled(config.objectLogEnable()).executor(streamCompactionExecutor); + S3Stream stream = new S3Stream( + metadata.streamId(), metadata.epoch(), + metadata.startOffset(), metadata.endOffset(), + storage, streamManager, builder, id -> { + openedStreams.remove(id); + return null; + }, networkInboundBucket, networkOutboundBucket); + openedStreams.put(streamId, stream); + return stream; + }); } @Override @@ -194,8 +195,8 @@ private static class CompactionTasksSummary { private final long timeCostInMs; private CompactionTasksSummary(long involvedStreamCount, long sourceObjectsTotalSize, long sourceObjectsCount, - long targetObjectsCount, long smallSizeCopyWriteCount, - long timeCostInMs) { + long targetObjectsCount, long smallSizeCopyWriteCount, + long timeCostInMs) { this.involvedStreamCount = involvedStreamCount; this.sourceObjectsTotalSize = sourceObjectsTotalSize; this.sourceObjectsCount = sourceObjectsCount; @@ -211,13 +212,13 @@ public static Builder builder() { @Override public String toString() { return "CompactionTasksSummary{" + - "involvedStreamCount=" + involvedStreamCount + - ", sourceObjectsTotalSize=" + sourceObjectsTotalSize + - ", sourceObjectsCount=" + sourceObjectsCount + - ", targetObjectsCount=" + targetObjectsCount + - ", smallSizeCopyWriteCount=" + smallSizeCopyWriteCount + - ", timeCostInMs=" + timeCostInMs + - '}'; + "involvedStreamCount=" + involvedStreamCount + + ", sourceObjectsTotalSize=" + sourceObjectsTotalSize + + ", sourceObjectsCount=" + sourceObjectsCount + + ", targetObjectsCount=" + targetObjectsCount + + ", smallSizeCopyWriteCount=" + smallSizeCopyWriteCount + + ", timeCostInMs=" + timeCostInMs + + '}'; } public static class Builder { diff --git a/s3stream/src/main/java/com/automq/stream/s3/Storage.java b/s3stream/src/main/java/com/automq/stream/s3/Storage.java index ea398f7ac..cde3619bf 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/Storage.java +++ b/s3stream/src/main/java/com/automq/stream/s3/Storage.java @@ -21,7 +21,6 @@ import com.automq.stream.s3.context.AppendContext; import com.automq.stream.s3.context.FetchContext; import com.automq.stream.s3.model.StreamRecordBatch; - import java.util.concurrent.CompletableFuture; /** @@ -44,7 +43,8 @@ default CompletableFuture append(StreamRecordBatch streamRecord) { return append(AppendContext.DEFAULT, streamRecord); } - CompletableFuture read(FetchContext context, long streamId, long startOffset, long endOffset, int maxBytes); + CompletableFuture read(FetchContext context, long streamId, long startOffset, long endOffset, + int maxBytes); default CompletableFuture read(long streamId, long startOffset, long endOffset, int maxBytes) { return read(FetchContext.DEFAULT, streamId, startOffset, endOffset, maxBytes); diff --git a/s3stream/src/main/java/com/automq/stream/s3/StreamDataBlock.java b/s3stream/src/main/java/com/automq/stream/s3/StreamDataBlock.java index b3c3e5d82..3fb8ed413 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/StreamDataBlock.java +++ b/s3stream/src/main/java/com/automq/stream/s3/StreamDataBlock.java @@ -18,7 +18,6 @@ package com.automq.stream.s3; import io.netty.buffer.ByteBuf; - import java.util.Comparator; import java.util.Objects; import java.util.concurrent.CompletableFuture; @@ -38,7 +37,8 @@ public class StreamDataBlock { private final CompletableFuture dataCf = new CompletableFuture<>(); private final AtomicInteger refCount = new AtomicInteger(1); - public StreamDataBlock(long streamId, long startOffset, long endOffset, long objectId, ObjectReader.DataBlockIndex dataBlockIndex) { + public StreamDataBlock(long streamId, long startOffset, long endOffset, long objectId, + ObjectReader.DataBlockIndex dataBlockIndex) { this.streamId = streamId; this.startOffset = startOffset; this.endOffset = endOffset; @@ -47,7 +47,7 @@ public StreamDataBlock(long streamId, long startOffset, long endOffset, long obj } public StreamDataBlock(long streamId, long startOffset, long endOffset, int blockId, - long objectId, long blockPosition, int blockSize, int recordCount) { + long objectId, long blockPosition, int blockSize, int recordCount) { this.streamId = streamId; this.startOffset = startOffset; this.endOffset = endOffset; @@ -120,21 +120,23 @@ public void release() { @Override public String toString() { return "StreamDataBlock{" + - "objectId=" + objectId + - ", streamId=" + streamId + - ", startOffset=" + startOffset + - ", endOffset=" + endOffset + - ", dataBlockIndex=" + dataBlockIndex + - '}'; + "objectId=" + objectId + + ", streamId=" + streamId + + ", startOffset=" + startOffset + + ", endOffset=" + endOffset + + ", dataBlockIndex=" + dataBlockIndex + + '}'; } @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; StreamDataBlock that = (StreamDataBlock) o; return streamId == that.streamId && startOffset == that.startOffset && endOffset == that.endOffset - && objectId == that.objectId && dataBlockIndex.equals(that.dataBlockIndex); + && objectId == that.objectId && dataBlockIndex.equals(that.dataBlockIndex); } @Override diff --git a/s3stream/src/main/java/com/automq/stream/s3/StreamObjectCopier.java b/s3stream/src/main/java/com/automq/stream/s3/StreamObjectCopier.java index effe193e3..1bb1f8714 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/StreamObjectCopier.java +++ b/s3stream/src/main/java/com/automq/stream/s3/StreamObjectCopier.java @@ -17,15 +17,14 @@ package com.automq.stream.s3; +import com.automq.stream.s3.metadata.ObjectUtils; +import com.automq.stream.s3.metadata.S3ObjectMetadata; +import com.automq.stream.s3.metadata.S3ObjectType; import com.automq.stream.s3.network.ThrottleStrategy; import com.automq.stream.s3.operator.S3Operator; import com.automq.stream.s3.operator.Writer; import io.netty.buffer.ByteBuf; import io.netty.buffer.CompositeByteBuf; -import com.automq.stream.s3.metadata.ObjectUtils; -import com.automq.stream.s3.metadata.S3ObjectMetadata; -import com.automq.stream.s3.metadata.S3ObjectType; - import java.util.LinkedList; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -117,38 +116,6 @@ public long size() { return size; } - private class IndexBlock { - private final CompositeByteBuf buf; - private final long position; - - public IndexBlock() { - position = nextObjectDataStartPosition; - buf = DirectByteBufAlloc.compositeByteBuffer(); - // block count - buf.addComponent(true, DirectByteBufAlloc.byteBuffer(4).writeInt(blockCount)); - // block index - for (StreamObjectIndexData indexData : completedObjects) { - buf.addComponent(true, indexData.blockBuf()); - } - // object stream range - for (StreamObjectIndexData indexData : completedObjects) { - buf.addComponent(true, indexData.rangesBuf()); - } - } - - public ByteBuf buffer() { - return buf.duplicate(); - } - - public long position() { - return position; - } - - public int size() { - return buf.readableBytes(); - } - } - static class StreamObjectIndexData { private final ByteBuf blockBuf; private final ByteBuf rangesBuf; @@ -184,4 +151,36 @@ public ByteBuf rangesBuf() { return rangesBuf.duplicate(); } } + + private class IndexBlock { + private final CompositeByteBuf buf; + private final long position; + + public IndexBlock() { + position = nextObjectDataStartPosition; + buf = DirectByteBufAlloc.compositeByteBuffer(); + // block count + buf.addComponent(true, DirectByteBufAlloc.byteBuffer(4).writeInt(blockCount)); + // block index + for (StreamObjectIndexData indexData : completedObjects) { + buf.addComponent(true, indexData.blockBuf()); + } + // object stream range + for (StreamObjectIndexData indexData : completedObjects) { + buf.addComponent(true, indexData.rangesBuf()); + } + } + + public ByteBuf buffer() { + return buf.duplicate(); + } + + public long position() { + return position; + } + + public int size() { + return buf.readableBytes(); + } + } } diff --git a/s3stream/src/main/java/com/automq/stream/s3/StreamObjectsCompactionTask.java b/s3stream/src/main/java/com/automq/stream/s3/StreamObjectsCompactionTask.java index daf3622d9..8681c66ab 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/StreamObjectsCompactionTask.java +++ b/s3stream/src/main/java/com/automq/stream/s3/StreamObjectsCompactionTask.java @@ -17,19 +17,12 @@ package com.automq.stream.s3; +import com.automq.stream.s3.metadata.S3ObjectMetadata; +import com.automq.stream.s3.metadata.S3ObjectType; import com.automq.stream.s3.objects.CompactStreamObjectRequest; import com.automq.stream.s3.objects.ObjectManager; import com.automq.stream.s3.operator.S3Operator; import com.automq.stream.s3.operator.Writer; -import com.automq.stream.s3.metadata.S3ObjectMetadata; -import com.automq.stream.s3.metadata.S3ObjectType; - -import java.util.concurrent.ExecutorService; -import java.util.concurrent.ForkJoinPool; -import java.util.concurrent.atomic.AtomicInteger; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.ArrayList; import java.util.Collections; import java.util.LinkedList; @@ -39,8 +32,13 @@ import java.util.Stack; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ForkJoinPool; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Stream objects compaction task. @@ -48,52 +46,54 @@ */ public class StreamObjectsCompactionTask { private static final Logger LOGGER = LoggerFactory.getLogger(StreamObjectsCompactionTask.class); - private final Logger s3ObjectLogger; /** * The max number of compact groups. It comes from the limit of S3 multipart upload. */ private static final int MAX_COMPACT_GROUPS = Writer.MAX_PART_COUNT - 1; - private Queue> compactGroups; + private final Logger s3ObjectLogger; private final long compactedStreamObjectMaxSizeInBytes; private final long eligibleStreamObjectLivingTimeInMs; private final boolean s3ObjectLogEnabled; - private long nextStartSearchingOffset; private final S3Stream stream; private final ObjectManager objectManager; private final S3Operator s3Operator; - private List compactionResults; private final String logIdent; private final ExecutorService executor; + private Queue> compactGroups; + private long nextStartSearchingOffset; + private List compactionResults; /** * Constructor of StreamObjectsCompactionTask. - * @param objectManager object manager. - * @param s3Operator s3 operator. - * @param stream stream. + * + * @param objectManager object manager. + * @param s3Operator s3 operator. + * @param stream stream. * @param compactedStreamObjectMaxSizeInBytes compacted stream object max size in bytes. - * If it is bigger than {@link Writer#MAX_OBJECT_SIZE}, - * it will be set to {@link Writer#MAX_OBJECT_SIZE}. - * @param eligibleStreamObjectLivingTimeInMs eligible stream object living time in ms. + * If it is bigger than {@link Writer#MAX_OBJECT_SIZE}, + * it will be set to {@link Writer#MAX_OBJECT_SIZE}. + * @param eligibleStreamObjectLivingTimeInMs eligible stream object living time in ms. */ public StreamObjectsCompactionTask(ObjectManager objectManager, S3Operator s3Operator, S3Stream stream, - long compactedStreamObjectMaxSizeInBytes, long eligibleStreamObjectLivingTimeInMs) { + long compactedStreamObjectMaxSizeInBytes, long eligibleStreamObjectLivingTimeInMs) { this(objectManager, s3Operator, stream, compactedStreamObjectMaxSizeInBytes, eligibleStreamObjectLivingTimeInMs, false, ForkJoinPool.commonPool()); } /** * Constructor of StreamObjectsCompactionTask. - * @param objectManager object manager. - * @param s3Operator s3 operator. - * @param stream stream. + * + * @param objectManager object manager. + * @param s3Operator s3 operator. + * @param stream stream. * @param compactedStreamObjectMaxSizeInBytes compacted stream object max size in bytes. - * If it is bigger than {@link Writer#MAX_OBJECT_SIZE}, - * it will be set to {@link Writer#MAX_OBJECT_SIZE}. - * @param eligibleStreamObjectLivingTimeInMs eligible stream object living time in ms. - * @param s3ObjectLogEnabled is s3 object log enabled. + * If it is bigger than {@link Writer#MAX_OBJECT_SIZE}, + * it will be set to {@link Writer#MAX_OBJECT_SIZE}. + * @param eligibleStreamObjectLivingTimeInMs eligible stream object living time in ms. + * @param s3ObjectLogEnabled is s3 object log enabled. */ public StreamObjectsCompactionTask(ObjectManager objectManager, S3Operator s3Operator, S3Stream stream, - long compactedStreamObjectMaxSizeInBytes, long eligibleStreamObjectLivingTimeInMs, - boolean s3ObjectLogEnabled, ExecutorService executor) { + long compactedStreamObjectMaxSizeInBytes, long eligibleStreamObjectLivingTimeInMs, + boolean s3ObjectLogEnabled, ExecutorService executor) { this.objectManager = objectManager; this.s3Operator = s3Operator; this.stream = stream; @@ -107,7 +107,8 @@ public StreamObjectsCompactionTask(ObjectManager objectManager, S3Operator s3Ope this.executor = executor; } - private CompletableFuture doCompaction(List streamObjectMetadataList) { + private CompletableFuture doCompaction( + List streamObjectMetadataList) { long startTimestamp = System.currentTimeMillis(); long startOffset = streamObjectMetadataList.get(0).s3StreamObjectMetadata().startOffset(); long endOffset = streamObjectMetadataList.get(streamObjectMetadataList.size() - 1).s3StreamObjectMetadata().endOffset(); @@ -146,9 +147,9 @@ private CompletableFuture doCompaction(List { LOGGER.debug("{} stream objects compaction task with range [{}, {}) is done, objects {} => object {}, size {}", - logIdent, startOffset, endOffset, request.getSourceObjectIds(), request.getObjectId(), request.getObjectSize()); + logIdent, startOffset, endOffset, request.getSourceObjectIds(), request.getObjectId(), request.getObjectSize()); return new CompactionResult(stream.streamId(), startOffset, endOffset, request.getSourceObjectIds(), - request.getObjectId(), request.getObjectSize(), System.currentTimeMillis() - startTimestamp, smallSizeCopyWriteCount.get()); + request.getObjectId(), request.getObjectSize(), System.currentTimeMillis() - startTimestamp, smallSizeCopyWriteCount.get()); }); }, executor); } @@ -199,143 +200,6 @@ public CompactionSummary getCompactionsSummary() { return new CompactionSummary(streamId, startOffset, endOffset, timeCostInMs, totalObjectSize, sourceObjectsCount, targetObjectCount, smallSizeCopyWriteCount); } - static class CompactionResult { - private final long streamId; - private final long startOffset; - private final long endOffset; - private final List sourceObjectIds; - private final long objectId; - private final long objectSize; - private final long timeCostInMs; - private final int smallSizeCopyWriteCount; - - public CompactionResult(long streamId, long startOffset, long endOffset, List sourceObjectIds, long objectId, long objectSize, long timeCostInMs, int smallSizeCopyWriteCount) { - this.streamId = streamId; - this.startOffset = startOffset; - this.endOffset = endOffset; - this.sourceObjectIds = sourceObjectIds; - this.objectId = objectId; - this.objectSize = objectSize; - this.timeCostInMs = timeCostInMs; - this.smallSizeCopyWriteCount = smallSizeCopyWriteCount; - } - - public long getSmallSizeCopyWriteCount() { - return smallSizeCopyWriteCount; - } - - public long getStreamId() { - return streamId; - } - - public long getStartOffset() { - return startOffset; - } - - public long getEndOffset() { - return endOffset; - } - - public List getSourceObjectIds() { - return sourceObjectIds; - } - - public long getObjectId() { - return objectId; - } - - public long getObjectSize() { - return objectSize; - } - - public long getTimeCostInMs() { - return timeCostInMs; - } - - @Override - public String toString() { - return "CompactionResult{" + - "streamId=" + streamId + - ", startOffset=" + startOffset + - ", endOffset=" + endOffset + - ", sourceObjectIds=" + sourceObjectIds + - ", objectId=" + objectId + - ", objectSize=" + objectSize + - ", timeCostInMs=" + timeCostInMs + - ", smallSizeCopyWriteCount=" + smallSizeCopyWriteCount + - '}'; - } - } - - public static class CompactionSummary { - private final long streamId; - private final long startOffset; - private final long endOffset; - private final long timeCostInMs; - private final long totalObjectSize; - private final long sourceObjectsCount; - private final long targetObjectCount; - private final long smallSizeCopyWriteCount; - - public CompactionSummary(long streamId, long startOffset, long endOffset, long timeCostInMs, - long totalObjectSize, long sourceObjectsCount, long targetObjectCount, long smallSizeCopyWriteCount) { - this.streamId = streamId; - this.startOffset = startOffset; - this.endOffset = endOffset; - this.timeCostInMs = timeCostInMs; - this.totalObjectSize = totalObjectSize; - this.sourceObjectsCount = sourceObjectsCount; - this.targetObjectCount = targetObjectCount; - this.smallSizeCopyWriteCount = smallSizeCopyWriteCount; - } - - public long getStreamId() { - return streamId; - } - - public long getStartOffset() { - return startOffset; - } - - public long getEndOffset() { - return endOffset; - } - - public long getTimeCostInMs() { - return timeCostInMs; - } - - public long getTotalObjectSize() { - return totalObjectSize; - } - - public long getSourceObjectsCount() { - return sourceObjectsCount; - } - - public long getTargetObjectCount() { - return targetObjectCount; - } - - public long getSmallSizeCopyWriteCount() { - return smallSizeCopyWriteCount; - } - - @Override - public String toString() { - return "CompactionSummary{" + - "streamId=" + streamId + - ", startOffset=" + startOffset + - ", endOffset=" + endOffset + - ", timeCostInMs=" + timeCostInMs + - ", totalObjectSize=" + totalObjectSize + - ", sourceObjectsCount=" + sourceObjectsCount + - ", targetObjectCount=" + targetObjectCount + - ", smallSizeCopyWriteCount=" + smallSizeCopyWriteCount + - '}'; - } - } - public void prepare() throws ExecutionException, InterruptedException { this.compactionResults = new ArrayList<>(); this.compactGroups = prepareCompactGroups(this.nextStartSearchingOffset); @@ -351,7 +215,8 @@ public void prepare() throws ExecutionException, InterruptedException { * @param startSearchingOffset start searching offset. * @return compact groups. */ - public Queue> prepareCompactGroups(long startSearchingOffset) throws ExecutionException, InterruptedException { + public Queue> prepareCompactGroups( + long startSearchingOffset) throws ExecutionException, InterruptedException { long startOffset = Math.max(startSearchingOffset, stream.startOffset()); List rawFetchedStreamObjects = objectManager .getStreamObjects(stream.streamId(), startOffset, stream.nextOffset(), Integer.MAX_VALUE).get(); @@ -377,7 +242,8 @@ public long getNextStartSearchingOffset() { } // no operation for now. - public void close() {} + public void close() { + } /** * Calculate next start searching offset. It will be used for next compaction task. @@ -499,39 +365,163 @@ private Queue> groupEligibleObjects(Lis return groups; } + private long calculateTimePassedInMs(List streamObjects) { + return System.currentTimeMillis() - streamObjects.stream().mapToLong(S3ObjectMetadata::committedTimestamp).max().orElse(0L); + } + + private long calculateTotalSize(List streamObjects) { + return streamObjects.stream().mapToLong(S3ObjectMetadata::objectSize).sum(); + } + + static class CompactionResult { + private final long streamId; + private final long startOffset; + private final long endOffset; + private final List sourceObjectIds; + private final long objectId; + private final long objectSize; + private final long timeCostInMs; + private final int smallSizeCopyWriteCount; + + public CompactionResult(long streamId, long startOffset, long endOffset, List sourceObjectIds, + long objectId, long objectSize, long timeCostInMs, int smallSizeCopyWriteCount) { + this.streamId = streamId; + this.startOffset = startOffset; + this.endOffset = endOffset; + this.sourceObjectIds = sourceObjectIds; + this.objectId = objectId; + this.objectSize = objectSize; + this.timeCostInMs = timeCostInMs; + this.smallSizeCopyWriteCount = smallSizeCopyWriteCount; + } + + public long getSmallSizeCopyWriteCount() { + return smallSizeCopyWriteCount; + } + + public long getStreamId() { + return streamId; + } + + public long getStartOffset() { + return startOffset; + } + + public long getEndOffset() { + return endOffset; + } + + public List getSourceObjectIds() { + return sourceObjectIds; + } + + public long getObjectId() { + return objectId; + } + + public long getObjectSize() { + return objectSize; + } + + public long getTimeCostInMs() { + return timeCostInMs; + } + + @Override + public String toString() { + return "CompactionResult{" + + "streamId=" + streamId + + ", startOffset=" + startOffset + + ", endOffset=" + endOffset + + ", sourceObjectIds=" + sourceObjectIds + + ", objectId=" + objectId + + ", objectSize=" + objectSize + + ", timeCostInMs=" + timeCostInMs + + ", smallSizeCopyWriteCount=" + smallSizeCopyWriteCount + + '}'; + } + } + + public static class CompactionSummary { + private final long streamId; + private final long startOffset; + private final long endOffset; + private final long timeCostInMs; + private final long totalObjectSize; + private final long sourceObjectsCount; + private final long targetObjectCount; + private final long smallSizeCopyWriteCount; + + public CompactionSummary(long streamId, long startOffset, long endOffset, long timeCostInMs, + long totalObjectSize, long sourceObjectsCount, long targetObjectCount, long smallSizeCopyWriteCount) { + this.streamId = streamId; + this.startOffset = startOffset; + this.endOffset = endOffset; + this.timeCostInMs = timeCostInMs; + this.totalObjectSize = totalObjectSize; + this.sourceObjectsCount = sourceObjectsCount; + this.targetObjectCount = targetObjectCount; + this.smallSizeCopyWriteCount = smallSizeCopyWriteCount; + } + + public long getStreamId() { + return streamId; + } + + public long getStartOffset() { + return startOffset; + } + + public long getEndOffset() { + return endOffset; + } + + public long getTimeCostInMs() { + return timeCostInMs; + } + + public long getTotalObjectSize() { + return totalObjectSize; + } + + public long getSourceObjectsCount() { + return sourceObjectsCount; + } + + public long getTargetObjectCount() { + return targetObjectCount; + } + + public long getSmallSizeCopyWriteCount() { + return smallSizeCopyWriteCount; + } + + @Override + public String toString() { + return "CompactionSummary{" + + "streamId=" + streamId + + ", startOffset=" + startOffset + + ", endOffset=" + endOffset + + ", timeCostInMs=" + timeCostInMs + + ", totalObjectSize=" + totalObjectSize + + ", sourceObjectsCount=" + sourceObjectsCount + + ", targetObjectCount=" + targetObjectCount + + ", smallSizeCopyWriteCount=" + smallSizeCopyWriteCount + + '}'; + } + } + /** * Wrapper for {@link S3ObjectMetadata} with split copy count. */ public static class S3StreamObjectMetadataSplitWrapper { private final S3ObjectMetadata s3StreamObjectMetadata; private final int splitCopyCount; + public S3StreamObjectMetadataSplitWrapper(S3ObjectMetadata s3StreamObjectMetadata, int splitCopyCount) { this.s3StreamObjectMetadata = s3StreamObjectMetadata; this.splitCopyCount = splitCopyCount; } - public S3ObjectMetadata s3StreamObjectMetadata() { - return s3StreamObjectMetadata; - } - public int splitCopyCount() { - return splitCopyCount; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (!(o instanceof S3StreamObjectMetadataSplitWrapper)) { - return false; - } - S3StreamObjectMetadataSplitWrapper that = (S3StreamObjectMetadataSplitWrapper) o; - return splitCopyCount == that.splitCopyCount && s3StreamObjectMetadata.equals(that.s3StreamObjectMetadata); - } - - @Override - public int hashCode() { - return Objects.hash(s3StreamObjectMetadata, splitCopyCount); - } public static S3StreamObjectMetadataSplitWrapper parse(S3ObjectMetadata s3StreamObjectMetadata) { if (s3StreamObjectMetadata.getOffsetRanges() == null || s3StreamObjectMetadata.getOffsetRanges().size() != 1) { @@ -569,14 +559,31 @@ public static long calculateSplitCopyCount(List streamObjects) .sum(); } - } + public S3ObjectMetadata s3StreamObjectMetadata() { + return s3StreamObjectMetadata; + } - private long calculateTimePassedInMs(List streamObjects) { - return System.currentTimeMillis() - streamObjects.stream().mapToLong(S3ObjectMetadata::committedTimestamp).max().orElse(0L); - } + public int splitCopyCount() { + return splitCopyCount; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof S3StreamObjectMetadataSplitWrapper)) { + return false; + } + S3StreamObjectMetadataSplitWrapper that = (S3StreamObjectMetadataSplitWrapper) o; + return splitCopyCount == that.splitCopyCount && s3StreamObjectMetadata.equals(that.s3StreamObjectMetadata); + } + + @Override + public int hashCode() { + return Objects.hash(s3StreamObjectMetadata, splitCopyCount); + } - private long calculateTotalSize(List streamObjects) { - return streamObjects.stream().mapToLong(S3ObjectMetadata::objectSize).sum(); } public static class HaltException extends RuntimeException { @@ -598,6 +605,7 @@ public Builder(ObjectManager objectManager, S3Operator s3Operator) { this.objectManager = objectManager; this.s3Operator = s3Operator; } + public Builder withStream(S3Stream stream) { this.stream = stream; return this; @@ -605,19 +613,22 @@ public Builder withStream(S3Stream stream) { /** * Set compacted stream object max size. + * * @param compactedStreamObjectMaxSizeInBytes compacted stream object max size in bytes. - * If it is bigger than {@link Writer#MAX_OBJECT_SIZE}, - * it will be set to {@link Writer#MAX_OBJECT_SIZE}. + * If it is bigger than {@link Writer#MAX_OBJECT_SIZE}, + * it will be set to {@link Writer#MAX_OBJECT_SIZE}. * @return builder. */ public Builder compactedStreamObjectMaxSizeInBytes(long compactedStreamObjectMaxSizeInBytes) { this.compactedStreamObjectMaxSizeInBytes = compactedStreamObjectMaxSizeInBytes; return this; } + public Builder eligibleStreamObjectLivingTimeInMs(long eligibleStreamObjectLivingTimeInMs) { this.eligibleStreamObjectLivingTimeInMs = eligibleStreamObjectLivingTimeInMs; return this; } + public Builder s3ObjectLogEnabled(boolean s3ObjectLogEnabled) { this.s3ObjectLogEnabled = s3ObjectLogEnabled; return this; diff --git a/s3stream/src/main/java/com/automq/stream/s3/StreamRecordBatchCodec.java b/s3stream/src/main/java/com/automq/stream/s3/StreamRecordBatchCodec.java index b902a416f..e030fad74 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/StreamRecordBatchCodec.java +++ b/s3stream/src/main/java/com/automq/stream/s3/StreamRecordBatchCodec.java @@ -23,7 +23,7 @@ public class StreamRecordBatchCodec { public static final byte MAGIC_V0 = 0x22; public static final int HEADER_SIZE = - 1 // magic + 1 // magic + 8 // streamId + 8 // epoch + 8 // baseOffset diff --git a/s3stream/src/main/java/com/automq/stream/s3/WalWriteRequest.java b/s3stream/src/main/java/com/automq/stream/s3/WalWriteRequest.java index 5cc947d9b..bc0d1c722 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/WalWriteRequest.java +++ b/s3stream/src/main/java/com/automq/stream/s3/WalWriteRequest.java @@ -17,17 +17,15 @@ package com.automq.stream.s3; - import com.automq.stream.s3.context.AppendContext; import com.automq.stream.s3.model.StreamRecordBatch; - import java.util.concurrent.CompletableFuture; public class WalWriteRequest implements Comparable { final StreamRecordBatch record; final AppendContext context; - long offset; final CompletableFuture cf; + long offset; boolean persisted; public WalWriteRequest(StreamRecordBatch record, long offset, CompletableFuture cf) { diff --git a/s3stream/src/main/java/com/automq/stream/s3/cache/BlockCache.java b/s3stream/src/main/java/com/automq/stream/s3/cache/BlockCache.java index a740bf4e5..d75ca18e6 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/cache/BlockCache.java +++ b/s3stream/src/main/java/com/automq/stream/s3/cache/BlockCache.java @@ -17,18 +17,14 @@ package com.automq.stream.s3.cache; - import com.automq.stream.s3.DirectByteBufAlloc; +import com.automq.stream.s3.cache.DefaultS3BlockCache.ReadAheadRecord; import com.automq.stream.s3.metrics.S3StreamMetricsManager; import com.automq.stream.s3.model.StreamRecordBatch; -import com.automq.stream.s3.cache.DefaultS3BlockCache.ReadAheadRecord; import com.automq.stream.s3.trace.context.TraceContext; import com.automq.stream.utils.biniarysearch.StreamRecordBatchList; import io.opentelemetry.instrumentation.annotations.SpanAttribute; import io.opentelemetry.instrumentation.annotations.WithSpan; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -37,16 +33,17 @@ import java.util.Map; import java.util.NavigableMap; import java.util.SortedMap; - import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantReadWriteLock; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class BlockCache implements DirectByteBufAlloc.OOMHandler { - private static final Logger LOGGER = LoggerFactory.getLogger(BlockCache.class); public static final Integer ASYNC_READ_AHEAD_NOOP_OFFSET = -1; static final int BLOCK_SIZE = 1024 * 1024; - private final long maxSize; + private static final Logger LOGGER = LoggerFactory.getLogger(BlockCache.class); final Map stream2cache = new HashMap<>(); + private final long maxSize; private final LRUCache inactive = new LRUCache<>(); private final LRUCache active = new LRUCache<>(); private final AtomicLong size = new AtomicLong(); @@ -134,7 +131,7 @@ void put0(long streamId, long raAsyncOffset, long raEndOffset, List(); batchList.add(record); @@ -144,7 +141,7 @@ void put0(long streamId, long raAsyncOffset, long raEndOffset, List= endOffset || maxBytes <= 0) { return GetCacheResult.empty(); @@ -282,7 +279,7 @@ public GetCacheResult get0(long streamId, long startOffset, long endOffset, int } private int readFromCacheBlock(LinkedList records, CacheBlock cacheBlock, - long nextStartOffset, long endOffset, int nextMaxBytes) { + long nextStartOffset, long endOffset, int nextMaxBytes) { boolean matched = false; StreamRecordBatchList streamRecordBatchList = new StreamRecordBatchList(cacheBlock.records); int startIndex = streamRecordBatchList.search(nextStartOffset); @@ -363,7 +360,7 @@ private void logCacheStatus() { for (Map.Entry entry : streamCache.blocks().entrySet()) { CacheBlockKey key = new CacheBlockKey(streamId, entry.getValue().firstOffset); LOGGER.debug("[S3BlockCache] stream cache block, stream={}, {}-{}, inactive={}, active={}, total bytes: {} ", - streamId, entry.getValue().firstOffset, entry.getValue().lastOffset, inactive.containsKey(key), active.containsKey(key), entry.getValue().size); + streamId, entry.getValue().firstOffset, entry.getValue().lastOffset, inactive.containsKey(key), active.containsKey(key), entry.getValue().size); } } } finally { @@ -390,6 +387,10 @@ public int handle(int memoryRequired) { } } + public interface CacheEvictListener { + void onCacheEvict(long streamId, long startOffset, long endOffset, int size); + } + record CacheBlockKey(long streamId, long startOffset) { } @@ -444,8 +445,4 @@ public List getReadAheadRecords() { return readAheadRecords; } } - - public interface CacheEvictListener { - void onCacheEvict(long streamId, long startOffset, long endOffset, int size); - } } diff --git a/s3stream/src/main/java/com/automq/stream/s3/cache/DataBlockReadAccumulator.java b/s3stream/src/main/java/com/automq/stream/s3/cache/DataBlockReadAccumulator.java index 3197ea4c3..1a51712a1 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/cache/DataBlockReadAccumulator.java +++ b/s3stream/src/main/java/com/automq/stream/s3/cache/DataBlockReadAccumulator.java @@ -19,16 +19,15 @@ import com.automq.stream.s3.ObjectReader; import com.automq.stream.s3.StreamDataBlock; -import org.apache.commons.lang3.tuple.Pair; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.function.BiConsumer; +import org.apache.commons.lang3.tuple.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Accumulate inflight data block read requests to one real read request. diff --git a/s3stream/src/main/java/com/automq/stream/s3/cache/DataBlockRecords.java b/s3stream/src/main/java/com/automq/stream/s3/cache/DataBlockRecords.java index 031de3be3..6622c070e 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/cache/DataBlockRecords.java +++ b/s3stream/src/main/java/com/automq/stream/s3/cache/DataBlockRecords.java @@ -20,15 +20,14 @@ import com.automq.stream.s3.ObjectReader; import com.automq.stream.s3.model.StreamRecordBatch; import com.automq.stream.utils.CloseableIterator; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.ArrayList; import java.util.Collections; import java.util.LinkedList; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiConsumer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class DataBlockRecords { private static final Logger LOGGER = LoggerFactory.getLogger(DataBlockRecords.class); diff --git a/s3stream/src/main/java/com/automq/stream/s3/cache/DefaultS3BlockCache.java b/s3stream/src/main/java/com/automq/stream/s3/cache/DefaultS3BlockCache.java index dff3e1466..caaee8aeb 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/cache/DefaultS3BlockCache.java +++ b/s3stream/src/main/java/com/automq/stream/s3/cache/DefaultS3BlockCache.java @@ -31,9 +31,6 @@ import io.opentelemetry.api.trace.Span; import io.opentelemetry.instrumentation.annotations.SpanAttribute; import io.opentelemetry.instrumentation.annotations.WithSpan; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -43,6 +40,8 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static com.automq.stream.s3.metadata.ObjectUtils.NOOP_OFFSET; @@ -62,10 +61,10 @@ public DefaultS3BlockCache(Config config, ObjectManager objectManager, S3Operato this.cache = new BlockCache(config.blockCacheSize()); this.readAheadManager = new ReadAheadManager(blockSize, this.cache); this.mainExecutor = Threads.newFixedThreadPoolWithMonitor( - 2, - "s3-block-cache-main", - false, - LOGGER); + 2, + "s3-block-cache-main", + false, + LOGGER); this.inflightReadThrottle = new InflightReadThrottle(); this.streamReader = new StreamReader(s3Operator, objectManager, cache, inflightReadAheadTasks, inflightReadThrottle); } @@ -80,10 +79,10 @@ public void shutdown() { @Override @WithSpan public CompletableFuture read(TraceContext traceContext, - @SpanAttribute long streamId, - @SpanAttribute long startOffset, - @SpanAttribute long endOffset, - @SpanAttribute int maxBytes) { + @SpanAttribute long streamId, + @SpanAttribute long startOffset, + @SpanAttribute long endOffset, + @SpanAttribute int maxBytes) { if (LOGGER.isDebugEnabled()) { LOGGER.debug("[S3BlockCache] read data, stream={}, {}-{}, total bytes: {}", streamId, startOffset, endOffset, maxBytes); } @@ -93,7 +92,7 @@ public CompletableFuture read(TraceContext traceContext, CompletableFuture readCf = new CompletableFuture<>(); ReadAheadAgent agent = this.readAheadManager.getOrCreateReadAheadAgent(streamId, startOffset); UUID uuid = UUID.randomUUID(); - ReadTaskKey key = new ReadTaskKey(streamId, startOffset, endOffset, maxBytes , uuid); + ReadTaskKey key = new ReadTaskKey(streamId, startOffset, endOffset, maxBytes, uuid); ReadTaskContext context = new ReadTaskContext(agent, ReadBlockCacheStatus.INIT); this.inflightReadStatusMap.put(key, context); // submit read task to mainExecutor to avoid read slower the caller thread. @@ -108,7 +107,7 @@ public CompletableFuture read(TraceContext traceContext, } int totalReturnedSize = ret.getRecords().stream().mapToInt(StreamRecordBatch::size).sum(); this.readAheadManager.updateReadResult(streamId, startOffset, - ret.getRecords().get(ret.getRecords().size() - 1).getLastOffset(), totalReturnedSize); + ret.getRecords().get(ret.getRecords().size() - 1).getLastOffset(), totalReturnedSize); long timeElapsed = timerUtil.elapsedAs(TimeUnit.NANOSECONDS); boolean isCacheHit = ret.getCacheAccessType() == CacheAccessType.BLOCK_CACHE_HIT; @@ -116,7 +115,7 @@ public CompletableFuture read(TraceContext traceContext, S3StreamMetricsManager.recordReadCacheLatency(MetricsLevel.INFO, timeElapsed, S3Operation.READ_STORAGE_BLOCK_CACHE, isCacheHit); if (LOGGER.isDebugEnabled()) { LOGGER.debug("[S3BlockCache] read data complete, cache hit: {}, stream={}, {}-{}, total bytes: {}", - ret.getCacheAccessType() == CacheAccessType.BLOCK_CACHE_HIT, streamId, startOffset, endOffset, totalReturnedSize); + ret.getCacheAccessType() == CacheAccessType.BLOCK_CACHE_HIT, streamId, startOffset, endOffset, totalReturnedSize); } this.inflightReadThrottle.release(uuid); this.inflightReadStatusMap.remove(key); @@ -133,11 +132,11 @@ public CompletableFuture read(TraceContext traceContext, @WithSpan public CompletableFuture read0(TraceContext traceContext, - @SpanAttribute long streamId, - @SpanAttribute long startOffset, - @SpanAttribute long endOffset, - @SpanAttribute int maxBytes, - UUID uuid, ReadTaskContext context) { + @SpanAttribute long streamId, + @SpanAttribute long startOffset, + @SpanAttribute long endOffset, + @SpanAttribute int maxBytes, + UUID uuid, ReadTaskContext context) { ReadAheadAgent agent = context.agent; if (LOGGER.isDebugEnabled()) { @@ -156,7 +155,7 @@ public CompletableFuture read0(TraceContext traceContext, CompletableFuture readCf = new CompletableFuture<>(); context.setStatus(ReadBlockCacheStatus.WAIT_INFLIGHT_RA); inflightReadAheadTaskContext.cf.whenComplete((nil, ex) -> FutureUtil.exec(() -> FutureUtil.propagate( - read0(traceContext, streamId, startOffset, endOffset, maxBytes, uuid, context), readCf), readCf, LOGGER, "read0")); + read0(traceContext, streamId, startOffset, endOffset, maxBytes, uuid, context), readCf), readCf, LOGGER, "read0")); return readCf; } @@ -193,21 +192,21 @@ public CompletableFuture read0(TraceContext traceContext, LOGGER.debug("[S3BlockCache] read data cache miss, stream={}, {}-{}, total bytes: {} ", streamId, startOffset, endOffset, maxBytes); } return streamReader.syncReadAhead(traceContext, streamId, startOffset, endOffset, maxBytes, agent, uuid) - .thenCompose(rst -> { - if (!rst.isEmpty()) { - int remainBytes = maxBytes - rst.stream().mapToInt(StreamRecordBatch::size).sum(); - long lastOffset = rst.get(rst.size() - 1).getLastOffset(); - if (remainBytes > 0 && lastOffset < endOffset) { - // retry read - return read0(traceContext, streamId, lastOffset, endOffset, remainBytes, uuid, context).thenApply(rst2 -> { - List records = new ArrayList<>(rst); - records.addAll(rst2.getRecords()); - return new ReadDataBlock(records, CacheAccessType.BLOCK_CACHE_MISS); - }); - } + .thenCompose(rst -> { + if (!rst.isEmpty()) { + int remainBytes = maxBytes - rst.stream().mapToInt(StreamRecordBatch::size).sum(); + long lastOffset = rst.get(rst.size() - 1).getLastOffset(); + if (remainBytes > 0 && lastOffset < endOffset) { + // retry read + return read0(traceContext, streamId, lastOffset, endOffset, remainBytes, uuid, context).thenApply(rst2 -> { + List records = new ArrayList<>(rst); + records.addAll(rst2.getRecords()); + return new ReadDataBlock(records, CacheAccessType.BLOCK_CACHE_MISS); + }); } - return CompletableFuture.completedFuture(new ReadDataBlock(rst, CacheAccessType.BLOCK_CACHE_MISS)); - }); + } + return CompletableFuture.completedFuture(new ReadDataBlock(rst, CacheAccessType.BLOCK_CACHE_MISS)); + }); } private void asyncReadAhead(long streamId, ReadAheadAgent agent, List readAheadRecords) { @@ -221,7 +220,7 @@ private void asyncReadAhead(long streamId, ReadAheadAgent agent, List acquire(TraceContext context, UUID uuid, int read try { if (readSize > maxInflightReadBytes) { return CompletableFuture.failedFuture(new IllegalArgumentException(String.format( - "read size %d exceeds max inflight read size %d", readSize, maxInflightReadBytes))); + "read size %d exceeds max inflight read size %d", readSize, maxInflightReadBytes))); } if (readSize <= 0) { return CompletableFuture.completedFuture(null); diff --git a/s3stream/src/main/java/com/automq/stream/s3/cache/LogCache.java b/s3stream/src/main/java/com/automq/stream/s3/cache/LogCache.java index 1854391f3..59e5065e3 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/cache/LogCache.java +++ b/s3stream/src/main/java/com/automq/stream/s3/cache/LogCache.java @@ -26,9 +26,6 @@ import com.automq.stream.utils.biniarysearch.StreamRecordBatchList; import io.opentelemetry.instrumentation.annotations.SpanAttribute; import io.opentelemetry.instrumentation.annotations.WithSpan; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -43,31 +40,33 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Consumer; import java.util.stream.Collectors; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static com.automq.stream.s3.cache.LogCache.StreamRange.NOOP_OFFSET; import static com.automq.stream.utils.FutureUtil.suppress; public class LogCache { - private static final Logger LOGGER = LoggerFactory.getLogger(LogCache.class); public static final long MATCH_ALL_STREAMS = -1L; + private static final Logger LOGGER = LoggerFactory.getLogger(LogCache.class); private static final int DEFAULT_MAX_BLOCK_STREAM_COUNT = 10000; private static final Consumer DEFAULT_BLOCK_FREE_LISTENER = block -> { }; + final List blocks = new ArrayList<>(); private final long capacity; private final long cacheBlockMaxSize; private final int maxCacheBlockStreamCount; - final List blocks = new ArrayList<>(); - private LogCacheBlock activeBlock; - private long confirmOffset; private final AtomicLong size = new AtomicLong(); private final Consumer blockFreeListener; - // read write lock which guards the LogCache.blocks private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); private final ReentrantReadWriteLock.ReadLock readLock = lock.readLock(); private final ReentrantReadWriteLock.WriteLock writeLock = lock.writeLock(); + private LogCacheBlock activeBlock; + private long confirmOffset; - public LogCache(long capacity, long cacheBlockMaxSize, int maxCacheBlockStreamCount, Consumer blockFreeListener) { + public LogCache(long capacity, long cacheBlockMaxSize, int maxCacheBlockStreamCount, + Consumer blockFreeListener) { this.capacity = capacity; this.cacheBlockMaxSize = cacheBlockMaxSize; this.maxCacheBlockStreamCount = maxCacheBlockStreamCount; @@ -134,10 +133,10 @@ public List get(long streamId, long startOffset, long endOffs */ @WithSpan public List get(TraceContext context, - @SpanAttribute long streamId, - @SpanAttribute long startOffset, - @SpanAttribute long endOffset, - @SpanAttribute int maxBytes) { + @SpanAttribute long streamId, + @SpanAttribute long startOffset, + @SpanAttribute long endOffset, + @SpanAttribute int maxBytes) { context.currentContext(); TimerUtil timerUtil = new TimerUtil(); List records; @@ -191,8 +190,8 @@ public List get0(long streamId, long startOffset, long endOff } } if (lastBlockStreamStartOffset == NOOP_OFFSET /* Mismatch */ - || lastBlockStreamStartOffset >= endOffset /* non-right intersect */ - || lastBlockStreamStartOffset <= startOffset /* left intersect */) { + || lastBlockStreamStartOffset >= endOffset /* non-right intersect */ + || lastBlockStreamStartOffset <= startOffset /* left intersect */) { return Collections.emptyList(); } return get0(streamId, lastBlockStreamStartOffset, endOffset, maxBytes); @@ -212,7 +211,6 @@ public LogCacheBlock archiveCurrentBlock() { } } - public Optional archiveCurrentBlockIfContains(long streamId) { writeLock.lock(); try { @@ -305,14 +303,14 @@ public long size() { public static class LogCacheBlock { private static final AtomicLong BLOCK_ID_ALLOC = new AtomicLong(); + final Map map = new ConcurrentHashMap<>(); private final long blockId; private final long maxSize; private final int maxStreamCount; private final long createdTimestamp = System.currentTimeMillis(); - final Map map = new ConcurrentHashMap<>(); private final AtomicLong size = new AtomicLong(); - private long confirmOffset; volatile boolean free; + private long confirmOffset; public LogCacheBlock(long maxSize, int maxStreamCount) { this.blockId = BLOCK_ID_ALLOC.getAndIncrement(); @@ -324,7 +322,6 @@ public LogCacheBlock(long maxSize) { this(maxSize, DEFAULT_MAX_BLOCK_STREAM_COUNT); } - public long blockId() { return blockId; } @@ -360,8 +357,8 @@ StreamRange getStreamRange(long streamId) { public Map> records() { return map.entrySet().stream() - .map(e -> Map.entry(e.getKey(), e.getValue().records)) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + .map(e -> Map.entry(e.getKey(), e.getValue().records)) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); } public long confirmOffset() { @@ -415,7 +412,7 @@ static class StreamCache { synchronized void add(StreamRecordBatch recordBatch) { if (recordBatch.getBaseOffset() != endOffset && endOffset != NOOP_OFFSET) { RuntimeException ex = new IllegalArgumentException(String.format("streamId=%s record batch base offset mismatch, expect %s, actual %s", - recordBatch.getStreamId(), endOffset, recordBatch.getBaseOffset())); + recordBatch.getStreamId(), endOffset, recordBatch.getBaseOffset())); LOGGER.error("[FATAL]", ex); } records.add(recordBatch); diff --git a/s3stream/src/main/java/com/automq/stream/s3/cache/ObjectReaderLRUCache.java b/s3stream/src/main/java/com/automq/stream/s3/cache/ObjectReaderLRUCache.java index 622824430..e0d68461e 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/cache/ObjectReaderLRUCache.java +++ b/s3stream/src/main/java/com/automq/stream/s3/cache/ObjectReaderLRUCache.java @@ -18,7 +18,6 @@ package com.automq.stream.s3.cache; import com.automq.stream.s3.ObjectReader; - import java.util.Optional; public class ObjectReaderLRUCache extends LRUCache { @@ -43,12 +42,12 @@ public synchronized void put(Long key, ObjectReader value) { private int objectSize() { return cacheEntrySet.stream().filter(entry -> entry.getValue().basicObjectInfo().isDone()) - .mapToInt(entry -> { - try { - return entry.getValue().basicObjectInfo().get().size(); - } catch (Exception e) { - return 0; - } - }).sum(); + .mapToInt(entry -> { + try { + return entry.getValue().basicObjectInfo().get().size(); + } catch (Exception e) { + return 0; + } + }).sum(); } } diff --git a/s3stream/src/main/java/com/automq/stream/s3/cache/ReadAheadAgent.java b/s3stream/src/main/java/com/automq/stream/s3/cache/ReadAheadAgent.java index 88be764e7..9923fa3e4 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/cache/ReadAheadAgent.java +++ b/s3stream/src/main/java/com/automq/stream/s3/cache/ReadAheadAgent.java @@ -22,19 +22,18 @@ import com.automq.stream.s3.metrics.TimerUtil; import com.automq.stream.utils.LogContext; import com.google.common.base.Objects; -import org.apache.commons.lang3.tuple.Pair; -import org.slf4j.Logger; - import java.util.ArrayList; import java.util.List; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; +import org.apache.commons.lang3.tuple.Pair; +import org.slf4j.Logger; public class ReadAheadAgent { - private final Logger logger; private static final Integer MAX_READ_AHEAD_SIZE = 40 * 1024 * 1024; // 40MB private static final Integer S3_OPERATION_DELAY_MS = 400; // 400ms + private final Logger logger; private final Lock lock = new ReentrantLock(); private final TimerUtil timer; private final long streamId; @@ -191,9 +190,9 @@ public void evict(long startOffset, long endOffset) { try { lock.lock(); if (startOffset >= endOffset - || lastReadOffset >= readAheadEndOffset - || endOffset <= lastReadOffset - || startOffset >= readAheadEndOffset) { + || lastReadOffset >= readAheadEndOffset + || endOffset <= lastReadOffset + || startOffset >= readAheadEndOffset) { return; } @@ -209,8 +208,10 @@ public void evict(long startOffset, long endOffset) { @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; ReadAheadAgent agent = (ReadAheadAgent) o; return streamId == agent.streamId && lastReadOffset == agent.lastReadOffset; } @@ -223,12 +224,12 @@ public int hashCode() { @Override public String toString() { return "ReadAheadAgent{" + - "stream=" + streamId + - ", bytesPerSecond=" + bytePerSecond + - ", lastReadOffset=" + lastReadOffset + - ", lastReadSize=" + lastReadSize + - ", readAheadEndOffset=" + readAheadEndOffset + - ", evictedOffsetRanges=" + evictedOffsetRanges + - '}'; + "stream=" + streamId + + ", bytesPerSecond=" + bytePerSecond + + ", lastReadOffset=" + lastReadOffset + + ", lastReadSize=" + lastReadSize + + ", readAheadEndOffset=" + readAheadEndOffset + + ", evictedOffsetRanges=" + evictedOffsetRanges + + '}'; } } diff --git a/s3stream/src/main/java/com/automq/stream/s3/cache/ReadAheadManager.java b/s3stream/src/main/java/com/automq/stream/s3/cache/ReadAheadManager.java index 4dda7f2b8..f0ec66ed9 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/cache/ReadAheadManager.java +++ b/s3stream/src/main/java/com/automq/stream/s3/cache/ReadAheadManager.java @@ -18,14 +18,13 @@ package com.automq.stream.s3.cache; import com.automq.stream.utils.LogContext; -import org.slf4j.Logger; - import java.util.Map; import java.util.NavigableMap; import java.util.Objects; import java.util.Set; import java.util.TreeMap; import java.util.concurrent.ConcurrentHashMap; +import org.slf4j.Logger; public class ReadAheadManager implements BlockCache.CacheEvictListener { private static final Logger LOGGER = new LogContext("[S3BlockCache] ").logger(ReadAheadManager.class); diff --git a/s3stream/src/main/java/com/automq/stream/s3/cache/ReadDataBlock.java b/s3stream/src/main/java/com/automq/stream/s3/cache/ReadDataBlock.java index 0d5aa7805..4be4b32f5 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/cache/ReadDataBlock.java +++ b/s3stream/src/main/java/com/automq/stream/s3/cache/ReadDataBlock.java @@ -18,7 +18,6 @@ package com.automq.stream.s3.cache; import com.automq.stream.s3.model.StreamRecordBatch; - import java.util.List; import java.util.OptionalLong; diff --git a/s3stream/src/main/java/com/automq/stream/s3/cache/S3BlockCache.java b/s3stream/src/main/java/com/automq/stream/s3/cache/S3BlockCache.java index 161780a29..2db87a9c8 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/cache/S3BlockCache.java +++ b/s3stream/src/main/java/com/automq/stream/s3/cache/S3BlockCache.java @@ -18,7 +18,6 @@ package com.automq.stream.s3.cache; import com.automq.stream.s3.trace.context.TraceContext; - import java.util.concurrent.CompletableFuture; /** @@ -28,7 +27,8 @@ */ public interface S3BlockCache { - CompletableFuture read(TraceContext context, long streamId, long startOffset, long endOffset, int maxBytes); + CompletableFuture read(TraceContext context, long streamId, long startOffset, long endOffset, + int maxBytes); default CompletableFuture read(long streamId, long startOffset, long endOffset, int maxBytes) { return read(TraceContext.DEFAULT, streamId, startOffset, endOffset, maxBytes); diff --git a/s3stream/src/main/java/com/automq/stream/s3/cache/StreamReader.java b/s3stream/src/main/java/com/automq/stream/s3/cache/StreamReader.java index 4ec9b4105..6855d4fc1 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/cache/StreamReader.java +++ b/s3stream/src/main/java/com/automq/stream/s3/cache/StreamReader.java @@ -33,11 +33,6 @@ import io.opentelemetry.api.common.Attributes; import io.opentelemetry.instrumentation.annotations.SpanAttribute; import io.opentelemetry.instrumentation.annotations.WithSpan; -import org.apache.commons.lang3.tuple.ImmutablePair; -import org.apache.commons.lang3.tuple.Pair; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -52,10 +47,14 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class StreamReader { - private static final Logger LOGGER = LoggerFactory.getLogger(StreamReader.class); public static final Integer MAX_OBJECT_READER_SIZE = 100 * 1024 * 1024; // 100MB; + private static final Logger LOGGER = LoggerFactory.getLogger(StreamReader.class); private static final Integer READ_OBJECT_INDEX_STEP = 2; private final S3Operator s3Operator; private final ObjectManager objectManager; @@ -65,24 +64,24 @@ public class StreamReader { private final Map inflightReadAheadTaskMap; private final InflightReadThrottle inflightReadThrottle; private final ExecutorService streamReaderExecutor = Threads.newFixedThreadPoolWithMonitor( - 2, - "s3-stream-reader", - false, - LOGGER); + 2, + "s3-stream-reader", + false, + LOGGER); private final ExecutorService backgroundExecutor = Threads.newFixedThreadPoolWithMonitor( - 2, - "s3-stream-reader-background", - true, - LOGGER); + 2, + "s3-stream-reader-background", + true, + LOGGER); private final ExecutorService errorHandlerExecutor = Threads.newFixedThreadPoolWithMonitor( - 1, - "s3-stream-reader-error-handler", - true, - LOGGER); + 1, + "s3-stream-reader-error-handler", + true, + LOGGER); public StreamReader(S3Operator operator, ObjectManager objectManager, BlockCache blockCache, - Map inflightReadAheadTaskMap, - InflightReadThrottle inflightReadThrottle) { + Map inflightReadAheadTaskMap, + InflightReadThrottle inflightReadThrottle) { this.s3Operator = operator; this.objectManager = objectManager; this.objectReaders = new ObjectReaderLRUCache(MAX_OBJECT_READER_SIZE); @@ -93,9 +92,11 @@ public StreamReader(S3Operator operator, ObjectManager objectManager, BlockCache } // for test - public StreamReader(S3Operator operator, ObjectManager objectManager, BlockCache blockCache, ObjectReaderLRUCache objectReaders, - DataBlockReadAccumulator dataBlockReadAccumulator, Map inflightReadAheadTaskMap, - InflightReadThrottle inflightReadThrottle) { + public StreamReader(S3Operator operator, ObjectManager objectManager, BlockCache blockCache, + ObjectReaderLRUCache objectReaders, + DataBlockReadAccumulator dataBlockReadAccumulator, + Map inflightReadAheadTaskMap, + InflightReadThrottle inflightReadThrottle) { this.s3Operator = operator; this.objectManager = objectManager; this.objectReaders = objectReaders; @@ -113,10 +114,10 @@ public void shutdown() { @WithSpan public CompletableFuture> syncReadAhead(TraceContext traceContext, - @SpanAttribute long streamId, - @SpanAttribute long startOffset, - @SpanAttribute long endOffset, - @SpanAttribute int maxBytes, ReadAheadAgent agent, UUID uuid) { + @SpanAttribute long streamId, + @SpanAttribute long startOffset, + @SpanAttribute long endOffset, + @SpanAttribute int maxBytes, ReadAheadAgent agent, UUID uuid) { if (LOGGER.isDebugEnabled()) { LOGGER.debug("[S3BlockCache] sync read ahead, stream={}, {}-{}, maxBytes={}", streamId, startOffset, endOffset, maxBytes); } @@ -125,32 +126,33 @@ public CompletableFuture> syncReadAhead(TraceContext tra DefaultS3BlockCache.ReadAheadTaskKey readAheadTaskKey = new DefaultS3BlockCache.ReadAheadTaskKey(streamId, startOffset); // put a placeholder task at start offset to prevent next cache miss request spawn duplicated read ahead task DefaultS3BlockCache.ReadAheadTaskContext readAheadTaskContext = new DefaultS3BlockCache.ReadAheadTaskContext(new CompletableFuture<>(), - DefaultS3BlockCache.ReadBlockCacheStatus.WAIT_DATA_INDEX); + DefaultS3BlockCache.ReadBlockCacheStatus.WAIT_DATA_INDEX); if (inflightReadAheadTaskMap.putIfAbsent(readAheadTaskKey, readAheadTaskContext) == null) { context.taskKeySet.add(readAheadTaskKey); } return getDataBlockIndices(traceContext, streamId, endOffset, context) - .thenComposeAsync(v -> - handleSyncReadAhead(traceContext, streamId, startOffset, endOffset, maxBytes, agent, uuid, timer, context), streamReaderExecutor) - .whenComplete((nil, ex) -> { - for (DefaultS3BlockCache.ReadAheadTaskKey key : context.taskKeySet) { - completeInflightTask0(key, ex); - } - context.taskKeySet.clear(); - S3StreamMetricsManager.recordReadAheadLatency(MetricsLevel.INFO, timer.elapsedAs(TimeUnit.NANOSECONDS), S3Operation.BLOCK_CACHE_READ_AHEAD, true); - }); + .thenComposeAsync(v -> + handleSyncReadAhead(traceContext, streamId, startOffset, endOffset, maxBytes, agent, uuid, timer, context), streamReaderExecutor) + .whenComplete((nil, ex) -> { + for (DefaultS3BlockCache.ReadAheadTaskKey key : context.taskKeySet) { + completeInflightTask0(key, ex); + } + context.taskKeySet.clear(); + S3StreamMetricsManager.recordReadAheadLatency(MetricsLevel.INFO, timer.elapsedAs(TimeUnit.NANOSECONDS), S3Operation.BLOCK_CACHE_READ_AHEAD, true); + }); } @WithSpan - CompletableFuture> handleSyncReadAhead(TraceContext traceContext, long streamId, long startOffset, long endOffset, - int maxBytes, ReadAheadAgent agent, UUID uuid, - TimerUtil timer, ReadContext context) { + CompletableFuture> handleSyncReadAhead(TraceContext traceContext, long streamId, + long startOffset, long endOffset, + int maxBytes, ReadAheadAgent agent, UUID uuid, + TimerUtil timer, ReadContext context) { if (context.streamDataBlocksPair.isEmpty()) { return CompletableFuture.completedFuture(Collections.emptyList()); } if (LOGGER.isDebugEnabled()) { LOGGER.debug("[S3BlockCache] stream={}, {}-{}, read data block indices cost: {} ms", streamId, startOffset, endOffset, - timer.elapsedAs(TimeUnit.MILLISECONDS)); + timer.elapsedAs(TimeUnit.MILLISECONDS)); } List>> cfList = new ArrayList<>(); @@ -166,7 +168,7 @@ CompletableFuture> handleSyncReadAhead(TraceContext trac if (LOGGER.isDebugEnabled()) { LOGGER.debug("[S3BlockCache] sync ra acquire size: {}, uuid={}, stream={}, {}-{}, {}", - totalReserveSize, uuid, streamId, startOffset, endOffset, maxBytes); + totalReserveSize, uuid, streamId, startOffset, endOffset, maxBytes); } CompletableFuture throttleCf = inflightReadThrottle.acquire(traceContext, uuid, totalReserveSize); @@ -178,8 +180,8 @@ CompletableFuture> handleSyncReadAhead(TraceContext trac StreamDataBlock streamDataBlock = pair.getRight(); if (LOGGER.isDebugEnabled()) { LOGGER.debug("[S3BlockCache] sync ra, stream={}, {}-{}, read data block {} from {} [{}, {}), size={}", - streamId, startOffset, endOffset, streamDataBlock.getBlockId(), objectReader.objectKey(), - streamDataBlock.getBlockStartPosition(), streamDataBlock.getBlockEndPosition(), streamDataBlock.getBlockSize()); + streamId, startOffset, endOffset, streamDataBlock.getBlockId(), objectReader.objectKey(), + streamDataBlock.getBlockStartPosition(), streamDataBlock.getBlockEndPosition(), streamDataBlock.getBlockSize()); } String dataBlockKey = streamDataBlock.getObjectId() + "-" + streamDataBlock.getBlockId(); sortedDataBlockKeyList.add(dataBlockKey); @@ -191,35 +193,35 @@ CompletableFuture> handleSyncReadAhead(TraceContext trac boolean isNotAlignedFirstBlock = i == 0 && startOffset != streamDataBlock.getStartOffset(); if (isNotAlignedFirstBlock && context.taskKeySet.contains(taskKey)) { setInflightReadAheadStatus(new DefaultS3BlockCache.ReadAheadTaskKey(streamId, startOffset), - DefaultS3BlockCache.ReadBlockCacheStatus.WAIT_FETCH_DATA); + DefaultS3BlockCache.ReadBlockCacheStatus.WAIT_FETCH_DATA); } try { CompletableFuture> cf = TraceUtils.runWithSpanAsync(new TraceContext(traceContext), Attributes.empty(), "StreamReader::readDataBlock", - () -> reserveResult.cf().thenApplyAsync(dataBlock -> { - if (dataBlock.records().isEmpty()) { - return new ArrayList(); - } - // retain records to be returned - dataBlock.records().forEach(StreamRecordBatch::retain); - recordsMap.put(dataBlockKey, dataBlock.records()); - - // retain records to be put into block cache - dataBlock.records().forEach(StreamRecordBatch::retain); - blockCache.put(streamId, dataBlock.records()); - dataBlock.release(); - - return dataBlock.records(); - }, backgroundExecutor).whenComplete((ret, ex) -> { - if (ex != null) { - LOGGER.error("[S3BlockCache] sync ra fail to read data block, stream={}, {}-{}, data block: {}", - streamId, startOffset, endOffset, streamDataBlock, ex); - } - completeInflightTask(context, taskKey, ex); - if (isNotAlignedFirstBlock) { - // in case of first data block and startOffset is not aligned with start of data block - completeInflightTask(context, new DefaultS3BlockCache.ReadAheadTaskKey(streamId, startOffset), ex); - } - })); + () -> reserveResult.cf().thenApplyAsync(dataBlock -> { + if (dataBlock.records().isEmpty()) { + return new ArrayList(); + } + // retain records to be returned + dataBlock.records().forEach(StreamRecordBatch::retain); + recordsMap.put(dataBlockKey, dataBlock.records()); + + // retain records to be put into block cache + dataBlock.records().forEach(StreamRecordBatch::retain); + blockCache.put(streamId, dataBlock.records()); + dataBlock.release(); + + return dataBlock.records(); + }, backgroundExecutor).whenComplete((ret, ex) -> { + if (ex != null) { + LOGGER.error("[S3BlockCache] sync ra fail to read data block, stream={}, {}-{}, data block: {}", + streamId, startOffset, endOffset, streamDataBlock, ex); + } + completeInflightTask(context, taskKey, ex); + if (isNotAlignedFirstBlock) { + // in case of first data block and startOffset is not aligned with start of data block + completeInflightTask(context, new DefaultS3BlockCache.ReadAheadTaskKey(streamId, startOffset), ex); + } + })); cfList.add(cf); } catch (Throwable e) { throw new IllegalArgumentException(e); @@ -231,8 +233,8 @@ CompletableFuture> handleSyncReadAhead(TraceContext trac return CompletableFuture.allOf(cfList.toArray(CompletableFuture[]::new)).thenApply(vv -> { if (LOGGER.isDebugEnabled()) { LOGGER.debug("[S3BlockCache] sync read ahead complete, stream={}, {}-{}, maxBytes: {}, " + - "result: {}-{}, {}, cost: {} ms", streamId, startOffset, endOffset, maxBytes, - startOffset, context.lastOffset, context.totalReadSize, timer.elapsedAs(TimeUnit.MILLISECONDS)); + "result: {}-{}, {}, cost: {} ms", streamId, startOffset, endOffset, maxBytes, + startOffset, context.lastOffset, context.totalReadSize, timer.elapsedAs(TimeUnit.MILLISECONDS)); } List recordsToReturn = new LinkedList<>(); @@ -255,14 +257,14 @@ CompletableFuture> handleSyncReadAhead(TraceContext trac remainBytes -= record.size(); } long lastReadOffset = recordsToReturn.isEmpty() ? totalRecords.get(0).getBaseOffset() - : recordsToReturn.get(recordsToReturn.size() - 1).getLastOffset(); + : recordsToReturn.get(recordsToReturn.size() - 1).getLastOffset(); blockCache.setReadAheadRecord(streamId, lastReadOffset, context.lastOffset); agent.updateReadAheadResult(context.lastOffset, context.totalReadSize); return recordsToReturn; }).whenComplete((ret, ex) -> { if (ex != null) { LOGGER.error("[S3BlockCache] sync read ahead fail, stream={}, {}-{}, maxBytes: {}, cost: {} ms", - streamId, startOffset, endOffset, maxBytes, timer.elapsedAs(TimeUnit.MILLISECONDS), ex); + streamId, startOffset, endOffset, maxBytes, timer.elapsedAs(TimeUnit.MILLISECONDS), ex); errorHandlerExecutor.execute(() -> cleanUpOnCompletion(cfList)); } context.releaseReader(); @@ -288,29 +290,30 @@ public void asyncReadAhead(long streamId, long startOffset, long endOffset, int DefaultS3BlockCache.ReadAheadTaskKey readAheadTaskKey = new DefaultS3BlockCache.ReadAheadTaskKey(streamId, startOffset); // put a placeholder task at start offset to prevent next cache miss request spawn duplicated read ahead task DefaultS3BlockCache.ReadAheadTaskContext readAheadTaskContext = new DefaultS3BlockCache.ReadAheadTaskContext(new CompletableFuture<>(), - DefaultS3BlockCache.ReadBlockCacheStatus.WAIT_DATA_INDEX); + DefaultS3BlockCache.ReadBlockCacheStatus.WAIT_DATA_INDEX); inflightReadAheadTaskMap.putIfAbsent(readAheadTaskKey, readAheadTaskContext); context.taskKeySet.add(readAheadTaskKey); getDataBlockIndices(TraceContext.DEFAULT, streamId, endOffset, context) - .thenAcceptAsync(v -> - handleAsyncReadAhead(streamId, startOffset, endOffset, maxBytes, agent, timer, context), streamReaderExecutor) - .whenComplete((nil, ex) -> { - for (DefaultS3BlockCache.ReadAheadTaskKey key : context.taskKeySet) { - completeInflightTask0(key, ex); - } - context.taskKeySet.clear(); - S3StreamMetricsManager.recordReadAheadLatency(MetricsLevel.INFO, timer.elapsedAs(TimeUnit.NANOSECONDS), S3Operation.BLOCK_CACHE_READ_AHEAD, false); - }); + .thenAcceptAsync(v -> + handleAsyncReadAhead(streamId, startOffset, endOffset, maxBytes, agent, timer, context), streamReaderExecutor) + .whenComplete((nil, ex) -> { + for (DefaultS3BlockCache.ReadAheadTaskKey key : context.taskKeySet) { + completeInflightTask0(key, ex); + } + context.taskKeySet.clear(); + S3StreamMetricsManager.recordReadAheadLatency(MetricsLevel.INFO, timer.elapsedAs(TimeUnit.NANOSECONDS), S3Operation.BLOCK_CACHE_READ_AHEAD, false); + }); } - CompletableFuture handleAsyncReadAhead(long streamId, long startOffset, long endOffset, int maxBytes, ReadAheadAgent agent, - TimerUtil timer, ReadContext context) { + CompletableFuture handleAsyncReadAhead(long streamId, long startOffset, long endOffset, int maxBytes, + ReadAheadAgent agent, + TimerUtil timer, ReadContext context) { if (context.streamDataBlocksPair.isEmpty()) { return CompletableFuture.completedFuture(null); } if (LOGGER.isDebugEnabled()) { LOGGER.debug("[S3BlockCache] stream={}, {}-{}, read data block indices cost: {} ms", streamId, startOffset, endOffset, - timer.elapsedAs(TimeUnit.MILLISECONDS)); + timer.elapsedAs(TimeUnit.MILLISECONDS)); } List> cfList = new ArrayList<>(); @@ -324,8 +327,8 @@ CompletableFuture handleAsyncReadAhead(long streamId, long startOffset, lo StreamDataBlock streamDataBlock = pair.getRight(); if (LOGGER.isDebugEnabled()) { LOGGER.debug("[S3BlockCache] async ra, stream={}, {}-{}, read data block {} from {} [{}, {}), size={}", - streamId, startOffset, endOffset, streamDataBlock.getBlockId(), objectReader.objectKey(), - streamDataBlock.getBlockStartPosition(), streamDataBlock.getBlockEndPosition(), streamDataBlock.getBlockSize()); + streamId, startOffset, endOffset, streamDataBlock.getBlockId(), objectReader.objectKey(), + streamDataBlock.getBlockStartPosition(), streamDataBlock.getBlockEndPosition(), streamDataBlock.getBlockSize()); } UUID uuid = UUID.randomUUID(); DefaultS3BlockCache.ReadAheadTaskKey taskKey = new DefaultS3BlockCache.ReadAheadTaskKey(streamId, streamDataBlock.getStartOffset()); @@ -348,7 +351,7 @@ CompletableFuture handleAsyncReadAhead(long streamId, long startOffset, lo }, backgroundExecutor).whenComplete((ret, ex) -> { if (ex != null) { LOGGER.error("[S3BlockCache] async ra fail to read data block, stream={}, {}-{}, data block: {}", - streamId, startOffset, endOffset, streamDataBlock, ex); + streamId, startOffset, endOffset, streamDataBlock, ex); } inflightReadThrottle.release(uuid); completeInflightTask(context, taskKey, ex); @@ -361,7 +364,7 @@ CompletableFuture handleAsyncReadAhead(long streamId, long startOffset, lo if (LOGGER.isDebugEnabled()) { LOGGER.debug("[S3BlockCache] async ra acquire size: {}, uuid={}, stream={}, {}-{}, {}", - reserveResult.reserveSize(), uuid, streamId, startOffset, endOffset, maxBytes); + reserveResult.reserveSize(), uuid, streamId, startOffset, endOffset, maxBytes); } if (reserveResult.reserveSize() > 0) { inflightReadThrottle.acquire(TraceContext.DEFAULT, uuid, reserveResult.reserveSize()).thenAcceptAsync(nil -> { @@ -371,7 +374,7 @@ CompletableFuture handleAsyncReadAhead(long streamId, long startOffset, lo } if (isNotAlignedFirstBlock && context.taskKeySet.contains(taskKey)) { setInflightReadAheadStatus(new DefaultS3BlockCache.ReadAheadTaskKey(streamId, startOffset), - DefaultS3BlockCache.ReadBlockCacheStatus.WAIT_FETCH_DATA); + DefaultS3BlockCache.ReadBlockCacheStatus.WAIT_FETCH_DATA); } dataBlockReadAccumulator.readDataBlock(objectReader, streamDataBlock.dataBlockIndex()); }, streamReaderExecutor).exceptionally(ex -> { @@ -383,13 +386,13 @@ CompletableFuture handleAsyncReadAhead(long streamId, long startOffset, lo return CompletableFuture.allOf(cfList.toArray(CompletableFuture[]::new)).whenComplete((ret, ex) -> { if (ex != null) { LOGGER.error("[S3BlockCache] async ra failed, stream={}, {}-{}, maxBytes: {}, " + - "result: {}-{}, {}, cost: {} ms, ", streamId, startOffset, endOffset, maxBytes, - startOffset, context.lastOffset, context.totalReadSize, timer.elapsedAs(TimeUnit.MILLISECONDS), ex); + "result: {}-{}, {}, cost: {} ms, ", streamId, startOffset, endOffset, maxBytes, + startOffset, context.lastOffset, context.totalReadSize, timer.elapsedAs(TimeUnit.MILLISECONDS), ex); } else { if (LOGGER.isDebugEnabled()) { LOGGER.debug("[S3BlockCache] async ra complete, stream={}, {}-{}, maxBytes: {}, " + - "result: {}-{}, {}, cost: {} ms", streamId, startOffset, endOffset, maxBytes, - startOffset, context.lastOffset, context.totalReadSize, timer.elapsedAs(TimeUnit.MILLISECONDS)); + "result: {}-{}, {}, cost: {} ms", streamId, startOffset, endOffset, maxBytes, + startOffset, context.lastOffset, context.totalReadSize, timer.elapsedAs(TimeUnit.MILLISECONDS)); } } context.releaseReader(); @@ -398,27 +401,28 @@ CompletableFuture handleAsyncReadAhead(long streamId, long startOffset, lo } @WithSpan - CompletableFuture getDataBlockIndices(TraceContext traceContext, long streamId, long endOffset, ReadContext context) { + CompletableFuture getDataBlockIndices(TraceContext traceContext, long streamId, long endOffset, + ReadContext context) { traceContext.currentContext(); CompletableFuture getObjectsCf = CompletableFuture.completedFuture(false); if (context.objectIndex >= context.objects.size()) { getObjectsCf = objectManager - .getObjects(streamId, context.nextStartOffset, endOffset, READ_OBJECT_INDEX_STEP) - .orTimeout(1, TimeUnit.MINUTES) - .thenApply(objects -> { - context.objects = objects; - context.objectIndex = 0; - if (context.objects.isEmpty()) { - if (endOffset == -1L) { // background read ahead - return true; - } else { - LOGGER.error("[BUG] fail to read, expect objects not empty, streamId={}, startOffset={}, endOffset={}", - streamId, context.nextStartOffset, endOffset); - throw new IllegalStateException("fail to read, expect objects not empty"); - } + .getObjects(streamId, context.nextStartOffset, endOffset, READ_OBJECT_INDEX_STEP) + .orTimeout(1, TimeUnit.MINUTES) + .thenApply(objects -> { + context.objects = objects; + context.objectIndex = 0; + if (context.objects.isEmpty()) { + if (endOffset == -1L) { // background read ahead + return true; + } else { + LOGGER.error("[BUG] fail to read, expect objects not empty, streamId={}, startOffset={}, endOffset={}", + streamId, context.nextStartOffset, endOffset); + throw new IllegalStateException("fail to read, expect objects not empty"); } - return false; - }); + } + return false; + }); } CompletableFuture findIndexCf = getObjectsCf.thenComposeAsync(emptyObjects -> { if (emptyObjects) { @@ -442,7 +446,7 @@ CompletableFuture getDataBlockIndices(TraceContext traceContext, long stre for (StreamDataBlock streamDataBlock : streamDataBlocks) { DefaultS3BlockCache.ReadAheadTaskKey taskKey = new DefaultS3BlockCache.ReadAheadTaskKey(streamId, streamDataBlock.getStartOffset()); DefaultS3BlockCache.ReadAheadTaskContext readAheadContext = new DefaultS3BlockCache.ReadAheadTaskContext(new CompletableFuture<>(), - DefaultS3BlockCache.ReadBlockCacheStatus.WAIT_THROTTLE); + DefaultS3BlockCache.ReadBlockCacheStatus.WAIT_THROTTLE); if (inflightReadAheadTaskMap.putIfAbsent(taskKey, readAheadContext) == null) { context.taskKeySet.add(taskKey); } @@ -473,7 +477,8 @@ CompletableFuture getDataBlockIndices(TraceContext traceContext, long stre }, streamReaderExecutor); } - private void setInflightReadAheadStatus(DefaultS3BlockCache.ReadAheadTaskKey key, DefaultS3BlockCache.ReadBlockCacheStatus status) { + private void setInflightReadAheadStatus(DefaultS3BlockCache.ReadAheadTaskKey key, + DefaultS3BlockCache.ReadBlockCacheStatus status) { inflightReadAheadTaskMap.computeIfPresent(key, (k, readAheadContext) -> { readAheadContext.status = status; return readAheadContext; diff --git a/s3stream/src/main/java/com/automq/stream/s3/compact/CompactionAnalyzer.java b/s3stream/src/main/java/com/automq/stream/s3/compact/CompactionAnalyzer.java index 5b31591e3..85014af4e 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/compact/CompactionAnalyzer.java +++ b/s3stream/src/main/java/com/automq/stream/s3/compact/CompactionAnalyzer.java @@ -17,15 +17,11 @@ package com.automq.stream.s3.compact; +import com.automq.stream.s3.StreamDataBlock; 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.StreamDataBlock; import com.automq.stream.utils.LogContext; -import org.apache.commons.lang3.tuple.ImmutablePair; -import org.apache.commons.lang3.tuple.Pair; -import org.slf4j.Logger; - import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -37,6 +33,9 @@ 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; +import org.slf4j.Logger; public class CompactionAnalyzer { private final Logger logger; @@ -45,12 +44,13 @@ public class CompactionAnalyzer { private final int maxStreamNumInStreamSet; private final int maxStreamObjectNum; - public CompactionAnalyzer(long compactionCacheSize, long streamSplitSize, int maxStreamNumInStreamSet, int maxStreamObjectNum) { + public CompactionAnalyzer(long compactionCacheSize, long streamSplitSize, int maxStreamNumInStreamSet, + int maxStreamObjectNum) { this(compactionCacheSize, streamSplitSize, maxStreamNumInStreamSet, maxStreamObjectNum, new LogContext("[CompactionAnalyzer]")); } public CompactionAnalyzer(long compactionCacheSize, long streamSplitSize, - int maxStreamNumInStreamSet, int maxStreamObjectNum, LogContext logContext) { + int maxStreamNumInStreamSet, int maxStreamObjectNum, LogContext logContext) { this.logger = logContext.logger(CompactionAnalyzer.class); this.compactionCacheSize = compactionCacheSize; this.streamSplitSize = streamSplitSize; @@ -58,7 +58,8 @@ public CompactionAnalyzer(long compactionCacheSize, long streamSplitSize, this.maxStreamObjectNum = maxStreamObjectNum; } - public List analyze(Map> streamDataBlockMap, Set excludedObjectIds) { + public List analyze(Map> streamDataBlockMap, + Set excludedObjectIds) { if (streamDataBlockMap.isEmpty()) { return Collections.emptyList(); } @@ -81,10 +82,11 @@ public List analyze(Map> streamDataB * with compaction limitation ({@code maxStreamObjectNum} and {@code maxStreamNumInStreamSet}). * * @param streamDataBlockMap stream data blocks map, key: object id, value: stream data blocks - * @param excludedObjectIds objects that are excluded from compaction because of compaction limitation + * @param excludedObjectIds objects that are excluded from compaction because of compaction limitation * @return list of {@link CompactedObjectBuilder} */ - List groupObjectWithLimits(Map> streamDataBlockMap, Set excludedObjectIds) { + List groupObjectWithLimits(Map> streamDataBlockMap, + Set excludedObjectIds) { List sortedStreamDataBlocks = sortStreamRangePositions(streamDataBlockMap); List compactedObjectBuilders = new ArrayList<>(); CompactionStats stats = null; @@ -121,7 +123,8 @@ List groupObjectWithLimits(Map maxStreamNumInStreamSet || streamObjectNum > maxStreamObjectNum); + } + while (streamNumInStreamSet > maxStreamNumInStreamSet || streamObjectNum > maxStreamObjectNum); return compactedObjectBuilders; } @@ -129,14 +132,14 @@ List groupObjectWithLimits(Map compactedObjectBuilders, - CompactionStats stats, Set objectsToRemove) { + CompactionStats stats, Set objectsToRemove) { List sortedCompactedObjectIndexList = new ArrayList<>(); for (CompactedObjectBuilder compactedObjectBuilder : compactedObjectBuilders) { // find all compacted objects of the same type @@ -151,8 +154,8 @@ private void addObjectsToRemove(CompactionType compactionType, List generatePlanWithCacheLimit(List com } private CompactedObjectBuilder addOrMergeCompactedObject(CompactedObjectBuilder compactedObjectBuilder, - List compactedObjects, - CompactedObjectBuilder compactedStreamSetObjectBuilder) { + List compactedObjects, + CompactedObjectBuilder compactedStreamSetObjectBuilder) { if (compactedObjectBuilder.type() == CompactionType.SPLIT) { compactedObjects.add(compactedObjectBuilder.build()); } else { @@ -251,7 +254,8 @@ private boolean shouldSplitObject(CompactedObjectBuilder compactedObjectBuilder) return true; } - private CompactionPlan generateCompactionPlan(List compactedObjects, CompactedObjectBuilder compactedStreamSetObject) { + private CompactionPlan generateCompactionPlan(List compactedObjects, + CompactedObjectBuilder compactedStreamSetObject) { if (compactedStreamSetObject != null) { compactedObjects.add(compactedStreamSetObject.build()); } @@ -292,7 +296,7 @@ private List compactObjects(List stream } else { // should not go there logger.error("FATAL ERROR: illegal stream range position, last offset: {}, curr: {}", - builder.lastOffset(), streamDataBlock); + builder.lastOffset(), streamDataBlock); return new ArrayList<>(); } } else { @@ -316,21 +320,21 @@ private List compactObjects(List stream Map> filterBlocksToCompact(Map> streamDataBlocksMap) { // group stream data blocks by stream id, key: stream id, value: ids of objects that contains this stream Map> streamToObjectIds = streamDataBlocksMap.values().stream() - .flatMap(Collection::stream) - .collect(Collectors.groupingBy(StreamDataBlock::getStreamId, Collectors.mapping(StreamDataBlock::getObjectId, Collectors.toSet()))); + .flatMap(Collection::stream) + .collect(Collectors.groupingBy(StreamDataBlock::getStreamId, Collectors.mapping(StreamDataBlock::getObjectId, Collectors.toSet()))); Set objectIdsToCompact = streamToObjectIds - .entrySet().stream() - .filter(e -> e.getValue().size() > 1) - .flatMap(e -> e.getValue().stream()) - .collect(Collectors.toSet()); + .entrySet().stream() + .filter(e -> e.getValue().size() > 1) + .flatMap(e -> e.getValue().stream()) + .collect(Collectors.toSet()); return streamDataBlocksMap.entrySet().stream() - .filter(e -> objectIdsToCompact.contains(e.getKey())) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + .filter(e -> objectIdsToCompact.contains(e.getKey())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); } private CompactedObjectBuilder splitAndAddBlock(CompactedObjectBuilder builder, - StreamDataBlock streamDataBlock, - List compactedObjectBuilders) { + StreamDataBlock streamDataBlock, + List compactedObjectBuilders) { if (builder.currStreamBlockSize() > streamSplitSize) { builder = splitObject(builder, compactedObjectBuilders); } @@ -339,7 +343,7 @@ private CompactedObjectBuilder splitAndAddBlock(CompactedObjectBuilder builder, } private CompactedObjectBuilder splitObject(CompactedObjectBuilder builder, - List compactedObjectBuilders) { + List compactedObjectBuilders) { CompactedObjectBuilder splitBuilder = builder.splitCurrentStream(); splitBuilder.setType(CompactionType.SPLIT); if (builder.totalBlockSize() != 0) { @@ -377,7 +381,7 @@ public AbstractCompactedObjectComparator(Map objectStatsMap) { protected int compareCompactedObject(CompactedObjectBuilder o1, CompactedObjectBuilder o2) { return Integer.compare(CompactionUtils.getTotalObjectStats(o1, objectStatsMap), - CompactionUtils.getTotalObjectStats(o2, objectStatsMap)); + CompactionUtils.getTotalObjectStats(o2, objectStatsMap)); } } diff --git a/s3stream/src/main/java/com/automq/stream/s3/compact/CompactionManager.java b/s3stream/src/main/java/com/automq/stream/s3/compact/CompactionManager.java index 5cdea0103..840cf8ead 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/compact/CompactionManager.java +++ b/s3stream/src/main/java/com/automq/stream/s3/compact/CompactionManager.java @@ -82,7 +82,8 @@ public class CompactionManager { private final long compactionCacheSize; private Bucket compactionBucket = null; - public CompactionManager(Config config, ObjectManager objectManager, StreamManager streamManager, S3Operator s3Operator) { + public CompactionManager(Config config, ObjectManager objectManager, StreamManager streamManager, + S3Operator s3Operator) { String logPrefix = String.format("[CompactionManager id=%d] ", config.nodeId()); this.logger = new LogContext(logPrefix).logger(CompactionManager.class); this.s3ObjectLogger = S3ObjectLogger.logger(logPrefix); @@ -101,16 +102,16 @@ public CompactionManager(Config config, ObjectManager objectManager, StreamManag maxStreamNumPerStreamSetObject = config.maxStreamNumPerStreamSetObject(); maxStreamObjectNumPerCommit = config.maxStreamObjectNumPerCommit(); this.compactionAnalyzer = new CompactionAnalyzer(compactionCacheSize, streamSplitSize, maxStreamNumPerStreamSetObject, - maxStreamObjectNumPerCommit, new LogContext(String.format("[CompactionAnalyzer id=%d] ", config.nodeId()))); + maxStreamObjectNumPerCommit, new LogContext(String.format("[CompactionAnalyzer id=%d] ", config.nodeId()))); this.compactScheduledExecutor = Threads.newSingleThreadScheduledExecutor( - ThreadUtils.createThreadFactory("schedule-compact-executor-%d", true), logger); + ThreadUtils.createThreadFactory("schedule-compact-executor-%d", true), logger); this.bucketCallbackScheduledExecutor = Threads.newSingleThreadScheduledExecutor( - ThreadUtils.createThreadFactory("s3-data-block-reader-bucket-cb-%d", true), logger); + ThreadUtils.createThreadFactory("s3-data-block-reader-bucket-cb-%d", true), logger); this.compactThreadPool = Executors.newFixedThreadPool(1, new DefaultThreadFactory("object-compaction-manager")); this.forceSplitThreadPool = Executors.newFixedThreadPool(1, new DefaultThreadFactory("force-split-executor")); this.logger.info("Compaction manager initialized with config: compactionInterval: {} min, compactionCacheSize: {} bytes, " + - "streamSplitSize: {} bytes, forceSplitObjectPeriod: {} min, maxObjectNumToCompact: {}, maxStreamNumInStreamSet: {}, maxStreamObjectNum: {}", - compactionInterval, compactionCacheSize, streamSplitSize, forceSplitObjectPeriod, maxObjectNumToCompact, maxStreamNumPerStreamSetObject, maxStreamObjectNumPerCommit); + "streamSplitSize: {} bytes, forceSplitObjectPeriod: {} min, maxObjectNumToCompact: {}, maxStreamNumInStreamSet: {}, maxStreamObjectNum: {}", + compactionInterval, compactionCacheSize, streamSplitSize, forceSplitObjectPeriod, maxObjectNumToCompact, maxStreamNumPerStreamSetObject, maxStreamObjectNumPerCommit); } public void start() { @@ -124,11 +125,11 @@ private void scheduleNextCompaction(long delayMillis) { try { logger.info("Compaction started"); this.compact() - .thenAccept(result -> logger.info("Compaction complete, total cost {} ms", timerUtil.elapsedAs(TimeUnit.MILLISECONDS))) - .exceptionally(ex -> { - logger.error("Compaction failed, cost {} ms, ", timerUtil.elapsedAs(TimeUnit.MILLISECONDS), ex); - return null; - }).join(); + .thenAccept(result -> logger.info("Compaction complete, total cost {} ms", timerUtil.elapsedAs(TimeUnit.MILLISECONDS))) + .exceptionally(ex -> { + logger.error("Compaction failed, cost {} ms, ", timerUtil.elapsedAs(TimeUnit.MILLISECONDS), ex); + return null; + }).join(); } catch (Exception ex) { logger.error("Error while compacting objects ", ex); } @@ -146,13 +147,14 @@ public void shutdown() { private CompletableFuture compact() { return this.objectManager.getServerObjects().thenComposeAsync(objectMetadataList -> { List streamIds = objectMetadataList.stream().flatMap(e -> e.getOffsetRanges().stream()) - .map(StreamOffsetRange::getStreamId).distinct().toList(); + .map(StreamOffsetRange::getStreamId).distinct().toList(); return this.streamManager.getStreams(streamIds).thenAcceptAsync(streamMetadataList -> - this.compact(streamMetadataList, objectMetadataList), compactThreadPool); + this.compact(streamMetadataList, objectMetadataList), compactThreadPool); }, compactThreadPool); } - private void compact(List streamMetadataList, List objectMetadataList) throws CompletionException { + private void compact(List streamMetadataList, + List objectMetadataList) throws CompletionException { logger.info("Get {} stream set objects from metadata", objectMetadataList.size()); if (objectMetadataList.isEmpty()) { return; @@ -169,10 +171,10 @@ private void compact(List streamMetadataList, List 0) { expectReadBytesPerSec = totalSize / expectCompleteTime / 60; compactionBucket = Bucket.builder().addLimit(limit -> limit - .capacity(expectReadBytesPerSec) - .refillIntervally(expectReadBytesPerSec, Duration.ofSeconds(1))).build(); + .capacity(expectReadBytesPerSec) + .refillIntervally(expectReadBytesPerSec, Duration.ofSeconds(1))).build(); logger.info("Throttle compaction read to {} bytes/s, expect to complete in no less than {}min", - expectReadBytesPerSec, expectCompleteTime); + expectReadBytesPerSec, expectCompleteTime); } else { logger.warn("Compaction interval {}min is too small, there will be no throttle for compaction", compactionInterval); } @@ -192,7 +194,7 @@ void forceSplitObjects(List streamMetadataList, List streamMetadataList, List { - logger.info("Commit force split request succeed, time cost: {} ms", timerUtil.elapsedAs(TimeUnit.MILLISECONDS)); - if (s3ObjectLogEnable) { - s3ObjectLogger.trace("[Compact] {}", request); - } - }) - .exceptionally(ex -> { - logger.error("Commit force split request failed, ex: ", ex); - return null; - }) - .join(); + .thenAccept(resp -> { + logger.info("Commit force split request succeed, time cost: {} ms", timerUtil.elapsedAs(TimeUnit.MILLISECONDS)); + if (s3ObjectLogEnable) { + s3ObjectLogger.trace("[Compact] {}", request); + } + }) + .exceptionally(ex -> { + logger.error("Commit force split request failed, ex: ", ex); + return null; + }) + .join(); } } private void compactObjects(List streamMetadataList, List objectsToCompact) - throws CompletionException { + throws CompletionException { if (objectsToCompact.isEmpty()) { return; } @@ -243,20 +245,20 @@ private void compactObjects(List streamMetadataList, List { - logger.info("Commit compact request succeed, time cost: {} ms", timerUtil.elapsedAs(TimeUnit.MILLISECONDS)); - if (s3ObjectLogEnable) { - s3ObjectLogger.trace("[Compact] {}", request); - } - }) - .exceptionally(ex -> { - logger.error("Commit compact request failed, ex: ", ex); - return null; - }) - .join(); + .thenAccept(resp -> { + logger.info("Commit compact request succeed, time cost: {} ms", timerUtil.elapsedAs(TimeUnit.MILLISECONDS)); + if (s3ObjectLogEnable) { + s3ObjectLogger.trace("[Compact] {}", request); + } + }) + .exceptionally(ex -> { + logger.error("Commit compact request failed, ex: ", ex); + return null; + }) + .join(); } private void logCompactionPlans(List compactionPlans, Set excludedObjectIds) { @@ -265,19 +267,19 @@ private void logCompactionPlans(List compactionPlans, Set return; } long streamObjectNum = compactionPlans.stream() - .mapToLong(p -> p.compactedObjects().stream() - .filter(o -> o.type() == CompactionType.SPLIT) - .count()) - .sum(); + .mapToLong(p -> p.compactedObjects().stream() + .filter(o -> o.type() == CompactionType.SPLIT) + .count()) + .sum(); long streamSetObjectSize = compactionPlans.stream() - .mapToLong(p -> p.compactedObjects().stream() - .filter(o -> o.type() == CompactionType.COMPACT) - .mapToLong(CompactedObject::size) - .sum()) - .sum(); + .mapToLong(p -> p.compactedObjects().stream() + .filter(o -> o.type() == CompactionType.COMPACT) + .mapToLong(CompactedObject::size) + .sum()) + .sum(); int streamSetObjectNum = streamSetObjectSize > 0 ? 1 : 0; logger.info("Compaction plans: expect to generate {} Stream Object, {} stream set object with size {} in {} iterations, objects excluded: {}", - streamObjectNum, streamSetObjectNum, streamSetObjectSize, compactionPlans.size(), excludedObjectIds); + streamObjectNum, streamSetObjectNum, streamSetObjectSize, compactionPlans.size(), excludedObjectIds); } public CompletableFuture forceSplitAll() { @@ -285,7 +287,7 @@ public CompletableFuture forceSplitAll() { //TODO: deal with metadata delay this.compactScheduledExecutor.execute(() -> this.objectManager.getServerObjects().thenAcceptAsync(objectMetadataList -> { List streamIds = objectMetadataList.stream().flatMap(e -> e.getOffsetRanges().stream()) - .map(StreamOffsetRange::getStreamId).distinct().toList(); + .map(StreamOffsetRange::getStreamId).distinct().toList(); this.streamManager.getStreams(streamIds).thenAcceptAsync(streamMetadataList -> { if (objectMetadataList.isEmpty()) { logger.info("No stream set objects to force split"); @@ -307,16 +309,17 @@ public CompletableFuture forceSplitAll() { * Split specified stream set object into stream objects. * * @param streamMetadataList metadata of opened streams - * @param objectMetadata stream set object to split + * @param objectMetadata stream set object to split * @return List of CompletableFuture of StreamObject */ - private Collection> splitStreamSetObject(List streamMetadataList, S3ObjectMetadata objectMetadata) { + private Collection> splitStreamSetObject(List streamMetadataList, + S3ObjectMetadata objectMetadata) { if (objectMetadata == null) { return new ArrayList<>(); } Map> streamDataBlocksMap = CompactionUtils.blockWaitObjectIndices(streamMetadataList, - Collections.singletonList(objectMetadata), s3Operator, logger); + Collections.singletonList(objectMetadata), s3Operator, logger); if (streamDataBlocksMap.isEmpty()) { // object not exist, metadata is out of date logger.warn("Object {} not exist, metadata is out of date", objectMetadata.objectId()); @@ -332,7 +335,8 @@ private Collection> splitStreamSetObject(List> groupAndSplitStreamDataBlocks(S3ObjectMetadata objectMetadata, List streamDataBlocks) { + Collection> groupAndSplitStreamDataBlocks(S3ObjectMetadata objectMetadata, + List streamDataBlocks) { List, CompletableFuture>> groupedDataBlocks = new ArrayList<>(); List> groupedStreamDataBlocks = CompactionUtils.groupStreamDataBlocks(streamDataBlocks); for (List group : groupedStreamDataBlocks) { @@ -348,7 +352,7 @@ Collection> groupAndSplitStreamDataBlocks(S3Obje Pair, CompletableFuture> group = groupedDataBlocks.get(index); List groupedStreamDataBlock = group.getLeft(); long size = groupedStreamDataBlock.get(groupedStreamDataBlock.size() - 1).getBlockEndPosition() - - groupedStreamDataBlock.get(0).getBlockStartPosition(); + groupedStreamDataBlock.get(0).getBlockStartPosition(); if (readSize + size > compactionCacheSize) { break; } @@ -362,51 +366,52 @@ Collection> groupAndSplitStreamDataBlocks(S3Obje } // prepare N stream objects at one time objectManager.prepareObject(batchGroup.size(), TimeUnit.MINUTES.toMillis(CompactionConstants.S3_OBJECT_TTL_MINUTES)) - .thenComposeAsync(objectId -> { - List blocksToRead = batchGroup.stream().flatMap(p -> p.getLeft().stream()).toList(); - DataBlockReader reader = new DataBlockReader(objectMetadata, s3Operator, compactionBucket, bucketCallbackScheduledExecutor); - // batch read - reader.readBlocks(blocksToRead, Math.min(CompactionConstants.S3_OBJECT_MAX_READ_BATCH, networkBandwidth)); - - List> cfs = new ArrayList<>(); - for (Pair, CompletableFuture> pair : batchGroup) { - List blocks = pair.getLeft(); - DataBlockWriter writer = new DataBlockWriter(objectId, s3Operator, config.objectPartSize()); - CompletableFuture cf = CompactionUtils.chainWriteDataBlock(writer, blocks, forceSplitThreadPool); - long finalObjectId = objectId; - cfs.add(cf.thenAccept(nil -> writer.close()).whenComplete((ret, ex) -> { - if (ex != null) { - logger.error("write to stream object {} failed", finalObjectId, ex); - writer.release(); - blocks.forEach(StreamDataBlock::release); - return; - } - StreamObject streamObject = new StreamObject(); - streamObject.setObjectId(finalObjectId); - streamObject.setStreamId(blocks.get(0).getStreamId()); - streamObject.setStartOffset(blocks.get(0).getStartOffset()); - streamObject.setEndOffset(blocks.get(blocks.size() - 1).getEndOffset()); - streamObject.setObjectSize(writer.size()); - pair.getValue().complete(streamObject); - })); - objectId++; - } - return CompletableFuture.allOf(cfs.toArray(new CompletableFuture[0])); - }, forceSplitThreadPool) - .exceptionally(ex -> { - logger.error("Force split object {} failed", objectMetadata.objectId(), ex); - for (Pair, CompletableFuture> pair : groupedDataBlocks) { - pair.getValue().completeExceptionally(ex); - } - throw new IllegalStateException(String.format("Force split object %d failed", objectMetadata.objectId()), ex); - }).join(); + .thenComposeAsync(objectId -> { + List blocksToRead = batchGroup.stream().flatMap(p -> p.getLeft().stream()).toList(); + DataBlockReader reader = new DataBlockReader(objectMetadata, s3Operator, compactionBucket, bucketCallbackScheduledExecutor); + // batch read + reader.readBlocks(blocksToRead, Math.min(CompactionConstants.S3_OBJECT_MAX_READ_BATCH, networkBandwidth)); + + List> cfs = new ArrayList<>(); + for (Pair, CompletableFuture> pair : batchGroup) { + List blocks = pair.getLeft(); + DataBlockWriter writer = new DataBlockWriter(objectId, s3Operator, config.objectPartSize()); + CompletableFuture cf = CompactionUtils.chainWriteDataBlock(writer, blocks, forceSplitThreadPool); + long finalObjectId = objectId; + cfs.add(cf.thenAccept(nil -> writer.close()).whenComplete((ret, ex) -> { + if (ex != null) { + logger.error("write to stream object {} failed", finalObjectId, ex); + writer.release(); + blocks.forEach(StreamDataBlock::release); + return; + } + StreamObject streamObject = new StreamObject(); + streamObject.setObjectId(finalObjectId); + streamObject.setStreamId(blocks.get(0).getStreamId()); + streamObject.setStartOffset(blocks.get(0).getStartOffset()); + streamObject.setEndOffset(blocks.get(blocks.size() - 1).getEndOffset()); + streamObject.setObjectSize(writer.size()); + pair.getValue().complete(streamObject); + })); + objectId++; + } + return CompletableFuture.allOf(cfs.toArray(new CompletableFuture[0])); + }, forceSplitThreadPool) + .exceptionally(ex -> { + logger.error("Force split object {} failed", objectMetadata.objectId(), ex); + for (Pair, CompletableFuture> pair : groupedDataBlocks) { + pair.getValue().completeExceptionally(ex); + } + throw new IllegalStateException(String.format("Force split object %d failed", objectMetadata.objectId()), ex); + }).join(); } return groupedDataBlocks.stream().map(Pair::getValue).collect(Collectors.toList()); } - CommitStreamSetObjectRequest buildSplitRequest(List streamMetadataList, S3ObjectMetadata objectToSplit) - throws CompletionException { + CommitStreamSetObjectRequest buildSplitRequest(List streamMetadataList, + S3ObjectMetadata objectToSplit) + throws CompletionException { Collection> cfs = splitStreamSetObject(streamMetadataList, objectToSplit); if (cfs.isEmpty()) { logger.error("Force split object {} failed, no stream object generated", objectToSplit.objectId()); @@ -434,15 +439,16 @@ CommitStreamSetObjectRequest buildSplitRequest(List streamMetada return request; } - CommitStreamSetObjectRequest buildCompactRequest(List streamMetadataList, List objectsToCompact) - throws CompletionException { + CommitStreamSetObjectRequest buildCompactRequest(List streamMetadataList, + List objectsToCompact) + throws CompletionException { CommitStreamSetObjectRequest request = new CommitStreamSetObjectRequest(); Set compactedObjectIds = new HashSet<>(); logger.info("{} stream set objects as compact candidates, total compaction size: {}", - objectsToCompact.size(), objectsToCompact.stream().mapToLong(S3ObjectMetadata::objectSize).sum()); + objectsToCompact.size(), objectsToCompact.stream().mapToLong(S3ObjectMetadata::objectSize).sum()); Map> streamDataBlockMap = CompactionUtils.blockWaitObjectIndices(streamMetadataList, - objectsToCompact, s3Operator, logger); + objectsToCompact, s3Operator, logger); for (List blocks : streamDataBlockMap.values()) { for (StreamDataBlock block : blocks) { if (block.getBlockSize() > compactionCacheSize) { @@ -462,7 +468,7 @@ CommitStreamSetObjectRequest buildCompactRequest(List streamMeta request.setCompactedObjectIds(new ArrayList<>(compactedObjectIds)); List compactedObjectMetadata = objectsToCompact.stream() - .filter(e -> compactedObjectIds.contains(e.objectId())).toList(); + .filter(e -> compactedObjectIds.contains(e.objectId())).toList(); if (isSanityCheckFailed(streamMetadataList, compactedObjectMetadata, request)) { logger.error("Sanity check failed, compaction result is illegal"); return null; @@ -472,17 +478,17 @@ CommitStreamSetObjectRequest buildCompactRequest(List streamMeta } boolean isSanityCheckFailed(List streamMetadataList, List compactedObjects, - CommitStreamSetObjectRequest request) { + CommitStreamSetObjectRequest request) { Map streamMetadataMap = streamMetadataList.stream() - .collect(Collectors.toMap(StreamMetadata::streamId, e -> e)); + .collect(Collectors.toMap(StreamMetadata::streamId, e -> e)); Map objectMetadataMap = compactedObjects.stream() - .collect(Collectors.toMap(S3ObjectMetadata::objectId, e -> e)); + .collect(Collectors.toMap(S3ObjectMetadata::objectId, e -> e)); List compactedStreamOffsetRanges = new ArrayList<>(); request.getStreamRanges().forEach(o -> compactedStreamOffsetRanges.add(new StreamOffsetRange(o.getStreamId(), o.getStartOffset(), o.getEndOffset()))); request.getStreamObjects().forEach(o -> compactedStreamOffsetRanges.add(new StreamOffsetRange(o.getStreamId(), o.getStartOffset(), o.getEndOffset()))); Map> sortedStreamOffsetRanges = compactedStreamOffsetRanges.stream() - .collect(Collectors.groupingBy(StreamOffsetRange::getStreamId)); + .collect(Collectors.groupingBy(StreamOffsetRange::getStreamId)); sortedStreamOffsetRanges.replaceAll((k, v) -> sortAndMerge(v)); for (long objectId : request.getCompactedObjectIds()) { S3ObjectMetadata metadata = objectMetadataMap.get(objectId); @@ -507,7 +513,7 @@ boolean isSanityCheckFailed(List streamMetadataList, List= compactedStreamOffsetRange.getStartOffset() - && streamOffsetRange.getEndOffset() <= compactedStreamOffsetRange.getEndOffset()) { + && streamOffsetRange.getEndOffset() <= compactedStreamOffsetRange.getEndOffset()) { contained = true; break; } @@ -529,7 +535,7 @@ private List sortAndMerge(List streamOffse long streamId = streamOffsetRangeList.get(0).getStreamId(); Collections.sort(streamOffsetRangeList); List mergedList = new ArrayList<>(); - long start = -1L; + long start = -1L; long end = -1L; for (int i = 0; i < streamOffsetRangeList.size() - 1; i++) { StreamOffsetRange curr = streamOffsetRangeList.get(i); @@ -551,25 +557,26 @@ private List sortAndMerge(List streamOffse Map> convertS3Objects(List streamSetObjectMetadata) { return new HashMap<>(streamSetObjectMetadata.stream() - .collect(Collectors.partitioningBy(e -> (System.currentTimeMillis() - e.dataTimeInMs()) - >= TimeUnit.MINUTES.toMillis(this.forceSplitObjectPeriod)))); + .collect(Collectors.partitioningBy(e -> (System.currentTimeMillis() - e.dataTimeInMs()) + >= TimeUnit.MINUTES.toMillis(this.forceSplitObjectPeriod)))); } - void executeCompactionPlans(CommitStreamSetObjectRequest request, List compactionPlans, List s3ObjectMetadata) - throws CompletionException { + void executeCompactionPlans(CommitStreamSetObjectRequest request, List compactionPlans, + List s3ObjectMetadata) + throws CompletionException { if (compactionPlans.isEmpty()) { return; } Map s3ObjectMetadataMap = s3ObjectMetadata.stream() - .collect(Collectors.toMap(S3ObjectMetadata::objectId, e -> e)); + .collect(Collectors.toMap(S3ObjectMetadata::objectId, e -> e)); List sortedStreamDataBlocks = new ArrayList<>(); for (int i = 0; i < compactionPlans.size(); i++) { // iterate over each compaction plan CompactionPlan compactionPlan = compactionPlans.get(i); long totalSize = compactionPlan.streamDataBlocksMap().values().stream().flatMap(List::stream) - .mapToLong(StreamDataBlock::getBlockSize).sum(); + .mapToLong(StreamDataBlock::getBlockSize).sum(); logger.info("Compaction progress {}/{}, read from {} stream set objects, total size: {}", i + 1, compactionPlans.size(), - compactionPlan.streamDataBlocksMap().size(), totalSize); + compactionPlan.streamDataBlocksMap().size(), totalSize); for (Map.Entry> streamDataBlocEntry : compactionPlan.streamDataBlocksMap().entrySet()) { S3ObjectMetadata metadata = s3ObjectMetadataMap.get(streamDataBlocEntry.getKey()); List streamDataBlocks = streamDataBlocEntry.getValue(); @@ -592,16 +599,16 @@ void executeCompactionPlans(CommitStreamSetObjectRequest request, List uploader.forceUploadStreamSetObject()) - .exceptionally(ex -> { - logger.error("Error while uploading compaction objects", ex); - uploader.release().thenAccept(v -> { - for (CompactedObject compactedObject : compactionPlan.compactedObjects()) { - compactedObject.streamDataBlocks().forEach(StreamDataBlock::release); - } - }).join(); - throw new IllegalStateException("Error while uploading compaction objects", ex); + .thenAccept(v -> uploader.forceUploadStreamSetObject()) + .exceptionally(ex -> { + logger.error("Error while uploading compaction objects", ex); + uploader.release().thenAccept(v -> { + for (CompactedObject compactedObject : compactionPlan.compactedObjects()) { + compactedObject.streamDataBlocks().forEach(StreamDataBlock::release); + } }).join(); + throw new IllegalStateException("Error while uploading compaction objects", ex); + }).join(); streamObjectCfList.stream().map(CompletableFuture::join).forEach(request::addStreamObject); } List objectStreamRanges = CompactionUtils.buildObjectStreamRange(sortedStreamDataBlocks); diff --git a/s3stream/src/main/java/com/automq/stream/s3/compact/CompactionPlan.java b/s3stream/src/main/java/com/automq/stream/s3/compact/CompactionPlan.java index 00c7d77ef..56693fd40 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/compact/CompactionPlan.java +++ b/s3stream/src/main/java/com/automq/stream/s3/compact/CompactionPlan.java @@ -17,9 +17,8 @@ package com.automq.stream.s3.compact; -import com.automq.stream.s3.compact.objects.CompactedObject; import com.automq.stream.s3.StreamDataBlock; - +import com.automq.stream.s3.compact.objects.CompactedObject; import java.util.List; import java.util.Map; @@ -28,7 +27,7 @@ public class CompactionPlan { private final Map> streamDataBlocksMap; public CompactionPlan(List compactedObjects, - Map> streamDataBlocksMap) { + Map> streamDataBlocksMap) { this.compactedObjects = compactedObjects; this.streamDataBlocksMap = streamDataBlocksMap; } diff --git a/s3stream/src/main/java/com/automq/stream/s3/compact/CompactionStats.java b/s3stream/src/main/java/com/automq/stream/s3/compact/CompactionStats.java index 2ae6f57db..3c1c4ab5b 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/compact/CompactionStats.java +++ b/s3stream/src/main/java/com/automq/stream/s3/compact/CompactionStats.java @@ -19,7 +19,6 @@ import com.automq.stream.s3.compact.objects.CompactedObjectBuilder; import com.automq.stream.s3.compact.objects.CompactionType; - import java.util.HashMap; import java.util.List; import java.util.Map; diff --git a/s3stream/src/main/java/com/automq/stream/s3/compact/CompactionUploader.java b/s3stream/src/main/java/com/automq/stream/s3/compact/CompactionUploader.java index 031781005..5177e8423 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/compact/CompactionUploader.java +++ b/s3stream/src/main/java/com/automq/stream/s3/compact/CompactionUploader.java @@ -18,21 +18,20 @@ package com.automq.stream.s3.compact; import com.automq.stream.s3.Config; +import com.automq.stream.s3.StreamDataBlock; import com.automq.stream.s3.compact.objects.CompactedObject; import com.automq.stream.s3.compact.objects.CompactionType; -import com.automq.stream.s3.StreamDataBlock; import com.automq.stream.s3.compact.operator.DataBlockWriter; import com.automq.stream.s3.objects.ObjectManager; import com.automq.stream.s3.objects.StreamObject; import com.automq.stream.s3.operator.S3Operator; import com.automq.stream.utils.ThreadUtils; import com.automq.stream.utils.Threads; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class CompactionUploader { private final static Logger LOGGER = LoggerFactory.getLogger(CompactionUploader.class); @@ -50,9 +49,9 @@ public CompactionUploader(ObjectManager objectManager, S3Operator s3Operator, Co this.s3Operator = s3Operator; this.config = config; this.streamObjectUploadPool = Threads.newFixedThreadPool(config.streamSetObjectCompactionUploadConcurrency(), - ThreadUtils.createThreadFactory("compaction-stream-object-uploader-%d", true), LOGGER); + ThreadUtils.createThreadFactory("compaction-stream-object-uploader-%d", true), LOGGER); this.streamSetObjectUploadPool = Threads.newSingleThreadScheduledExecutor( - ThreadUtils.createThreadFactory("compaction-stream-set-object-uploader-%d", true), LOGGER); + ThreadUtils.createThreadFactory("compaction-stream-set-object-uploader-%d", true), LOGGER); } public void stop() { @@ -60,7 +59,8 @@ public void stop() { this.streamObjectUploadPool.shutdown(); } - public CompletableFuture chainWriteStreamSetObject(CompletableFuture prev, CompactedObject compactedObject) { + public CompletableFuture chainWriteStreamSetObject(CompletableFuture prev, + CompactedObject compactedObject) { if (compactedObject.type() != CompactionType.COMPACT) { return CompletableFuture.failedFuture(new IllegalArgumentException("wrong compacted object type, expected COMPACT")); } @@ -93,30 +93,30 @@ public CompletableFuture writeStreamObject(CompactedObject compact return CompletableFuture.completedFuture(null); } return objectManager.prepareObject(1, TimeUnit.MINUTES.toMillis(CompactionConstants.S3_OBJECT_TTL_MINUTES)) - .thenComposeAsync(objectId -> { - if (isAborted) { - // release data that has not been uploaded + .thenComposeAsync(objectId -> { + if (isAborted) { + // release data that has not been uploaded + compactedObject.streamDataBlocks().forEach(StreamDataBlock::release); + return CompletableFuture.completedFuture(null); + } + DataBlockWriter dataBlockWriter = new DataBlockWriter(objectId, s3Operator, config.objectPartSize()); + CompletableFuture cf = CompactionUtils.chainWriteDataBlock(dataBlockWriter, compactedObject.streamDataBlocks(), streamObjectUploadPool); + return cf.thenCompose(nil -> dataBlockWriter.close()).thenApply(nil -> { + StreamObject streamObject = new StreamObject(); + streamObject.setObjectId(objectId); + streamObject.setStreamId(compactedObject.streamDataBlocks().get(0).getStreamId()); + streamObject.setStartOffset(compactedObject.streamDataBlocks().get(0).getStartOffset()); + streamObject.setEndOffset(compactedObject.streamDataBlocks().get(compactedObject.streamDataBlocks().size() - 1).getEndOffset()); + streamObject.setObjectSize(dataBlockWriter.size()); + return streamObject; + }).whenComplete((ret, ex) -> { + if (ex != null) { + LOGGER.error("write to stream object {} failed", objectId, ex); + dataBlockWriter.release(); compactedObject.streamDataBlocks().forEach(StreamDataBlock::release); - return CompletableFuture.completedFuture(null); } - DataBlockWriter dataBlockWriter = new DataBlockWriter(objectId, s3Operator, config.objectPartSize()); - CompletableFuture cf = CompactionUtils.chainWriteDataBlock(dataBlockWriter, compactedObject.streamDataBlocks(), streamObjectUploadPool); - return cf.thenCompose(nil -> dataBlockWriter.close()).thenApply(nil -> { - StreamObject streamObject = new StreamObject(); - streamObject.setObjectId(objectId); - streamObject.setStreamId(compactedObject.streamDataBlocks().get(0).getStreamId()); - streamObject.setStartOffset(compactedObject.streamDataBlocks().get(0).getStartOffset()); - streamObject.setEndOffset(compactedObject.streamDataBlocks().get(compactedObject.streamDataBlocks().size() - 1).getEndOffset()); - streamObject.setObjectSize(dataBlockWriter.size()); - return streamObject; - }).whenComplete((ret, ex) -> { - if (ex != null) { - LOGGER.error("write to stream object {} failed", objectId, ex); - dataBlockWriter.release(); - compactedObject.streamDataBlocks().forEach(StreamDataBlock::release); - } - }); - }, streamObjectUploadPool); + }); + }, streamObjectUploadPool); } public CompletableFuture forceUploadStreamSetObject() { diff --git a/s3stream/src/main/java/com/automq/stream/s3/compact/CompactionUtils.java b/s3stream/src/main/java/com/automq/stream/s3/compact/CompactionUtils.java index 078640664..825a97421 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/compact/CompactionUtils.java +++ b/s3stream/src/main/java/com/automq/stream/s3/compact/CompactionUtils.java @@ -43,7 +43,7 @@ public static List buildObjectStreamRange(List buildObjectStreamRange(List buildObjectStreamRange(List> blockWaitObjectIndices(List streamMetadataList, - List objectMetadataList, - S3Operator s3Operator) { + List objectMetadataList, + S3Operator s3Operator) { return blockWaitObjectIndices(streamMetadataList, objectMetadataList, s3Operator, null); } public static Map> blockWaitObjectIndices(List streamMetadataList, - List objectMetadataList, - S3Operator s3Operator, - Logger logger) { + List objectMetadataList, + S3Operator s3Operator, + Logger logger) { Map streamMetadataMap = streamMetadataList.stream() - .collect(Collectors.toMap(StreamMetadata::streamId, s -> s)); + .collect(Collectors.toMap(StreamMetadata::streamId, s -> s)); Map>> objectStreamRangePositionFutures = new HashMap<>(); for (S3ObjectMetadata objectMetadata : objectMetadataList) { DataBlockReader dataBlockReader = new DataBlockReader(objectMetadata, s3Operator); @@ -80,33 +80,33 @@ public static Map> blockWaitObjectIndices(List { - try { - List streamDataBlocks = f.getValue().join(); - List validStreamDataBlocks = new ArrayList<>(); - // filter out invalid stream data blocks in case metadata is inconsistent with S3 index block - for (StreamDataBlock streamDataBlock : streamDataBlocks) { - if (!streamMetadataMap.containsKey(streamDataBlock.getStreamId())) { - // non-exist stream - continue; - } - if (streamDataBlock.getEndOffset() <= streamMetadataMap.get(streamDataBlock.getStreamId()).startOffset()) { - // trimmed stream data block - continue; - } - validStreamDataBlocks.add(streamDataBlock); + .map(f -> { + try { + List streamDataBlocks = f.getValue().join(); + List validStreamDataBlocks = new ArrayList<>(); + // filter out invalid stream data blocks in case metadata is inconsistent with S3 index block + for (StreamDataBlock streamDataBlock : streamDataBlocks) { + if (!streamMetadataMap.containsKey(streamDataBlock.getStreamId())) { + // non-exist stream + continue; } - return new AbstractMap.SimpleEntry<>(f.getKey(), validStreamDataBlocks); - } catch (Exception ex) { - // continue compaction without invalid object - if (logger != null) { - logger.warn("failed to get data block index for object {}", f.getKey(), ex); + if (streamDataBlock.getEndOffset() <= streamMetadataMap.get(streamDataBlock.getStreamId()).startOffset()) { + // trimmed stream data block + continue; } - return null; + validStreamDataBlocks.add(streamDataBlock); } - }) - .filter(Objects::nonNull) - .collect(Collectors.toMap(AbstractMap.SimpleEntry::getKey, AbstractMap.SimpleEntry::getValue)); + return new AbstractMap.SimpleEntry<>(f.getKey(), validStreamDataBlocks); + } catch (Exception ex) { + // continue compaction without invalid object + if (logger != null) { + logger.warn("failed to get data block index for object {}", f.getKey(), ex); + } + return null; + } + }) + .filter(Objects::nonNull) + .collect(Collectors.toMap(AbstractMap.SimpleEntry::getKey, AbstractMap.SimpleEntry::getValue)); } /** @@ -121,7 +121,7 @@ public static List> groupStreamDataBlocks(List chainWriteDataBlock(DataBlockWriter dataBlockWriter, List streamDataBlocks, ExecutorService executorService) { + public static CompletableFuture chainWriteDataBlock(DataBlockWriter dataBlockWriter, + List streamDataBlocks, ExecutorService executorService) { CompletableFuture cf = null; for (StreamDataBlock streamDataBlock : streamDataBlocks) { if (cf == null) { diff --git a/s3stream/src/main/java/com/automq/stream/s3/compact/objects/CompactedObject.java b/s3stream/src/main/java/com/automq/stream/s3/compact/objects/CompactedObject.java index 1bf17e408..bb9c07a9e 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/compact/objects/CompactedObject.java +++ b/s3stream/src/main/java/com/automq/stream/s3/compact/objects/CompactedObject.java @@ -18,7 +18,6 @@ package com.automq.stream.s3.compact.objects; import com.automq.stream.s3.StreamDataBlock; - import java.util.List; public class CompactedObject { diff --git a/s3stream/src/main/java/com/automq/stream/s3/compact/objects/CompactedObjectBuilder.java b/s3stream/src/main/java/com/automq/stream/s3/compact/objects/CompactedObjectBuilder.java index 08ab010a3..d758eda7e 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/compact/objects/CompactedObjectBuilder.java +++ b/s3stream/src/main/java/com/automq/stream/s3/compact/objects/CompactedObjectBuilder.java @@ -18,15 +18,14 @@ package com.automq.stream.s3.compact.objects; import com.automq.stream.s3.StreamDataBlock; - import java.util.ArrayList; import java.util.List; import java.util.Set; import java.util.stream.Collectors; public class CompactedObjectBuilder { - private CompactionType type; private final List streamDataBlocks; + private CompactionType type; private int currStreamIndexHead; private int currStreamIndexTail; diff --git a/s3stream/src/main/java/com/automq/stream/s3/compact/operator/DataBlockReader.java b/s3stream/src/main/java/com/automq/stream/s3/compact/operator/DataBlockReader.java index b595ca1f8..470846f29 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/compact/operator/DataBlockReader.java +++ b/s3stream/src/main/java/com/automq/stream/s3/compact/operator/DataBlockReader.java @@ -27,9 +27,6 @@ import io.github.bucket4j.Bucket; import io.netty.buffer.ByteBuf; import io.netty.buffer.CompositeByteBuf; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -37,6 +34,8 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ScheduledExecutorService; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; //TODO: refactor to reduce duplicate code with ObjectWriter public class DataBlockReader { @@ -52,7 +51,8 @@ public DataBlockReader(S3ObjectMetadata metadata, S3Operator s3Operator) { this(metadata, s3Operator, null, null); } - public DataBlockReader(S3ObjectMetadata metadata, S3Operator s3Operator, Bucket throttleBucket, ScheduledExecutorService bucketCallbackExecutor) { + public DataBlockReader(S3ObjectMetadata metadata, S3Operator s3Operator, Bucket throttleBucket, + ScheduledExecutorService bucketCallbackExecutor) { this.metadata = metadata; this.objectKey = metadata.key(); this.s3Operator = s3Operator; @@ -74,18 +74,18 @@ public void parseDataBlockIndex() { public void parseDataBlockIndex(long startPosition) { s3Operator.rangeRead(objectKey, startPosition, metadata.objectSize(), ThrottleStrategy.THROTTLE_2) - .thenAccept(buf -> { - try { - indexBlockCf.complete(IndexBlock.parse(buf, metadata.objectSize(), metadata.objectId())); - } catch (IndexBlockParseException ex) { - parseDataBlockIndex(ex.indexBlockPosition); - } - }).exceptionally(ex -> { - // unrecoverable error, possibly read on a deleted object - LOGGER.warn("s3 range read from {} [{}, {}) failed", objectKey, startPosition, metadata.objectSize(), ex); - indexBlockCf.completeExceptionally(ex); - return null; - }); + .thenAccept(buf -> { + try { + indexBlockCf.complete(IndexBlock.parse(buf, metadata.objectSize(), metadata.objectId())); + } catch (IndexBlockParseException ex) { + parseDataBlockIndex(ex.indexBlockPosition); + } + }).exceptionally(ex -> { + // unrecoverable error, possibly read on a deleted object + LOGGER.warn("s3 range read from {} [{}, {}) failed", objectKey, startPosition, metadata.objectSize(), ex); + indexBlockCf.completeExceptionally(ex); + return null; + }); } @@ -148,18 +148,18 @@ public void readContinuousBlocks(List streamDataBlocks, long ma final int iterations = cnt; final int finalEnd = end + 1; // include current block CompletableFuture.allOf(cfList.toArray(new CompletableFuture[0])) - .thenAccept(v -> { - CompositeByteBuf compositeByteBuf = DirectByteBufAlloc.compositeByteBuffer(); - for (int j = 0; j < iterations; j++) { - compositeByteBuf.addComponent(true, bufferMap.get(j)); - } - parseDataBlocks(compositeByteBuf, streamDataBlocks.subList(finalStart, finalEnd)); - }) - .exceptionally(ex -> { - LOGGER.error("read data from object {} failed", objectId, ex); - failDataBlocks(streamDataBlocks, ex); - return null; - }); + .thenAccept(v -> { + CompositeByteBuf compositeByteBuf = DirectByteBufAlloc.compositeByteBuffer(); + for (int j = 0; j < iterations; j++) { + compositeByteBuf.addComponent(true, bufferMap.get(j)); + } + parseDataBlocks(compositeByteBuf, streamDataBlocks.subList(finalStart, finalEnd)); + }) + .exceptionally(ex -> { + LOGGER.error("read data from object {} failed", objectId, ex); + failDataBlocks(streamDataBlocks, ex); + return null; + }); end++; } else { // read before current block @@ -178,18 +178,18 @@ public void readContinuousBlocks(List streamDataBlocks, long ma private void readContinuousBlocks0(List streamDataBlocks) { rangeRead(streamDataBlocks.get(0).getBlockStartPosition(), - streamDataBlocks.get(streamDataBlocks.size() - 1).getBlockEndPosition()) - .thenAccept(buf -> parseDataBlocks(buf, streamDataBlocks)) - .exceptionally(ex -> { - LOGGER.error("read data from object {} failed", metadata.objectId(), ex); - failDataBlocks(streamDataBlocks, ex); - return null; - }); + streamDataBlocks.get(streamDataBlocks.size() - 1).getBlockEndPosition()) + .thenAccept(buf -> parseDataBlocks(buf, streamDataBlocks)) + .exceptionally(ex -> { + LOGGER.error("read data from object {} failed", metadata.objectId(), ex); + failDataBlocks(streamDataBlocks, ex); + return null; + }); } private CompletableFuture rangeRead(long start, long end) { return rangeRead0(start, end).whenComplete((ret, ex) -> - S3StreamMetricsManager.recordCompactionReadSizeIn(MetricsLevel.INFO, ret.readableBytes())); + S3StreamMetricsManager.recordCompactionReadSizeIn(MetricsLevel.INFO, ret.readableBytes())); } private CompletableFuture rangeRead0(long start, long end) { @@ -203,14 +203,14 @@ private CompletableFuture rangeRead0(long start, long end) { }); } else { return throttleBucket.asScheduler().consume(end - start + 1, bucketCallbackExecutor) - .thenCompose(v -> - s3Operator.rangeRead(objectKey, start, end, ThrottleStrategy.THROTTLE_2).thenApply(buf -> { - // convert heap buffer to direct buffer - ByteBuf directBuf = DirectByteBufAlloc.byteBuffer(buf.readableBytes()); - directBuf.writeBytes(buf); - buf.release(); - return directBuf; - })); + .thenCompose(v -> + s3Operator.rangeRead(objectKey, start, end, ThrottleStrategy.THROTTLE_2).thenApply(buf -> { + // convert heap buffer to direct buffer + ByteBuf directBuf = DirectByteBufAlloc.byteBuffer(buf.readableBytes()); + directBuf.writeBytes(buf); + buf.release(); + return directBuf; + })); } } @@ -231,7 +231,8 @@ private void failDataBlocks(List streamDataBlocks, Throwable ex } static class IndexBlock { - static List parse(ByteBuf objectTailBuf, long objectSize, long objectId) throws IndexBlockParseException { + static List parse(ByteBuf objectTailBuf, long objectSize, + long objectId) throws IndexBlockParseException { try { long indexBlockPosition = objectTailBuf.getLong(objectTailBuf.readableBytes() - 48); int indexBlockSize = objectTailBuf.getInt(objectTailBuf.readableBytes() - 40); @@ -254,7 +255,7 @@ static List parse(ByteBuf objectTailBuf, long objectSize, long int rangeSize = streamRanges.readInt(); int blockIndex = streamRanges.readInt(); streamDataBlocks.add(new StreamDataBlock(streamId, startOffset, startOffset + rangeSize, - blockIndex, objectId, blockPosition, blockSize, recordCount)); + blockIndex, objectId, blockPosition, blockSize, recordCount)); } return streamDataBlocks; } diff --git a/s3stream/src/main/java/com/automq/stream/s3/compact/operator/DataBlockWriter.java b/s3stream/src/main/java/com/automq/stream/s3/compact/operator/DataBlockWriter.java index a4c230648..aabf9595a 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/compact/operator/DataBlockWriter.java +++ b/s3stream/src/main/java/com/automq/stream/s3/compact/operator/DataBlockWriter.java @@ -27,7 +27,6 @@ import com.automq.stream.s3.operator.Writer; import io.netty.buffer.ByteBuf; import io.netty.buffer.CompositeByteBuf; - import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -42,9 +41,9 @@ public class DataBlockWriter { private final List waitingUploadBlocks; private final Map> waitingUploadBlockCfs; private final List completedBlocks; - private IndexBlock indexBlock; private final Writer writer; private final long objectId; + private IndexBlock indexBlock; private long nextDataBlockPosition; private long size; diff --git a/s3stream/src/main/java/com/automq/stream/s3/failover/DefaultServerless.java b/s3stream/src/main/java/com/automq/stream/s3/failover/DefaultServerless.java index dc6e15535..2328c93ac 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/failover/DefaultServerless.java +++ b/s3stream/src/main/java/com/automq/stream/s3/failover/DefaultServerless.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.annotation.JsonIgnoreProperties; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; - import java.util.Arrays; import java.util.List; import java.util.concurrent.ExecutionException; @@ -31,9 +30,24 @@ public class DefaultServerless implements Serverless { private static final String SERVERLESS_CMD = "/opt/automq/scripts/amq-serverless"; + private static void check(String[] cmd, CommandResult rst) throws ExecutionException { + if (rst.code() != 0) { + throw new ExecutionException("Run " + Arrays.toString(cmd) + ", code:" + rst.code() + " failed: " + rst.stderr(), null); + } + } + + private static T jsonParse(String raw, Class clazz) throws ExecutionException { + ObjectMapper mapper = new ObjectMapper(); + try { + return mapper.readValue(raw, clazz); + } catch (JsonProcessingException e) { + throw new ExecutionException("json parse (" + raw + ") fail", e); + } + } + @Override public String attach(String volumeId, int nodeId) throws ExecutionException { - String[] cmd = new String[]{SERVERLESS_CMD, "volume", "attach", "-v", volumeId, "-n", Integer.toString(nodeId)}; + String[] cmd = new String[] {SERVERLESS_CMD, "volume", "attach", "-v", volumeId, "-n", Integer.toString(nodeId)}; CommandResult result = CommandUtils.run(cmd); check(cmd, result); return jsonParse(result.stdout(), AttachResult.class).getDeviceName(); @@ -41,21 +55,21 @@ public String attach(String volumeId, int nodeId) throws ExecutionException { @Override public void delete(String volumeId) throws ExecutionException { - String[] cmd = new String[]{SERVERLESS_CMD, "volume", "delete", "-v", volumeId}; + String[] cmd = new String[] {SERVERLESS_CMD, "volume", "delete", "-v", volumeId}; CommandResult result = CommandUtils.run(cmd); check(cmd, result); } @Override public void fence(String volumeId) throws ExecutionException { - String[] cmd = new String[]{SERVERLESS_CMD, "volume", "fence", "-v", volumeId}; + String[] cmd = new String[] {SERVERLESS_CMD, "volume", "fence", "-v", volumeId}; CommandResult result = CommandUtils.run(cmd); check(cmd, result); } @Override public List scan() throws ExecutionException { - String[] cmd = new String[]{SERVERLESS_CMD, "volume", "queryFailover"}; + String[] cmd = new String[] {SERVERLESS_CMD, "volume", "queryFailover"}; CommandResult result = CommandUtils.run(cmd); check(cmd, result); QueryFailedNode[] nodes = jsonParse(result.stdout(), QueryFailedNode[].class); @@ -67,21 +81,6 @@ public List scan() throws ExecutionException { }).collect(Collectors.toList()); } - private static void check(String[] cmd, CommandResult rst) throws ExecutionException { - if (rst.code() != 0) { - throw new ExecutionException("Run " + Arrays.toString(cmd) + ", code:" + rst.code() + " failed: " + rst.stderr(), null); - } - } - - private static T jsonParse(String raw, Class clazz) throws ExecutionException { - ObjectMapper mapper = new ObjectMapper(); - try { - return mapper.readValue(raw, clazz); - } catch (JsonProcessingException e) { - throw new ExecutionException("json parse (" + raw + ") fail", e); - } - } - @JsonIgnoreProperties(ignoreUnknown = true) static class AttachResult { private String deviceName; diff --git a/s3stream/src/main/java/com/automq/stream/s3/failover/Failover.java b/s3stream/src/main/java/com/automq/stream/s3/failover/Failover.java index 42d1d2fa2..9dde15cb9 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/failover/Failover.java +++ b/s3stream/src/main/java/com/automq/stream/s3/failover/Failover.java @@ -26,11 +26,10 @@ import com.automq.stream.utils.LogContext; import com.automq.stream.utils.ThreadUtils; import com.automq.stream.utils.Threads; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static com.automq.stream.s3.Constants.NOOP_EPOCH; import static com.automq.stream.s3.Constants.NOOP_NODE_ID; @@ -118,10 +117,10 @@ public FailoverResponse failover() throws Throwable { @Override public String toString() { return "FailoverTask{" + - "request=" + request + - ", nodeId=" + nodeId + - ", epoch=" + epoch + - '}'; + "request=" + request + + ", nodeId=" + nodeId + + ", epoch=" + epoch + + '}'; } } diff --git a/s3stream/src/main/java/com/automq/stream/s3/failover/FailoverRequest.java b/s3stream/src/main/java/com/automq/stream/s3/failover/FailoverRequest.java index 41e3c1582..8b11a7656 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/failover/FailoverRequest.java +++ b/s3stream/src/main/java/com/automq/stream/s3/failover/FailoverRequest.java @@ -22,14 +22,14 @@ public class FailoverRequest { private String volumeId; private String device; - public void setNodeId(int nodeId) { - this.nodeId = nodeId; - } - public int getNodeId() { return nodeId; } + public void setNodeId(int nodeId) { + this.nodeId = nodeId; + } + public String getVolumeId() { return volumeId; } @@ -49,9 +49,9 @@ public void setDevice(String device) { @Override public String toString() { return "FailoverRequest{" + - "nodeId=" + nodeId + - ", volumeId='" + volumeId + '\'' + - ", device='" + device + '\'' + - '}'; + "nodeId=" + nodeId + + ", volumeId='" + volumeId + '\'' + + ", device='" + device + '\'' + + '}'; } } diff --git a/s3stream/src/main/java/com/automq/stream/s3/failover/Serverless.java b/s3stream/src/main/java/com/automq/stream/s3/failover/Serverless.java index bc1a45ca6..180caac30 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/failover/Serverless.java +++ b/s3stream/src/main/java/com/automq/stream/s3/failover/Serverless.java @@ -75,11 +75,10 @@ public void setVolumeId(String volumeId) { @Override public String toString() { return "FailedNode{" + - "nodeId=" + nodeId + - ", volumeId='" + volumeId + '\'' + - '}'; + "nodeId=" + nodeId + + ", volumeId='" + volumeId + '\'' + + '}'; } } - } diff --git a/s3stream/src/main/java/com/automq/stream/s3/memory/MemoryMetadataManager.java b/s3stream/src/main/java/com/automq/stream/s3/memory/MemoryMetadataManager.java index 224aa47cb..705c815f3 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/memory/MemoryMetadataManager.java +++ b/s3stream/src/main/java/com/automq/stream/s3/memory/MemoryMetadataManager.java @@ -58,15 +58,15 @@ public static void advanceNodeId() { NODE_ID_ALLOC.getAndIncrement(); } + private static StreamOffsetRange to(ObjectStreamRange s) { + return new StreamOffsetRange(s.getStreamId(), s.getStartOffset(), s.getEndOffset()); + } + @Override public synchronized CompletableFuture prepareObject(int count, long ttl) { return CompletableFuture.completedFuture(objectIdAlloc.getAndAdd(count)); } - private static StreamOffsetRange to(ObjectStreamRange s) { - return new StreamOffsetRange(s.getStreamId(), s.getStartOffset(), s.getEndOffset()); - } - @Override public synchronized CompletableFuture commitStreamSetObject( CommitStreamSetObjectRequest request) { diff --git a/s3stream/src/main/java/com/automq/stream/s3/metadata/S3ObjectMetadata.java b/s3stream/src/main/java/com/automq/stream/s3/metadata/S3ObjectMetadata.java index 4c6c7dc94..cad694b53 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/metadata/S3ObjectMetadata.java +++ b/s3stream/src/main/java/com/automq/stream/s3/metadata/S3ObjectMetadata.java @@ -17,7 +17,6 @@ package com.automq.stream.s3.metadata; - import java.util.Collections; import java.util.List; import java.util.Objects; @@ -34,7 +33,6 @@ public class S3ObjectMetadata { * */ private final long orderId; - private long objectSize; private final S3ObjectType type; /** * stream offset ranges of the object. @@ -44,40 +42,40 @@ public class S3ObjectMetadata { * */ private final List offsetRanges; - /** - * real committed timestamp of the data in the object. - */ - private long committedTimestamp; - /** * logical timestamp in ms of the data in the object. */ private final long dataTimeInMs; + private long objectSize; + /** + * real committed timestamp of the data in the object. + */ + private long committedTimestamp; // Only used for testing public S3ObjectMetadata(long objectId, long objectSize, S3ObjectType type) { this(objectId, type, Collections.emptyList(), S3StreamConstant.INVALID_TS, S3StreamConstant.INVALID_TS, objectSize, - S3StreamConstant.INVALID_ORDER_ID); + S3StreamConstant.INVALID_ORDER_ID); } public S3ObjectMetadata(long objectId, S3ObjectType type, List offsetRanges, long dataTimeInMs) { this(objectId, type, offsetRanges, dataTimeInMs, S3StreamConstant.INVALID_TS, S3StreamConstant.INVALID_OBJECT_SIZE, - S3StreamConstant.INVALID_ORDER_ID); + S3StreamConstant.INVALID_ORDER_ID); } public S3ObjectMetadata(long objectId, S3ObjectType type, List offsetRanges, long dataTimeInMs, - long orderId) { + long orderId) { this(objectId, type, offsetRanges, dataTimeInMs, S3StreamConstant.INVALID_TS, S3StreamConstant.INVALID_OBJECT_SIZE, - orderId); + orderId); } public S3ObjectMetadata( - // these four params come from S3StreamSetObject or S3StreamObject - long objectId, S3ObjectType type, List offsetRanges, long dataTimeInMs, - // these two params come from S3Object - long committedTimestamp, long objectSize, - // this param only comes from S3StreamSetObject - long orderId) { + // these four params come from S3StreamSetObject or S3StreamObject + long objectId, S3ObjectType type, List offsetRanges, long dataTimeInMs, + // these two params come from S3Object + long committedTimestamp, long objectSize, + // this param only comes from S3StreamSetObject + long orderId) { this.objectId = objectId; this.orderId = orderId; this.objectSize = objectSize; @@ -151,7 +149,7 @@ public boolean intersect(long streamId, long startOffset, long endOffset) { public String toString() { return "S3ObjectMetadata(objectId=" + objectId + ", objectSize=" + objectSize + ", type=" + type + ", offsetRanges=" + offsetRanges - + ", committedTimestamp=" + committedTimestamp + ", dataTimestamp=" + dataTimeInMs + ")"; + + ", committedTimestamp=" + committedTimestamp + ", dataTimestamp=" + dataTimeInMs + ")"; } public String key() { @@ -168,7 +166,7 @@ public boolean equals(Object o) { } S3ObjectMetadata that = (S3ObjectMetadata) o; return objectId == that.objectId && orderId == that.orderId && objectSize == that.objectSize && committedTimestamp == that.committedTimestamp - && dataTimeInMs == that.dataTimeInMs && type == that.type && offsetRanges.equals(that.offsetRanges); + && dataTimeInMs == that.dataTimeInMs && type == that.type && offsetRanges.equals(that.offsetRanges); } @Override diff --git a/s3stream/src/main/java/com/automq/stream/s3/metadata/StreamMetadata.java b/s3stream/src/main/java/com/automq/stream/s3/metadata/StreamMetadata.java index 064f88a48..bdfc2197d 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/metadata/StreamMetadata.java +++ b/s3stream/src/main/java/com/automq/stream/s3/metadata/StreamMetadata.java @@ -79,11 +79,11 @@ public void state(StreamState state) { @Override public String toString() { return "StreamMetadata{" + - "streamId=" + streamId + - ", epoch=" + epoch + - ", startOffset=" + startOffset + - ", endOffset=" + endOffset + - ", state=" + state + - '}'; + "streamId=" + streamId + + ", epoch=" + epoch + + ", startOffset=" + startOffset + + ", endOffset=" + endOffset + + ", state=" + state + + '}'; } } diff --git a/s3stream/src/main/java/com/automq/stream/s3/metadata/StreamOffsetRange.java b/s3stream/src/main/java/com/automq/stream/s3/metadata/StreamOffsetRange.java index 135d7fe9a..21edc4316 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/metadata/StreamOffsetRange.java +++ b/s3stream/src/main/java/com/automq/stream/s3/metadata/StreamOffsetRange.java @@ -53,8 +53,8 @@ public long getEndOffset() { public boolean intersect(long startOffset, long endOffset) { return startOffset <= endOffset - && startOffset >= this.startOffset && startOffset <= this.endOffset - && endOffset <= this.endOffset; + && startOffset >= this.startOffset && startOffset <= this.endOffset + && endOffset <= this.endOffset; } @Override diff --git a/s3stream/src/main/java/com/automq/stream/s3/metadata/StreamState.java b/s3stream/src/main/java/com/automq/stream/s3/metadata/StreamState.java index 32502816e..3860b8b78 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/metadata/StreamState.java +++ b/s3stream/src/main/java/com/automq/stream/s3/metadata/StreamState.java @@ -21,11 +21,11 @@ public enum StreamState { CLOSED, OPENED; - public byte toByte() { - return (byte) ordinal(); - } - public static StreamState fromByte(byte b) { return values()[b]; } + + public byte toByte() { + return (byte) ordinal(); + } } diff --git a/s3stream/src/main/java/com/automq/stream/s3/metrics/AttributesCache.java b/s3stream/src/main/java/com/automq/stream/s3/metrics/AttributesCache.java index 29437de0a..dff5ab2a1 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/metrics/AttributesCache.java +++ b/s3stream/src/main/java/com/automq/stream/s3/metrics/AttributesCache.java @@ -22,7 +22,6 @@ import com.automq.stream.s3.metrics.operations.S3Stage; import io.opentelemetry.api.common.Attributes; import io.opentelemetry.api.common.AttributesBuilder; - import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -76,9 +75,9 @@ String getObjectBucketLabel(long objectSize) { private Attributes buildAttributes(S3Operation operation, String sizeLabelName, boolean isSuccess) { AttributesBuilder attributesBuilder = defaultAttributes().toBuilder() - .put(S3StreamMetricsConstant.LABEL_OPERATION_TYPE, operation.getType().getName()) - .put(S3StreamMetricsConstant.LABEL_OPERATION_NAME, operation.getName()) - .put(S3StreamMetricsConstant.LABEL_STATUS, isSuccess ? "success" : "failed"); + .put(S3StreamMetricsConstant.LABEL_OPERATION_TYPE, operation.getType().getName()) + .put(S3StreamMetricsConstant.LABEL_OPERATION_NAME, operation.getName()) + .put(S3StreamMetricsConstant.LABEL_STATUS, isSuccess ? "success" : "failed"); if (operation == S3Operation.GET_OBJECT || operation == S3Operation.PUT_OBJECT || operation == S3Operation.UPLOAD_PART) { attributesBuilder.put(S3StreamMetricsConstant.LABEL_SIZE_NAME, sizeLabelName); } @@ -87,30 +86,30 @@ private Attributes buildAttributes(S3Operation operation, String sizeLabelName, private Attributes buildAttributes(S3Stage stage) { return defaultAttributes().toBuilder() - .put(S3StreamMetricsConstant.LABEL_OPERATION_TYPE, stage.getOperation().getType().getName()) - .put(S3StreamMetricsConstant.LABEL_OPERATION_NAME, stage.getOperation().getName()) - .put(S3StreamMetricsConstant.LABEL_STAGE, stage.getName()) - .build(); + .put(S3StreamMetricsConstant.LABEL_OPERATION_TYPE, stage.getOperation().getType().getName()) + .put(S3StreamMetricsConstant.LABEL_OPERATION_NAME, stage.getOperation().getName()) + .put(S3StreamMetricsConstant.LABEL_STAGE, stage.getName()) + .build(); } private Attributes buildAttributes(S3Operation operation, String status) { return defaultAttributes().toBuilder() - .put(S3StreamMetricsConstant.LABEL_OPERATION_TYPE, operation.getType().getName()) - .put(S3StreamMetricsConstant.LABEL_OPERATION_NAME, operation.getName()) - .put(S3StreamMetricsConstant.LABEL_STATUS, status) - .build(); + .put(S3StreamMetricsConstant.LABEL_OPERATION_TYPE, operation.getType().getName()) + .put(S3StreamMetricsConstant.LABEL_OPERATION_NAME, operation.getName()) + .put(S3StreamMetricsConstant.LABEL_STATUS, status) + .build(); } private Attributes buildAttributes(S3ObjectStage stage) { return defaultAttributes().toBuilder() - .put(S3StreamMetricsConstant.LABEL_STAGE, stage.getName()) - .build(); + .put(S3StreamMetricsConstant.LABEL_STAGE, stage.getName()) + .build(); } private Attributes buildAttributes(String source) { return defaultAttributes().toBuilder() - .put(S3StreamMetricsConstant.LABEL_ALLOCATE_BYTE_BUF_SOURCE, source) - .build(); + .put(S3StreamMetricsConstant.LABEL_ALLOCATE_BYTE_BUF_SOURCE, source) + .build(); } } diff --git a/s3stream/src/main/java/com/automq/stream/s3/metrics/S3StreamMetricsConstant.java b/s3stream/src/main/java/com/automq/stream/s3/metrics/S3StreamMetricsConstant.java index beb625dbe..cb524aa64 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/metrics/S3StreamMetricsConstant.java +++ b/s3stream/src/main/java/com/automq/stream/s3/metrics/S3StreamMetricsConstant.java @@ -18,55 +18,55 @@ package com.automq.stream.s3.metrics; import io.opentelemetry.api.common.AttributeKey; - import java.util.List; import java.util.concurrent.TimeUnit; public class S3StreamMetricsConstant { // value = 16KB * 2^i - public static final String[] OBJECT_SIZE_BUCKET_NAMES = {"16KB", - "32KB", - "64KB", - "128KB", - "256KB", - "512KB", - "1MB", - "2MB", - "4MB", - "8MB", - "16MB", - "32MB", - "64MB", - "128MB", - "inf"}; + public static final String[] OBJECT_SIZE_BUCKET_NAMES = { + "16KB", + "32KB", + "64KB", + "128KB", + "256KB", + "512KB", + "1MB", + "2MB", + "4MB", + "8MB", + "16MB", + "32MB", + "64MB", + "128MB", + "inf"}; public static final List LATENCY_BOUNDARIES = List.of( - TimeUnit.MICROSECONDS.toNanos(1), - TimeUnit.MICROSECONDS.toNanos(10), - TimeUnit.MICROSECONDS.toNanos(100), - TimeUnit.MILLISECONDS.toNanos(1), - TimeUnit.MILLISECONDS.toNanos(3), - TimeUnit.MILLISECONDS.toNanos(5), - TimeUnit.MILLISECONDS.toNanos(7), - TimeUnit.MILLISECONDS.toNanos(10), - TimeUnit.MILLISECONDS.toNanos(20), - TimeUnit.MILLISECONDS.toNanos(30), - TimeUnit.MILLISECONDS.toNanos(40), - TimeUnit.MILLISECONDS.toNanos(50), - TimeUnit.MILLISECONDS.toNanos(60), - TimeUnit.MILLISECONDS.toNanos(70), - TimeUnit.MILLISECONDS.toNanos(80), - TimeUnit.MILLISECONDS.toNanos(90), - TimeUnit.MILLISECONDS.toNanos(100), - TimeUnit.MILLISECONDS.toNanos(200), - TimeUnit.MILLISECONDS.toNanos(500), - TimeUnit.SECONDS.toNanos(1), - TimeUnit.SECONDS.toNanos(3), - TimeUnit.SECONDS.toNanos(5), - TimeUnit.SECONDS.toNanos(10), - TimeUnit.SECONDS.toNanos(30), - TimeUnit.MINUTES.toNanos(1), - TimeUnit.MINUTES.toNanos(3), - TimeUnit.MINUTES.toNanos(5) + TimeUnit.MICROSECONDS.toNanos(1), + TimeUnit.MICROSECONDS.toNanos(10), + TimeUnit.MICROSECONDS.toNanos(100), + TimeUnit.MILLISECONDS.toNanos(1), + TimeUnit.MILLISECONDS.toNanos(3), + TimeUnit.MILLISECONDS.toNanos(5), + TimeUnit.MILLISECONDS.toNanos(7), + TimeUnit.MILLISECONDS.toNanos(10), + TimeUnit.MILLISECONDS.toNanos(20), + TimeUnit.MILLISECONDS.toNanos(30), + TimeUnit.MILLISECONDS.toNanos(40), + TimeUnit.MILLISECONDS.toNanos(50), + TimeUnit.MILLISECONDS.toNanos(60), + TimeUnit.MILLISECONDS.toNanos(70), + TimeUnit.MILLISECONDS.toNanos(80), + TimeUnit.MILLISECONDS.toNanos(90), + TimeUnit.MILLISECONDS.toNanos(100), + TimeUnit.MILLISECONDS.toNanos(200), + TimeUnit.MILLISECONDS.toNanos(500), + TimeUnit.SECONDS.toNanos(1), + TimeUnit.SECONDS.toNanos(3), + TimeUnit.SECONDS.toNanos(5), + TimeUnit.SECONDS.toNanos(10), + TimeUnit.SECONDS.toNanos(30), + TimeUnit.MINUTES.toNanos(1), + TimeUnit.MINUTES.toNanos(3), + TimeUnit.MINUTES.toNanos(5) ); public static final String UPLOAD_SIZE_METRIC_NAME = "upload_size_total"; diff --git a/s3stream/src/main/java/com/automq/stream/s3/metrics/S3StreamMetricsManager.java b/s3stream/src/main/java/com/automq/stream/s3/metrics/S3StreamMetricsManager.java index f2477f9ab..da9cdf5cf 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/metrics/S3StreamMetricsManager.java +++ b/s3stream/src/main/java/com/automq/stream/s3/metrics/S3StreamMetricsManager.java @@ -26,7 +26,6 @@ import io.opentelemetry.api.metrics.LongHistogram; import io.opentelemetry.api.metrics.Meter; import io.opentelemetry.api.metrics.ObservableLongGauge; - import java.util.function.Supplier; public class S3StreamMetricsManager { @@ -85,182 +84,182 @@ public static void setMetricsLevel(MetricsLevel level) { public static void initMetrics(Meter meter, String prefix) { s3DownloadSizeInTotal = meter.counterBuilder(prefix + S3StreamMetricsConstant.DOWNLOAD_SIZE_METRIC_NAME) - .setDescription("S3 download size") - .setUnit("bytes") - .build(); + .setDescription("S3 download size") + .setUnit("bytes") + .build(); s3UploadSizeInTotal = meter.counterBuilder(prefix + S3StreamMetricsConstant.UPLOAD_SIZE_METRIC_NAME) - .setDescription("S3 upload size") - .setUnit("bytes") - .build(); + .setDescription("S3 upload size") + .setUnit("bytes") + .build(); operationLatency = meter.histogramBuilder(prefix + S3StreamMetricsConstant.OPERATION_LATENCY_METRIC_NAME) - .setDescription("Operations latency") - .setUnit("nanoseconds") - .ofLongs() - .setExplicitBucketBoundariesAdvice(S3StreamMetricsConstant.LATENCY_BOUNDARIES) - .build(); + .setDescription("Operations latency") + .setUnit("nanoseconds") + .ofLongs() + .setExplicitBucketBoundariesAdvice(S3StreamMetricsConstant.LATENCY_BOUNDARIES) + .build(); objectNumInTotal = meter.counterBuilder(prefix + S3StreamMetricsConstant.OBJECT_COUNT_METRIC_NAME) - .setDescription("Objects count") - .build(); + .setDescription("Objects count") + .build(); objectStageCost = meter.histogramBuilder(prefix + S3StreamMetricsConstant.OBJECT_STAGE_COST_METRIC_NAME) - .setDescription("Objects stage cost") - .setUnit("nanoseconds") - .ofLongs() - .setExplicitBucketBoundariesAdvice(S3StreamMetricsConstant.LATENCY_BOUNDARIES) - .build(); + .setDescription("Objects stage cost") + .setUnit("nanoseconds") + .ofLongs() + .setExplicitBucketBoundariesAdvice(S3StreamMetricsConstant.LATENCY_BOUNDARIES) + .build(); objectUploadSize = meter.histogramBuilder(prefix + S3StreamMetricsConstant.OBJECT_UPLOAD_SIZE_METRIC_NAME) - .setDescription("Objects upload size") - .setUnit("bytes") - .ofLongs() - .build(); + .setDescription("Objects upload size") + .setUnit("bytes") + .ofLongs() + .build(); objectDownloadSize = meter.histogramBuilder(prefix + S3StreamMetricsConstant.OBJECT_DOWNLOAD_SIZE_METRIC_NAME) - .setDescription("Objects download size") - .setUnit("bytes") - .ofLongs() - .build(); + .setDescription("Objects download size") + .setUnit("bytes") + .ofLongs() + .build(); networkInboundUsageInTotal = meter.counterBuilder(prefix + S3StreamMetricsConstant.NETWORK_INBOUND_USAGE_METRIC_NAME) - .setDescription("Network inbound usage") - .setUnit("bytes") - .build(); + .setDescription("Network inbound usage") + .setUnit("bytes") + .build(); networkOutboundUsageInTotal = meter.counterBuilder(prefix + S3StreamMetricsConstant.NETWORK_OUTBOUND_USAGE_METRIC_NAME) - .setDescription("Network outbound usage") - .setUnit("bytes") - .build(); + .setDescription("Network outbound usage") + .setUnit("bytes") + .build(); networkInboundAvailableBandwidth = meter.gaugeBuilder(prefix + S3StreamMetricsConstant.NETWORK_INBOUND_AVAILABLE_BANDWIDTH_METRIC_NAME) - .setDescription("Network inbound available bandwidth") - .setUnit("bytes") - .ofLongs() - .buildWithCallback(result -> { - if (MetricsLevel.INFO.isWithin(metricsLevel)) { - result.record(networkInboundAvailableBandwidthSupplier.get(), AttributesCache.INSTANCE.defaultAttributes()); - } - }); + .setDescription("Network inbound available bandwidth") + .setUnit("bytes") + .ofLongs() + .buildWithCallback(result -> { + if (MetricsLevel.INFO.isWithin(metricsLevel)) { + result.record(networkInboundAvailableBandwidthSupplier.get(), AttributesCache.INSTANCE.defaultAttributes()); + } + }); networkOutboundAvailableBandwidth = meter.gaugeBuilder(prefix + S3StreamMetricsConstant.NETWORK_OUTBOUND_AVAILABLE_BANDWIDTH_METRIC_NAME) - .setDescription("Network outbound available bandwidth") - .setUnit("bytes") - .ofLongs() - .buildWithCallback(result -> { - if (MetricsLevel.INFO.isWithin(metricsLevel)) { - result.record(networkOutboundAvailableBandwidthSupplier.get(), AttributesCache.INSTANCE.defaultAttributes()); - } - }); + .setDescription("Network outbound available bandwidth") + .setUnit("bytes") + .ofLongs() + .buildWithCallback(result -> { + if (MetricsLevel.INFO.isWithin(metricsLevel)) { + result.record(networkOutboundAvailableBandwidthSupplier.get(), AttributesCache.INSTANCE.defaultAttributes()); + } + }); networkInboundLimiterQueueSize = meter.gaugeBuilder(prefix + S3StreamMetricsConstant.NETWORK_INBOUND_LIMITER_QUEUE_SIZE_METRIC_NAME) - .setDescription("Network inbound limiter queue size") - .ofLongs() - .buildWithCallback(result -> { - if (MetricsLevel.DEBUG.isWithin(metricsLevel)) { - result.record((long) networkInboundLimiterQueueSizeSupplier.get(), AttributesCache.INSTANCE.defaultAttributes()); - } - }); + .setDescription("Network inbound limiter queue size") + .ofLongs() + .buildWithCallback(result -> { + if (MetricsLevel.DEBUG.isWithin(metricsLevel)) { + result.record((long) networkInboundLimiterQueueSizeSupplier.get(), AttributesCache.INSTANCE.defaultAttributes()); + } + }); networkOutboundLimiterQueueSize = meter.gaugeBuilder(prefix + S3StreamMetricsConstant.NETWORK_OUTBOUND_LIMITER_QUEUE_SIZE_METRIC_NAME) - .setDescription("Network outbound limiter queue size") - .ofLongs() - .buildWithCallback(result -> { - if (MetricsLevel.DEBUG.isWithin(metricsLevel)) { - result.record((long) networkOutboundLimiterQueueSizeSupplier.get(), AttributesCache.INSTANCE.defaultAttributes()); - } - }); + .setDescription("Network outbound limiter queue size") + .ofLongs() + .buildWithCallback(result -> { + if (MetricsLevel.DEBUG.isWithin(metricsLevel)) { + result.record((long) networkOutboundLimiterQueueSizeSupplier.get(), AttributesCache.INSTANCE.defaultAttributes()); + } + }); networkInboundLimiterQueueTime = meter.histogramBuilder(prefix + S3StreamMetricsConstant.NETWORK_INBOUND_LIMITER_QUEUE_TIME_METRIC_NAME) - .setDescription("Network inbound limiter queue time") - .setUnit("nanoseconds") - .ofLongs() - .setExplicitBucketBoundariesAdvice(S3StreamMetricsConstant.LATENCY_BOUNDARIES) - .build(); + .setDescription("Network inbound limiter queue time") + .setUnit("nanoseconds") + .ofLongs() + .setExplicitBucketBoundariesAdvice(S3StreamMetricsConstant.LATENCY_BOUNDARIES) + .build(); networkOutboundLimiterQueueTime = meter.histogramBuilder(prefix + S3StreamMetricsConstant.NETWORK_OUTBOUND_LIMITER_QUEUE_TIME_METRIC_NAME) - .setDescription("Network outbound limiter queue time") - .setUnit("nanoseconds") - .ofLongs() - .setExplicitBucketBoundariesAdvice(S3StreamMetricsConstant.LATENCY_BOUNDARIES) - .build(); + .setDescription("Network outbound limiter queue time") + .setUnit("nanoseconds") + .ofLongs() + .setExplicitBucketBoundariesAdvice(S3StreamMetricsConstant.LATENCY_BOUNDARIES) + .build(); allocateByteBufSize = meter.histogramBuilder(prefix + S3StreamMetricsConstant.ALLOCATE_BYTE_BUF_SIZE_METRIC_NAME) - .setDescription("Allocate byte buf size") - .setUnit("bytes") - .ofLongs() - .build(); + .setDescription("Allocate byte buf size") + .setUnit("bytes") + .ofLongs() + .build(); readAheadSize = meter.histogramBuilder(prefix + S3StreamMetricsConstant.READ_AHEAD_SIZE_METRIC_NAME) - .setDescription("Read ahead size") - .setUnit("bytes") - .ofLongs() - .build(); + .setDescription("Read ahead size") + .setUnit("bytes") + .ofLongs() + .build(); deltaWalStartOffset = meter.gaugeBuilder(prefix + S3StreamMetricsConstant.WAL_START_OFFSET) - .setDescription("Delta WAL start offset") - .ofLongs() - .buildWithCallback(result -> { - if (MetricsLevel.DEBUG.isWithin(metricsLevel)) { - result.record(deltaWalStartOffsetSupplier.get(), AttributesCache.INSTANCE.defaultAttributes()); - } - }); + .setDescription("Delta WAL start offset") + .ofLongs() + .buildWithCallback(result -> { + if (MetricsLevel.DEBUG.isWithin(metricsLevel)) { + result.record(deltaWalStartOffsetSupplier.get(), AttributesCache.INSTANCE.defaultAttributes()); + } + }); deltaWalTrimmedOffset = meter.gaugeBuilder(prefix + S3StreamMetricsConstant.WAL_TRIMMED_OFFSET) - .setDescription("Delta WAL trimmed offset") - .ofLongs() - .buildWithCallback(result -> { - if (MetricsLevel.DEBUG.isWithin(metricsLevel)) { - result.record(deltaWalTrimmedOffsetSupplier.get(), AttributesCache.INSTANCE.defaultAttributes()); - } - }); + .setDescription("Delta WAL trimmed offset") + .ofLongs() + .buildWithCallback(result -> { + if (MetricsLevel.DEBUG.isWithin(metricsLevel)) { + result.record(deltaWalTrimmedOffsetSupplier.get(), AttributesCache.INSTANCE.defaultAttributes()); + } + }); deltaWalCacheSize = meter.gaugeBuilder(prefix + S3StreamMetricsConstant.DELTA_WAL_CACHE_SIZE) - .setDescription("Delta WAL cache size") - .setUnit("bytes") - .ofLongs() - .buildWithCallback(result -> { - if (MetricsLevel.DEBUG.isWithin(metricsLevel)) { - result.record(deltaWALCacheSizeSupplier.get(), AttributesCache.INSTANCE.defaultAttributes()); - } - }); + .setDescription("Delta WAL cache size") + .setUnit("bytes") + .ofLongs() + .buildWithCallback(result -> { + if (MetricsLevel.DEBUG.isWithin(metricsLevel)) { + result.record(deltaWALCacheSizeSupplier.get(), AttributesCache.INSTANCE.defaultAttributes()); + } + }); blockCacheSize = meter.gaugeBuilder(prefix + S3StreamMetricsConstant.BLOCK_CACHE_SIZE) - .setDescription("Block cache size") - .setUnit("bytes") - .ofLongs() - .buildWithCallback(result -> { - if (MetricsLevel.DEBUG.isWithin(metricsLevel)) { - result.record(blockCacheSizeSupplier.get(), AttributesCache.INSTANCE.defaultAttributes()); - } - }); + .setDescription("Block cache size") + .setUnit("bytes") + .ofLongs() + .buildWithCallback(result -> { + if (MetricsLevel.DEBUG.isWithin(metricsLevel)) { + result.record(blockCacheSizeSupplier.get(), AttributesCache.INSTANCE.defaultAttributes()); + } + }); availableInflightReadAheadSize = meter.gaugeBuilder(prefix + S3StreamMetricsConstant.AVAILABLE_INFLIGHT_READ_AHEAD_SIZE_METRIC_NAME) - .setDescription("Available inflight read ahead size") - .setUnit("bytes") - .ofLongs() - .buildWithCallback(result -> { - if (MetricsLevel.DEBUG.isWithin(metricsLevel)) { - result.record((long) availableInflightReadAheadSizeSupplier.get(), AttributesCache.INSTANCE.defaultAttributes()); - } - }); + .setDescription("Available inflight read ahead size") + .setUnit("bytes") + .ofLongs() + .buildWithCallback(result -> { + if (MetricsLevel.DEBUG.isWithin(metricsLevel)) { + result.record((long) availableInflightReadAheadSizeSupplier.get(), AttributesCache.INSTANCE.defaultAttributes()); + } + }); availableInflightS3ReadQuota = meter.gaugeBuilder(prefix + S3StreamMetricsConstant.AVAILABLE_S3_INFLIGHT_READ_QUOTA_METRIC_NAME) - .setDescription("Available inflight S3 read quota") - .ofLongs() - .buildWithCallback(result -> { - if (MetricsLevel.DEBUG.isWithin(metricsLevel)) { - result.record((long) availableInflightS3ReadQuotaSupplier.get(), AttributesCache.INSTANCE.defaultAttributes()); - } - }); + .setDescription("Available inflight S3 read quota") + .ofLongs() + .buildWithCallback(result -> { + if (MetricsLevel.DEBUG.isWithin(metricsLevel)) { + result.record((long) availableInflightS3ReadQuotaSupplier.get(), AttributesCache.INSTANCE.defaultAttributes()); + } + }); availableInflightS3WriteQuota = meter.gaugeBuilder(prefix + S3StreamMetricsConstant.AVAILABLE_S3_INFLIGHT_WRITE_QUOTA_METRIC_NAME) - .setDescription("Available inflight S3 write quota") - .ofLongs() - .buildWithCallback(result -> { - if (MetricsLevel.DEBUG.isWithin(metricsLevel)) { - result.record((long) availableInflightS3WriteQuotaSupplier.get(), AttributesCache.INSTANCE.defaultAttributes()); - } - }); + .setDescription("Available inflight S3 write quota") + .ofLongs() + .buildWithCallback(result -> { + if (MetricsLevel.DEBUG.isWithin(metricsLevel)) { + result.record((long) availableInflightS3WriteQuotaSupplier.get(), AttributesCache.INSTANCE.defaultAttributes()); + } + }); inflightWALUploadTasksCount = meter.gaugeBuilder(prefix + S3StreamMetricsConstant.INFLIGHT_WAL_UPLOAD_TASKS_COUNT_METRIC_NAME) - .setDescription("Inflight upload WAL tasks count") - .ofLongs() - .buildWithCallback(result -> { - if (MetricsLevel.DEBUG.isWithin(metricsLevel)) { - result.record((long) inflightWALUploadTasksCountSupplier.get(), AttributesCache.INSTANCE.defaultAttributes()); - } - }); + .setDescription("Inflight upload WAL tasks count") + .ofLongs() + .buildWithCallback(result -> { + if (MetricsLevel.DEBUG.isWithin(metricsLevel)) { + result.record((long) inflightWALUploadTasksCountSupplier.get(), AttributesCache.INSTANCE.defaultAttributes()); + } + }); compactionReadSizeInTotal = meter.counterBuilder(prefix + S3StreamMetricsConstant.COMPACTION_READ_SIZE_METRIC_NAME) - .setDescription("Compaction read size") - .setUnit("bytes") - .build(); + .setDescription("Compaction read size") + .setUnit("bytes") + .build(); compactionWriteSizeInTotal = meter.counterBuilder(prefix + S3StreamMetricsConstant.COMPACTION_WRITE_SIZE_METRIC_NAME) - .setDescription("Compaction write size") - .setUnit("bytes") - .build(); + .setDescription("Compaction write size") + .setUnit("bytes") + .build(); } public static void registerNetworkLimiterSupplier(AsyncNetworkBandwidthLimiter.Type type, - Supplier networkAvailableBandwidthSupplier, - Supplier networkLimiterQueueSizeSupplier) { + Supplier networkAvailableBandwidthSupplier, + Supplier networkLimiterQueueSizeSupplier) { switch (type) { case INBOUND -> { S3StreamMetricsManager.networkInboundAvailableBandwidthSupplier = networkAvailableBandwidthSupplier; @@ -274,7 +273,7 @@ public static void registerNetworkLimiterSupplier(AsyncNetworkBandwidthLimiter.T } public static void registerDeltaWalOffsetSupplier(Supplier deltaWalStartOffsetSupplier, - Supplier deltaWalTrimmedOffsetSupplier) { + Supplier deltaWalTrimmedOffsetSupplier) { S3StreamMetricsManager.deltaWalStartOffsetSupplier = deltaWalStartOffsetSupplier; S3StreamMetricsManager.deltaWalTrimmedOffsetSupplier = deltaWalTrimmedOffsetSupplier; } @@ -295,11 +294,13 @@ public static void registerInflightS3WriteQuotaSupplier(Supplier inflig S3StreamMetricsManager.availableInflightS3WriteQuotaSupplier = inflightS3WriteQuotaSupplier; } - public static void registerInflightReadSizeLimiterSupplier(Supplier availableInflightReadAheadSizeSupplier) { + public static void registerInflightReadSizeLimiterSupplier( + Supplier availableInflightReadAheadSizeSupplier) { S3StreamMetricsManager.availableInflightReadAheadSizeSupplier = availableInflightReadAheadSizeSupplier; } - public static void registerInflightWALUploadTasksCountSupplier(Supplier inflightWALUploadTasksCountSupplier) { + public static void registerInflightWALUploadTasksCountSupplier( + Supplier inflightWALUploadTasksCountSupplier) { S3StreamMetricsManager.inflightWALUploadTasksCountSupplier = inflightWALUploadTasksCountSupplier; } @@ -319,7 +320,8 @@ public static void recordOperationLatency(MetricsLevel level, long value, S3Oper recordOperationLatency(level, value, operation, 0, true); } - public static void recordOperationLatency(MetricsLevel level, long value, S3Operation operation, boolean isSuccess) { + public static void recordOperationLatency(MetricsLevel level, long value, S3Operation operation, + boolean isSuccess) { recordOperationLatency(level, value, operation, 0, isSuccess); } @@ -327,7 +329,8 @@ public static void recordOperationLatency(MetricsLevel level, long value, S3Oper recordOperationLatency(level, value, operation, size, true); } - public static void recordOperationLatency(MetricsLevel level, long value, S3Operation operation, long size, boolean isSuccess) { + public static void recordOperationLatency(MetricsLevel level, long value, S3Operation operation, long size, + boolean isSuccess) { if (level.isWithin(metricsLevel)) { operationLatency.record(value, AttributesCache.INSTANCE.getAttributes(operation, size, isSuccess)); } @@ -339,7 +342,8 @@ public static void recordStageLatency(MetricsLevel level, long value, S3Stage st } } - public static void recordReadCacheLatency(MetricsLevel level, long value, S3Operation operation, boolean isCacheHit) { + public static void recordReadCacheLatency(MetricsLevel level, long value, S3Operation operation, + boolean isCacheHit) { if (level.isWithin(metricsLevel)) { operationLatency.record(value, AttributesCache.INSTANCE.getAttributes(operation, isCacheHit ? "hit" : "miss")); } @@ -387,11 +391,14 @@ public static void recordNetworkOutboundUsage(MetricsLevel level, long value) { } } - public static void recordNetworkLimiterQueueTime(MetricsLevel level, long value, AsyncNetworkBandwidthLimiter.Type type) { + public static void recordNetworkLimiterQueueTime(MetricsLevel level, long value, + AsyncNetworkBandwidthLimiter.Type type) { if (level.isWithin(metricsLevel)) { switch (type) { - case INBOUND -> networkInboundLimiterQueueTime.record(value, AttributesCache.INSTANCE.defaultAttributes()); - case OUTBOUND -> networkOutboundLimiterQueueTime.record(value, AttributesCache.INSTANCE.defaultAttributes()); + case INBOUND -> + networkInboundLimiterQueueTime.record(value, AttributesCache.INSTANCE.defaultAttributes()); + case OUTBOUND -> + networkOutboundLimiterQueueTime.record(value, AttributesCache.INSTANCE.defaultAttributes()); } } } diff --git a/s3stream/src/main/java/com/automq/stream/s3/metrics/operations/S3MetricsType.java b/s3stream/src/main/java/com/automq/stream/s3/metrics/operations/S3MetricsType.java index dc371afb7..64b63c133 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/metrics/operations/S3MetricsType.java +++ b/s3stream/src/main/java/com/automq/stream/s3/metrics/operations/S3MetricsType.java @@ -27,10 +27,10 @@ public enum S3MetricsType { S3Network("S3Network"); private static final Map MAP = Map.of( - "S3Stream", S3Stream, - "S3Storage", S3Storage, - "S3Request", S3Request, - "S3Object", S3Object + "S3Stream", S3Stream, + "S3Storage", S3Storage, + "S3Request", S3Request, + "S3Object", S3Object ); private final String name; @@ -50,7 +50,7 @@ public S3MetricsType of(String name) { @Override public String toString() { return "S3MetricsType{" + - "name='" + name + '\'' + - '}'; + "name='" + name + '\'' + + '}'; } } diff --git a/s3stream/src/main/java/com/automq/stream/s3/metrics/operations/S3Operation.java b/s3stream/src/main/java/com/automq/stream/s3/metrics/operations/S3Operation.java index 36c8783cb..f42939bf3 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/metrics/operations/S3Operation.java +++ b/s3stream/src/main/java/com/automq/stream/s3/metrics/operations/S3Operation.java @@ -93,8 +93,8 @@ public String getUniqueKey() { @Override public String toString() { return "Operation{" + - "type='" + type.getName() + '\'' + - ", name='" + name + '\'' + - '}'; + "type='" + type.getName() + '\'' + + ", name='" + name + '\'' + + '}'; } } diff --git a/s3stream/src/main/java/com/automq/stream/s3/metrics/operations/S3Stage.java b/s3stream/src/main/java/com/automq/stream/s3/metrics/operations/S3Stage.java index cfd2ce5ca..db0782d7c 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/metrics/operations/S3Stage.java +++ b/s3stream/src/main/java/com/automq/stream/s3/metrics/operations/S3Stage.java @@ -59,8 +59,8 @@ public String getName() { @Override public String toString() { return "S3Stage{" + - "operation=" + operation.getName() + - ", name='" + name + '\'' + - '}'; + "operation=" + operation.getName() + + ", name='" + name + '\'' + + '}'; } } diff --git a/s3stream/src/main/java/com/automq/stream/s3/model/StreamRecordBatch.java b/s3stream/src/main/java/com/automq/stream/s3/model/StreamRecordBatch.java index 3b8934fa7..20c10addd 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/model/StreamRecordBatch.java +++ b/s3stream/src/main/java/com/automq/stream/s3/model/StreamRecordBatch.java @@ -107,10 +107,10 @@ public int compareTo(StreamRecordBatch o) { @Override public String toString() { return "StreamRecordBatch{" + - "streamId=" + streamId + - ", epoch=" + epoch + - ", baseOffset=" + baseOffset + - ", count=" + count + - ", size=" + size() + '}'; + "streamId=" + streamId + + ", epoch=" + epoch + + ", baseOffset=" + baseOffset + + ", count=" + count + + ", size=" + size() + '}'; } } diff --git a/s3stream/src/main/java/com/automq/stream/s3/network/AsyncNetworkBandwidthLimiter.java b/s3stream/src/main/java/com/automq/stream/s3/network/AsyncNetworkBandwidthLimiter.java index 96e6e20fd..e028248c5 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/network/AsyncNetworkBandwidthLimiter.java +++ b/s3stream/src/main/java/com/automq/stream/s3/network/AsyncNetworkBandwidthLimiter.java @@ -20,7 +20,6 @@ import com.automq.stream.s3.metrics.MetricsLevel; import com.automq.stream.s3.metrics.S3StreamMetricsManager; import io.netty.util.concurrent.DefaultThreadFactory; - import java.util.Objects; import java.util.PriorityQueue; import java.util.Queue; @@ -37,11 +36,11 @@ public class AsyncNetworkBandwidthLimiter { private final Lock lock = new ReentrantLock(); private final Condition condition = lock.newCondition(); private final long maxTokens; - private long availableTokens; private final ScheduledExecutorService refillThreadPool; private final ExecutorService callbackThreadPool; private final Queue queuedCallbacks; private final Type type; + private long availableTokens; public AsyncNetworkBandwidthLimiter(Type type, long tokenSize, int refillIntervalMs, long maxTokenSize) { this.type = type; @@ -156,13 +155,6 @@ private void logMetrics(long size) { } } - record BucketItem(int priority, long size, CompletableFuture cf) implements Comparable { - @Override - public int compareTo(BucketItem o) { - return Long.compare(priority, o.priority); - } - } - public enum Type { INBOUND("Inbound"), OUTBOUND("Outbound"); @@ -177,4 +169,11 @@ public String getName() { return name; } } + + record BucketItem(int priority, long size, CompletableFuture cf) implements Comparable { + @Override + public int compareTo(BucketItem o) { + return Long.compare(priority, o.priority); + } + } } diff --git a/s3stream/src/main/java/com/automq/stream/s3/objects/CommitStreamSetObjectRequest.java b/s3stream/src/main/java/com/automq/stream/s3/objects/CommitStreamSetObjectRequest.java index cc3a3d172..abee93954 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/objects/CommitStreamSetObjectRequest.java +++ b/s3stream/src/main/java/com/automq/stream/s3/objects/CommitStreamSetObjectRequest.java @@ -74,10 +74,6 @@ public void setObjectSize(long objectSize) { this.objectSize = objectSize; } - public void setCompactedObjectIds(List compactedObjectIds) { - this.compactedObjectIds = compactedObjectIds; - } - public List getCompactedObjectIds() { if (compactedObjectIds == null) { return Collections.emptyList(); @@ -85,6 +81,10 @@ public List getCompactedObjectIds() { return compactedObjectIds; } + public void setCompactedObjectIds(List compactedObjectIds) { + this.compactedObjectIds = compactedObjectIds; + } + public List getStreamRanges() { if (streamRanges == null) { return Collections.emptyList(); @@ -132,12 +132,12 @@ public void setOrderId(long orderId) { @Override public String toString() { return "CommitStreamSetObjectRequest{" + - "objectId=" + objectId + - ", orderId=" + orderId + - ", objectSize=" + objectSize + - ", streamRanges=" + streamRanges + - ", streamObjects=" + streamObjects + - ", compactedObjectIds=" + compactedObjectIds + - '}'; + "objectId=" + objectId + + ", orderId=" + orderId + + ", objectSize=" + objectSize + + ", streamRanges=" + streamRanges + + ", streamObjects=" + streamObjects + + ", compactedObjectIds=" + compactedObjectIds + + '}'; } } diff --git a/s3stream/src/main/java/com/automq/stream/s3/objects/CompactStreamObjectRequest.java b/s3stream/src/main/java/com/automq/stream/s3/objects/CompactStreamObjectRequest.java index 3a90b1949..f63b8c1ce 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/objects/CompactStreamObjectRequest.java +++ b/s3stream/src/main/java/com/automq/stream/s3/objects/CompactStreamObjectRequest.java @@ -30,7 +30,8 @@ public class CompactStreamObjectRequest { */ private List sourceObjectIds; - public CompactStreamObjectRequest(long objectId, long objectSize, long streamId, long startOffset, long endOffset, List sourceObjectIds) { + public CompactStreamObjectRequest(long objectId, long objectSize, long streamId, long startOffset, long endOffset, + List sourceObjectIds) { this.objectId = objectId; this.objectSize = objectSize; this.streamId = streamId; @@ -90,12 +91,12 @@ public void setSourceObjectIds(List sourceObjectIds) { @Override public String toString() { return "CommitStreamObjectRequest{" + - "objectId=" + objectId + - ", objectSize=" + objectSize + - ", streamId=" + streamId + - ", startOffset=" + startOffset + - ", endOffset=" + endOffset + - ", sourceObjectIds=" + sourceObjectIds + - '}'; + "objectId=" + objectId + + ", objectSize=" + objectSize + + ", streamId=" + streamId + + ", startOffset=" + startOffset + + ", endOffset=" + endOffset + + ", sourceObjectIds=" + sourceObjectIds + + '}'; } } diff --git a/s3stream/src/main/java/com/automq/stream/s3/objects/ObjectManager.java b/s3stream/src/main/java/com/automq/stream/s3/objects/ObjectManager.java index 5ec7b40f2..99325b682 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/objects/ObjectManager.java +++ b/s3stream/src/main/java/com/automq/stream/s3/objects/ObjectManager.java @@ -18,7 +18,6 @@ package com.automq.stream.s3.objects; import com.automq.stream.s3.metadata.S3ObjectMetadata; - import java.util.List; import java.util.concurrent.CompletableFuture; @@ -61,10 +60,11 @@ public interface ObjectManager { *
  • Returned objects aren't physical object concept, they are logical object concept. * (regard each returned object-metadata as a slice of object) * - * @param streamId stream id. + * + * @param streamId stream id. * @param startOffset get range start offset. - * @param endOffset get range end offset. NOOP_OFFSET represent endOffset is unlimited. - * @param limit max object range count. + * @param endOffset get range end offset. NOOP_OFFSET represent endOffset is unlimited. + * @param limit max object range count. * @return {@link S3ObjectMetadata} */ CompletableFuture> getObjects(long streamId, long startOffset, long endOffset, int limit); @@ -82,12 +82,14 @@ public interface ObjectManager { *
  • Returned stream objects can be discontinuous of stream range. *
  • Ranges of the returned stream objects are in ascending order. * - * @param streamId stream id. + * + * @param streamId stream id. * @param startOffset get range start offset. - * @param endOffset get range end offset. - * @param limit max object count. + * @param endOffset get range end offset. + * @param limit max object count. * @return {@link S3ObjectMetadata} */ - CompletableFuture> getStreamObjects(long streamId, long startOffset, long endOffset, int limit); + CompletableFuture> getStreamObjects(long streamId, long startOffset, long endOffset, + int limit); } diff --git a/s3stream/src/main/java/com/automq/stream/s3/objects/ObjectStreamRange.java b/s3stream/src/main/java/com/automq/stream/s3/objects/ObjectStreamRange.java index 0e9642485..5df33594c 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/objects/ObjectStreamRange.java +++ b/s3stream/src/main/java/com/automq/stream/s3/objects/ObjectStreamRange.java @@ -39,38 +39,38 @@ public long getStreamId() { return streamId; } - public long getEpoch() { - return epoch; - } - - public long getStartOffset() { - return startOffset; - } - - public long getEndOffset() { - return endOffset; - } - - public int getSize() { - return size; - } - public void setStreamId(long streamId) { this.streamId = streamId; } + public long getEpoch() { + return epoch; + } + public void setEpoch(long epoch) { this.epoch = epoch; } + public long getStartOffset() { + return startOffset; + } + public void setStartOffset(long startOffset) { this.startOffset = startOffset; } + public long getEndOffset() { + return endOffset; + } + public void setEndOffset(long endOffset) { this.endOffset = endOffset; } + public int getSize() { + return size; + } + public void setSize(int size) { this.size = size; } diff --git a/s3stream/src/main/java/com/automq/stream/s3/objects/StreamObject.java b/s3stream/src/main/java/com/automq/stream/s3/objects/StreamObject.java index e44425e15..8bf2f3c8d 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/objects/StreamObject.java +++ b/s3stream/src/main/java/com/automq/stream/s3/objects/StreamObject.java @@ -17,7 +17,6 @@ package com.automq.stream.s3.objects; - import java.util.List; public class StreamObject { @@ -83,12 +82,12 @@ public void setSourceObjectIds(List sourceObjectIds) { @Override public String toString() { return "StreamObject{" + - "objectId=" + objectId + - ", objectSize=" + objectSize + - ", streamId=" + streamId + - ", startOffset=" + startOffset + - ", endOffset=" + endOffset + - ", sourceObjectIds=" + sourceObjectIds + - '}'; + "objectId=" + objectId + + ", objectSize=" + objectSize + + ", streamId=" + streamId + + ", startOffset=" + startOffset + + ", endOffset=" + endOffset + + ", sourceObjectIds=" + sourceObjectIds + + '}'; } } diff --git a/s3stream/src/main/java/com/automq/stream/s3/operator/DefaultS3Operator.java b/s3stream/src/main/java/com/automq/stream/s3/operator/DefaultS3Operator.java index abe18d40d..cb58f8241 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/operator/DefaultS3Operator.java +++ b/s3stream/src/main/java/com/automq/stream/s3/operator/DefaultS3Operator.java @@ -144,10 +144,6 @@ public DefaultS3Operator(String endpoint, String region, String bucket, boolean S3StreamMetricsManager.registerInflightS3WriteQuotaSupplier(inflightWriteLimiter::availablePermits); } - public static Builder builder() { - return new Builder(); - } - // used for test only. public DefaultS3Operator(S3AsyncClient s3Client, String bucket) { this(s3Client, bucket, false); @@ -168,6 +164,23 @@ public DefaultS3Operator(S3AsyncClient s3Client, String bucket) { } } + public static Builder builder() { + return new Builder(); + } + + private static boolean checkPartNumbers(CompletedMultipartUpload multipartUpload) { + Optional maxOpt = multipartUpload.parts().stream().map(CompletedPart::partNumber).max(Integer::compareTo); + return maxOpt.isPresent() && maxOpt.get() == multipartUpload.parts().size(); + } + + private static boolean isUnrecoverable(Throwable ex) { + ex = cause(ex); + if (ex instanceof S3Exception s3Ex) { + return s3Ex.statusCode() == HttpStatusCode.FORBIDDEN || s3Ex.statusCode() == HttpStatusCode.NOT_FOUND; + } + return false; + } + @Override public void close() { // TODO: complete in-flight CompletableFuture with ClosedException. @@ -559,11 +572,6 @@ public void completeMultipartUpload0(String path, String uploadId, List maxOpt = multipartUpload.parts().stream().map(CompletedPart::partNumber).max(Integer::compareTo); - return maxOpt.isPresent() && maxOpt.get() == multipartUpload.parts().size(); - } - private String range(long start, long end) { if (end == -1L) { return "bytes=" + start + "-"; @@ -571,14 +579,6 @@ private String range(long start, long end) { return "bytes=" + start + "-" + end; } - private static boolean isUnrecoverable(Throwable ex) { - ex = cause(ex); - if (ex instanceof S3Exception s3Ex) { - return s3Ex.statusCode() == HttpStatusCode.FORBIDDEN || s3Ex.statusCode() == HttpStatusCode.NOT_FOUND; - } - return false; - } - private void checkConfig() { if (this.networkInboundBandwidthLimiter != null) { if (this.networkInboundBandwidthLimiter.getMaxTokens() < Writer.MIN_PART_SIZE) { diff --git a/s3stream/src/main/java/com/automq/stream/s3/operator/MemoryS3Operator.java b/s3stream/src/main/java/com/automq/stream/s3/operator/MemoryS3Operator.java index b65b802de..b686544a8 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/operator/MemoryS3Operator.java +++ b/s3stream/src/main/java/com/automq/stream/s3/operator/MemoryS3Operator.java @@ -20,12 +20,11 @@ import com.automq.stream.utils.FutureUtil; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; -import software.amazon.awssdk.services.s3.model.CompletedPart; - import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; +import software.amazon.awssdk.services.s3.model.CompletedPart; public class MemoryS3Operator implements S3Operator { private final Map storage = new ConcurrentHashMap<>(); @@ -115,12 +114,14 @@ public CompletableFuture createMultipartUpload(String path) { } @Override - public CompletableFuture uploadPart(String path, String uploadId, int partNumber, ByteBuf data, ThrottleStrategy throttleStrategy) { + public CompletableFuture uploadPart(String path, String uploadId, int partNumber, ByteBuf data, + ThrottleStrategy throttleStrategy) { return FutureUtil.failedFuture(new UnsupportedOperationException()); } @Override - public CompletableFuture uploadPartCopy(String sourcePath, String path, long start, long end, String uploadId, int partNumber) { + public CompletableFuture uploadPartCopy(String sourcePath, String path, long start, long end, + String uploadId, int partNumber) { return FutureUtil.failedFuture(new UnsupportedOperationException()); } diff --git a/s3stream/src/main/java/com/automq/stream/s3/operator/MultiPartWriter.java b/s3stream/src/main/java/com/automq/stream/s3/operator/MultiPartWriter.java index e9dc3a119..c900724e6 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/operator/MultiPartWriter.java +++ b/s3stream/src/main/java/com/automq/stream/s3/operator/MultiPartWriter.java @@ -26,8 +26,6 @@ import com.automq.stream.utils.FutureUtil; import io.netty.buffer.ByteBuf; import io.netty.buffer.CompositeByteBuf; -import software.amazon.awssdk.services.s3.model.CompletedPart; - import java.util.LinkedList; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -35,24 +33,25 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; +import software.amazon.awssdk.services.s3.model.CompletedPart; public class MultiPartWriter implements Writer { private static final long MAX_MERGE_WRITE_SIZE = 16L * 1024 * 1024; + final CompletableFuture uploadIdCf = new CompletableFuture<>(); private final S3Operator operator; private final String path; - final CompletableFuture uploadIdCf = new CompletableFuture<>(); - private String uploadId; private final List> parts = new LinkedList<>(); private final AtomicInteger nextPartNumber = new AtomicInteger(1); - private CompletableFuture closeCf; /** * The minPartSize represents the minimum size of a part for a multipart object. */ private final long minPartSize; - private ObjectPart objectPart = null; private final TimerUtil timerUtil = new TimerUtil(); private final ThrottleStrategy throttleStrategy; private final AtomicLong totalWriteSize = new AtomicLong(0L); + private String uploadId; + private CompletableFuture closeCf; + private ObjectPart objectPart = null; public MultiPartWriter(S3Operator operator, String path, long minPartSize, ThrottleStrategy throttleStrategy) { this.operator = operator; @@ -64,11 +63,11 @@ public MultiPartWriter(S3Operator operator, String path, long minPartSize, Throt private void init() { FutureUtil.propagate( - operator.createMultipartUpload(path).thenApply(uploadId -> { - this.uploadId = uploadId; - return uploadId; - }), - uploadIdCf + operator.createMultipartUpload(path).thenApply(uploadId -> { + this.uploadId = uploadId; + return uploadId; + }), + uploadIdCf ); } @@ -176,11 +175,11 @@ private List genCompleteParts() { class ObjectPart { private final int partNumber = nextPartNumber.getAndIncrement(); + private final CompletableFuture partCf = new CompletableFuture<>(); + private final ThrottleStrategy throttleStrategy; private CompositeByteBuf partBuf = DirectByteBufAlloc.compositeByteBuffer(); private CompletableFuture lastRangeReadCf = CompletableFuture.completedFuture(null); - private final CompletableFuture partCf = new CompletableFuture<>(); private long size; - private final ThrottleStrategy throttleStrategy; public ObjectPart(ThrottleStrategy throttleStrategy) { this.throttleStrategy = throttleStrategy; @@ -208,8 +207,8 @@ public void readAndWrite(String sourcePath, long start, long end) { size += end - start; // TODO: parallel read and sequence add. this.lastRangeReadCf = lastRangeReadCf - .thenCompose(nil -> operator.rangeRead(sourcePath, start, end, throttleStrategy)) - .thenAccept(buf -> partBuf.addComponent(true, buf)); + .thenCompose(nil -> operator.rangeRead(sourcePath, start, end, throttleStrategy)) + .thenAccept(buf -> partBuf.addComponent(true, buf)); } public void upload() { diff --git a/s3stream/src/main/java/com/automq/stream/s3/operator/ProxyWriter.java b/s3stream/src/main/java/com/automq/stream/s3/operator/ProxyWriter.java index 5c12fcaac..948104629 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/operator/ProxyWriter.java +++ b/s3stream/src/main/java/com/automq/stream/s3/operator/ProxyWriter.java @@ -22,7 +22,6 @@ import com.automq.stream.utils.FutureUtil; import io.netty.buffer.ByteBuf; import io.netty.buffer.CompositeByteBuf; - import java.util.concurrent.CompletableFuture; /** @@ -30,11 +29,11 @@ * Else, we should use multi-part upload to upload it. */ class ProxyWriter implements Writer { + final ObjectWriter objectWriter = new ObjectWriter(); private final S3Operator operator; private final String path; private final long minPartSize; private final ThrottleStrategy throttleStrategy; - final ObjectWriter objectWriter = new ObjectWriter(); Writer multiPartWriter = null; public ProxyWriter(S3Operator operator, String path, long minPartSize, ThrottleStrategy throttleStrategy) { diff --git a/s3stream/src/main/java/com/automq/stream/s3/operator/S3Operator.java b/s3stream/src/main/java/com/automq/stream/s3/operator/S3Operator.java index b7da45a62..7a0881143 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/operator/S3Operator.java +++ b/s3stream/src/main/java/com/automq/stream/s3/operator/S3Operator.java @@ -19,10 +19,9 @@ import com.automq.stream.s3.network.ThrottleStrategy; import io.netty.buffer.ByteBuf; -import software.amazon.awssdk.services.s3.model.CompletedPart; - import java.util.List; import java.util.concurrent.CompletableFuture; +import software.amazon.awssdk.services.s3.model.CompletedPart; public interface S3Operator { @@ -31,9 +30,9 @@ public interface S3Operator { /** * Range read from object. * - * @param path object path. - * @param start range start. - * @param end range end. + * @param path object path. + * @param start range start. + * @param end range end. * @param throttleStrategy throttle strategy. * @return data. */ @@ -46,8 +45,8 @@ default CompletableFuture rangeRead(String path, long start, long end) /** * Write data to object. * - * @param path object path. The path should not start with '/' since Aliyun OSS does not support it. - * @param data data. + * @param path object path. The path should not start with '/' since Aliyun OSS does not support it. + * @param data data. * @param throttleStrategy throttle strategy. */ CompletableFuture write(String path, ByteBuf data, ThrottleStrategy throttleStrategy); @@ -59,20 +58,21 @@ default CompletableFuture write(String path, ByteBuf data) { /** * New multipart object writer. * - * @param path object path + * @param path object path * @param throttleStrategy throttle strategy. * @return {@link Writer} */ Writer writer(String path, ThrottleStrategy throttleStrategy); default Writer writer(String path) { - return writer(path, ThrottleStrategy.BYPASS); + return writer(path, ThrottleStrategy.BYPASS); } CompletableFuture delete(String path); /** * Delete a list of objects. + * * @param objectKeys object keys to delete. * @return deleted object keys. */ @@ -82,6 +82,7 @@ default Writer writer(String path) { /** * Create mutlipart upload + * * @param path object path. * @return upload id. */ @@ -89,9 +90,11 @@ default Writer writer(String path) { /** * Upload part. + * * @return {@link CompletedPart} */ - CompletableFuture uploadPart(String path, String uploadId, int partNumber, ByteBuf data, ThrottleStrategy throttleStrategy); + CompletableFuture uploadPart(String path, String uploadId, int partNumber, ByteBuf data, + ThrottleStrategy throttleStrategy); default CompletableFuture uploadPart(String path, String uploadId, int partNumber, ByteBuf data) { return uploadPart(path, uploadId, partNumber, data, ThrottleStrategy.BYPASS); @@ -99,9 +102,11 @@ default CompletableFuture uploadPart(String path, String uploadId /** * Upload part copy + * * @return {@link CompletedPart} */ - CompletableFuture uploadPartCopy(String sourcePath, String path, long start, long end, String uploadId, int partNumber); + CompletableFuture uploadPartCopy(String sourcePath, String path, long start, long end, + String uploadId, int partNumber); CompletableFuture completeMultipartUpload(String path, String uploadId, List parts); } diff --git a/s3stream/src/main/java/com/automq/stream/s3/operator/Writer.java b/s3stream/src/main/java/com/automq/stream/s3/operator/Writer.java index e0615b75c..92b023185 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/operator/Writer.java +++ b/s3stream/src/main/java/com/automq/stream/s3/operator/Writer.java @@ -18,14 +18,13 @@ package com.automq.stream.s3.operator; import io.netty.buffer.ByteBuf; - import java.util.concurrent.CompletableFuture; /** * Multipart object writer. *

    - * Writer should ensure that a part, even with size smaller than {@link Writer#MIN_PART_SIZE}, can still be uploaded. - * For other S3 limits, it is upper layer's responsibility to prevent reaching the limits. + * Writer should ensure that a part, even with size smaller than {@link Writer#MIN_PART_SIZE}, can still be uploaded. + * For other S3 limits, it is upper layer's responsibility to prevent reaching the limits. */ public interface Writer { /** diff --git a/s3stream/src/main/java/com/automq/stream/s3/streams/StreamManager.java b/s3stream/src/main/java/com/automq/stream/s3/streams/StreamManager.java index 994bdb9f3..a0d945fc9 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/streams/StreamManager.java +++ b/s3stream/src/main/java/com/automq/stream/s3/streams/StreamManager.java @@ -17,18 +17,16 @@ package com.automq.stream.s3.streams; - import com.automq.stream.s3.metadata.StreamMetadata; - import java.util.List; import java.util.concurrent.CompletableFuture; - public interface StreamManager { /** * Get current server opening streams. * When server is starting or recovering, WAL in EBS need streams offset to determine the recover point. + * * @return list of {@link StreamMetadata} */ CompletableFuture> getOpeningStreams(); @@ -40,8 +38,7 @@ public interface StreamManager { * @return list of {@link StreamMetadata} */ CompletableFuture> getStreams(List streamIds); - - + /** * Create a new stream. * diff --git a/s3stream/src/main/java/com/automq/stream/s3/trace/AttributeBindings.java b/s3stream/src/main/java/com/automq/stream/s3/trace/AttributeBindings.java index c60247aeb..f98a6e033 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/trace/AttributeBindings.java +++ b/s3stream/src/main/java/com/automq/stream/s3/trace/AttributeBindings.java @@ -20,7 +20,6 @@ import io.opentelemetry.api.common.AttributeKey; import io.opentelemetry.api.common.AttributesBuilder; import io.opentelemetry.instrumentation.annotations.SpanAttribute; - import java.lang.reflect.Method; import java.lang.reflect.Parameter; import java.lang.reflect.Type; @@ -54,20 +53,6 @@ public static AttributeBindings bind(Method method, String[] parametersNames) { return new AttributeBindings(bindings); } - public boolean isEmpty() { - return bindings == null || bindings.length == 0; - } - - public void apply(AttributesBuilder target, Object[] args) { - if (args.length != bindings.length) { - return; - } - - for (int i = 0; i < args.length; i++) { - bindings[i].apply(target, args[i]); - } - } - static BiFunction emptyBinding() { return (builder, arg) -> builder; } @@ -103,4 +88,18 @@ static BiFunction createBinding(St AttributeKey key = AttributeKey.stringKey(name); return (builder, arg) -> builder.put(key, arg.toString()); } + + public boolean isEmpty() { + return bindings == null || bindings.length == 0; + } + + public void apply(AttributesBuilder target, Object[] args) { + if (args.length != bindings.length) { + return; + } + + for (int i = 0; i < args.length; i++) { + bindings[i].apply(target, args[i]); + } + } } diff --git a/s3stream/src/main/java/com/automq/stream/s3/trace/SpanAttributesExtractor.java b/s3stream/src/main/java/com/automq/stream/s3/trace/SpanAttributesExtractor.java index ffa4c0c2d..5a2c223a4 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/trace/SpanAttributesExtractor.java +++ b/s3stream/src/main/java/com/automq/stream/s3/trace/SpanAttributesExtractor.java @@ -19,21 +19,20 @@ import io.opentelemetry.api.common.Attributes; import io.opentelemetry.api.common.AttributesBuilder; - import java.lang.reflect.Method; public final class SpanAttributesExtractor { private final MethodCache cache; - public static SpanAttributesExtractor create() { - return new SpanAttributesExtractor(new MethodCache<>()); - } - SpanAttributesExtractor(MethodCache cache) { this.cache = cache; } + public static SpanAttributesExtractor create() { + return new SpanAttributesExtractor(new MethodCache<>()); + } + public Attributes extract(Method method, String[] parametersNames, Object[] args) { AttributesBuilder attributes = Attributes.builder(); AttributeBindings bindings = diff --git a/s3stream/src/main/java/com/automq/stream/s3/trace/TraceUtils.java b/s3stream/src/main/java/com/automq/stream/s3/trace/TraceUtils.java index 4f94724d1..3e8fb88e3 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/trace/TraceUtils.java +++ b/s3stream/src/main/java/com/automq/stream/s3/trace/TraceUtils.java @@ -24,20 +24,20 @@ import io.opentelemetry.api.trace.Tracer; import io.opentelemetry.context.Context; import io.opentelemetry.instrumentation.annotations.WithSpan; +import java.lang.reflect.Method; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import org.aspectj.lang.ProceedingJoinPoint; import org.aspectj.lang.reflect.MethodSignature; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.lang.reflect.Method; -import java.util.concurrent.Callable; -import java.util.concurrent.CompletableFuture; - public class TraceUtils { private static final Logger LOGGER = LoggerFactory.getLogger(TraceUtils.class); private static final SpanAttributesExtractor EXTRACTOR = SpanAttributesExtractor.create(); - public static Object trace(TraceContext context, ProceedingJoinPoint joinPoint, WithSpan withSpan) throws Throwable { + public static Object trace(TraceContext context, ProceedingJoinPoint joinPoint, + WithSpan withSpan) throws Throwable { if (context.isTraceDisabled()) { return joinPoint.proceed(); } @@ -69,7 +69,8 @@ public static Object trace(TraceContext context, ProceedingJoinPoint joinPoint, } } - public static T runWithSpanSync(TraceContext context, Attributes attributes, String spanName, Callable callable) throws Throwable { + public static T runWithSpanSync(TraceContext context, Attributes attributes, String spanName, + Callable callable) throws Throwable { TraceContext.Scope scope = createAndStartSpan(context, spanName); if (scope == null) { return callable.call(); @@ -85,8 +86,9 @@ public static T runWithSpanSync(TraceContext context, Attributes attributes, } } - public static CompletableFuture runWithSpanAsync(TraceContext context, Attributes attributes, String spanName, - Callable> callable) throws Throwable { + public static CompletableFuture runWithSpanAsync(TraceContext context, Attributes attributes, + String spanName, + Callable> callable) throws Throwable { TraceContext.Scope scope = createAndStartSpan(context, spanName); if (scope == null) { return callable.call(); @@ -109,8 +111,8 @@ public static TraceContext.Scope createAndStartSpan(TraceContext context, String Tracer tracer = context.tracer(); Context parentContext = context.currentContext(); Span span = tracer.spanBuilder(name) - .setParent(parentContext) - .startSpan(); + .setParent(parentContext) + .startSpan(); return context.attachContext(parentContext.with(span)); } @@ -129,12 +131,14 @@ public static void endSpan(TraceContext.Scope scope, Throwable t) { scope.close(); } - private static CompletableFuture doTraceWhenReturnCompletableFuture(TraceContext.Scope scope, ProceedingJoinPoint joinPoint) throws Throwable { + private static CompletableFuture doTraceWhenReturnCompletableFuture(TraceContext.Scope scope, + ProceedingJoinPoint joinPoint) throws Throwable { CompletableFuture future = (CompletableFuture) joinPoint.proceed(); return future.whenComplete((r, t) -> endSpan(scope, t)); } - private static Object doTraceWhenReturnObject(TraceContext.Scope scope, ProceedingJoinPoint joinPoint) throws Throwable { + private static Object doTraceWhenReturnObject(TraceContext.Scope scope, + ProceedingJoinPoint joinPoint) throws Throwable { Object result = joinPoint.proceed(); endSpan(scope, null); return result; diff --git a/s3stream/src/main/java/com/automq/stream/s3/trace/context/TraceContext.java b/s3stream/src/main/java/com/automq/stream/s3/trace/context/TraceContext.java index a8342221a..f744fe1e0 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/trace/context/TraceContext.java +++ b/s3stream/src/main/java/com/automq/stream/s3/trace/context/TraceContext.java @@ -21,19 +21,18 @@ import io.opentelemetry.api.trace.Span; import io.opentelemetry.api.trace.Tracer; import io.opentelemetry.context.Context; +import javax.annotation.concurrent.NotThreadSafe; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.annotation.concurrent.NotThreadSafe; - /** * Trace context that holds the current trace context. This class is not thread safe and should be copied before * asynchronous usage. */ @NotThreadSafe public class TraceContext { - private static final Logger LOGGER = LoggerFactory.getLogger(TraceContext.class); public static final TraceContext DEFAULT = new TraceContext(false, null, null); + private static final Logger LOGGER = LoggerFactory.getLogger(TraceContext.class); private final boolean isTraceEnabled; private final Tracer tracer; private Context currContext; diff --git a/s3stream/src/main/java/com/automq/stream/s3/wal/Block.java b/s3stream/src/main/java/com/automq/stream/s3/wal/Block.java index 010b7919a..28a5187b5 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/wal/Block.java +++ b/s3stream/src/main/java/com/automq/stream/s3/wal/Block.java @@ -17,10 +17,8 @@ package com.automq.stream.s3.wal; - import com.automq.stream.s3.wal.util.WALUtil; import io.netty.buffer.ByteBuf; - import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.function.Function; @@ -46,7 +44,8 @@ public interface Block { * @param future The future of this record, which will be completed when the record is written to the WAL. * @return The start offset of this record. If the size of this block exceeds the limit, return -1. */ - long addRecord(long recordSize, Function recordSupplier, CompletableFuture future); + long addRecord(long recordSize, Function recordSupplier, + CompletableFuture future); /** * Futures of all records in this block. diff --git a/s3stream/src/main/java/com/automq/stream/s3/wal/BlockBatch.java b/s3stream/src/main/java/com/automq/stream/s3/wal/BlockBatch.java index 08f086a10..a7359a978 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/wal/BlockBatch.java +++ b/s3stream/src/main/java/com/automq/stream/s3/wal/BlockBatch.java @@ -32,13 +32,13 @@ public BlockBatch(Collection blocks) { assert !blocks.isEmpty(); this.blocks = blocks; this.startOffset = blocks.stream() - .map(Block::startOffset) - .min(Long::compareTo) - .orElseThrow(); + .map(Block::startOffset) + .min(Long::compareTo) + .orElseThrow(); this.endOffset = blocks.stream() - .map(b -> b.startOffset() + b.size()) - .max(Long::compareTo) - .orElseThrow(); + .map(b -> b.startOffset() + b.size()) + .max(Long::compareTo) + .orElseThrow(); } public long startOffset() { @@ -55,9 +55,9 @@ public Collection blocks() { public List> futures() { return blocks.stream() - .map(Block::futures) - .flatMap(List::stream) - .toList(); + .map(Block::futures) + .flatMap(List::stream) + .toList(); } diff --git a/s3stream/src/main/java/com/automq/stream/s3/wal/BlockImpl.java b/s3stream/src/main/java/com/automq/stream/s3/wal/BlockImpl.java index 1f96fd62e..b7ecd12f6 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/wal/BlockImpl.java +++ b/s3stream/src/main/java/com/automq/stream/s3/wal/BlockImpl.java @@ -25,7 +25,6 @@ import com.automq.stream.s3.wal.util.WALUtil; import io.netty.buffer.ByteBuf; import io.netty.buffer.CompositeByteBuf; - import java.util.LinkedList; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -77,7 +76,8 @@ public long startOffset() { * Note: this method is NOT thread safe. */ @Override - public long addRecord(long recordSize, Function recordSupplier, CompletableFuture future) { + public long addRecord(long recordSize, Function recordSupplier, + CompletableFuture future) { assert data == null; long requiredCapacity = nextOffset + recordSize; if (requiredCapacity > maxSize) { diff --git a/s3stream/src/main/java/com/automq/stream/s3/wal/BlockWALService.java b/s3stream/src/main/java/com/automq/stream/s3/wal/BlockWALService.java index 5c856f98a..d1f4a64d7 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/wal/BlockWALService.java +++ b/s3stream/src/main/java/com/automq/stream/s3/wal/BlockWALService.java @@ -32,10 +32,6 @@ import com.automq.stream.utils.Threads; import io.netty.buffer.ByteBuf; import io.netty.buffer.CompositeByteBuf; -import org.apache.commons.lang3.time.StopWatch; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.util.Collections; import java.util.Iterator; @@ -49,6 +45,9 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.Lock; import java.util.function.Function; +import org.apache.commons.lang3.time.StopWatch; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static com.automq.stream.s3.Constants.CAPACITY_NOT_SET; import static com.automq.stream.s3.Constants.NOOP_EPOCH; @@ -166,7 +165,8 @@ private synchronized void flushWALHeader() throws IOException { * * @throws ReadRecordException if the record is not found or the record is corrupted */ - private ByteBuf readRecord(long recoverStartOffset, Function logicalToPhysical) throws ReadRecordException { + private ByteBuf readRecord(long recoverStartOffset, + Function logicalToPhysical) throws ReadRecordException { final ByteBuf recordHeader = DirectByteBufAlloc.byteBuffer(RECORD_HEADER_SIZE); SlidingWindowService.RecordHeaderCoreData readRecordHeader; try { @@ -187,29 +187,30 @@ private ByteBuf readRecord(long recoverStartOffset, Function logical return recordBody; } - private SlidingWindowService.RecordHeaderCoreData parseRecordHeader(long recoverStartOffset, ByteBuf recordHeader, Function logicalToPhysical) throws ReadRecordException { + private SlidingWindowService.RecordHeaderCoreData parseRecordHeader(long recoverStartOffset, ByteBuf recordHeader, + Function logicalToPhysical) throws ReadRecordException { final long position = logicalToPhysical.apply(recoverStartOffset); try { int read = walChannel.read(recordHeader, position); if (read != RECORD_HEADER_SIZE) { throw new ReadRecordException( - WALUtil.alignNextBlock(recoverStartOffset), - String.format("failed to read record header: expected %d bytes, actual %d bytes, recoverStartOffset: %d", RECORD_HEADER_SIZE, read, recoverStartOffset) + WALUtil.alignNextBlock(recoverStartOffset), + String.format("failed to read record header: expected %d bytes, actual %d bytes, recoverStartOffset: %d", RECORD_HEADER_SIZE, read, recoverStartOffset) ); } } catch (IOException e) { LOGGER.error("failed to read record header, position: {}, recoverStartOffset: {}", position, recoverStartOffset, e); throw new ReadRecordException( - WALUtil.alignNextBlock(recoverStartOffset), - String.format("failed to read record header, recoverStartOffset: %d", recoverStartOffset) + WALUtil.alignNextBlock(recoverStartOffset), + String.format("failed to read record header, recoverStartOffset: %d", recoverStartOffset) ); } SlidingWindowService.RecordHeaderCoreData readRecordHeader = SlidingWindowService.RecordHeaderCoreData.unmarshal(recordHeader); if (readRecordHeader.getMagicCode() != RECORD_HEADER_MAGIC_CODE) { throw new ReadRecordException( - WALUtil.alignNextBlock(recoverStartOffset), - String.format("magic code mismatch: expected %d, actual %d, recoverStartOffset: %d", RECORD_HEADER_MAGIC_CODE, readRecordHeader.getMagicCode(), recoverStartOffset) + WALUtil.alignNextBlock(recoverStartOffset), + String.format("magic code mismatch: expected %d, actual %d, recoverStartOffset: %d", RECORD_HEADER_MAGIC_CODE, readRecordHeader.getMagicCode(), recoverStartOffset) ); } @@ -217,30 +218,31 @@ private SlidingWindowService.RecordHeaderCoreData parseRecordHeader(long recover int calculatedRecordHeaderCRC = WALUtil.crc32(recordHeader, RECORD_HEADER_WITHOUT_CRC_SIZE); if (recordHeaderCRC != calculatedRecordHeaderCRC) { throw new ReadRecordException( - WALUtil.alignNextBlock(recoverStartOffset), - String.format("record header crc mismatch: expected %d, actual %d, recoverStartOffset: %d", calculatedRecordHeaderCRC, recordHeaderCRC, recoverStartOffset) + WALUtil.alignNextBlock(recoverStartOffset), + String.format("record header crc mismatch: expected %d, actual %d, recoverStartOffset: %d", calculatedRecordHeaderCRC, recordHeaderCRC, recoverStartOffset) ); } int recordBodyLength = readRecordHeader.getRecordBodyLength(); if (recordBodyLength <= 0) { throw new ReadRecordException( - WALUtil.alignNextBlock(recoverStartOffset), - String.format("invalid record body length: %d, recoverStartOffset: %d", recordBodyLength, recoverStartOffset) + WALUtil.alignNextBlock(recoverStartOffset), + String.format("invalid record body length: %d, recoverStartOffset: %d", recordBodyLength, recoverStartOffset) ); } long recordBodyOffset = readRecordHeader.getRecordBodyOffset(); if (recordBodyOffset != recoverStartOffset + RECORD_HEADER_SIZE) { throw new ReadRecordException( - WALUtil.alignNextBlock(recoverStartOffset), - String.format("invalid record body offset: expected %d, actual %d, recoverStartOffset: %d", recoverStartOffset + RECORD_HEADER_SIZE, recordBodyOffset, recoverStartOffset) + WALUtil.alignNextBlock(recoverStartOffset), + String.format("invalid record body offset: expected %d, actual %d, recoverStartOffset: %d", recoverStartOffset + RECORD_HEADER_SIZE, recordBodyOffset, recoverStartOffset) ); } return readRecordHeader; } - private void parseRecordBody(long recoverStartOffset, SlidingWindowService.RecordHeaderCoreData readRecordHeader, ByteBuf recordBody, Function logicalToPhysical) throws ReadRecordException { + private void parseRecordBody(long recoverStartOffset, SlidingWindowService.RecordHeaderCoreData readRecordHeader, + ByteBuf recordBody, Function logicalToPhysical) throws ReadRecordException { long recordBodyOffset = readRecordHeader.getRecordBodyOffset(); int recordBodyLength = readRecordHeader.getRecordBodyLength(); try { @@ -248,15 +250,15 @@ private void parseRecordBody(long recoverStartOffset, SlidingWindowService.Recor int read = walChannel.read(recordBody, position); if (read != recordBodyLength) { throw new ReadRecordException( - WALUtil.alignNextBlock(recoverStartOffset + RECORD_HEADER_SIZE + recordBodyLength), - String.format("failed to read record body: expected %d bytes, actual %d bytes, recoverStartOffset: %d", recordBodyLength, read, recoverStartOffset) + WALUtil.alignNextBlock(recoverStartOffset + RECORD_HEADER_SIZE + recordBodyLength), + String.format("failed to read record body: expected %d bytes, actual %d bytes, recoverStartOffset: %d", recordBodyLength, read, recoverStartOffset) ); } } catch (IOException e) { LOGGER.error("failed to read record body, position: {}, recoverStartOffset: {}", recordBodyOffset, recoverStartOffset, e); throw new ReadRecordException( - WALUtil.alignNextBlock(recoverStartOffset + RECORD_HEADER_SIZE + recordBodyLength), - String.format("failed to read record body, recoverStartOffset: %d", recoverStartOffset) + WALUtil.alignNextBlock(recoverStartOffset + RECORD_HEADER_SIZE + recordBodyLength), + String.format("failed to read record body, recoverStartOffset: %d", recoverStartOffset) ); } @@ -264,8 +266,8 @@ private void parseRecordBody(long recoverStartOffset, SlidingWindowService.Recor int calculatedRecordBodyCRC = WALUtil.crc32(recordBody); if (recordBodyCRC != calculatedRecordBodyCRC) { throw new ReadRecordException( - WALUtil.alignNextBlock(recoverStartOffset + RECORD_HEADER_SIZE + recordBodyLength), - String.format("record body crc mismatch: expected %d, actual %d, recoverStartOffset: %d", calculatedRecordBodyCRC, recordBodyCRC, recoverStartOffset) + WALUtil.alignNextBlock(recoverStartOffset + RECORD_HEADER_SIZE + recordBodyLength), + String.format("record body crc mismatch: expected %d, actual %d, recoverStartOffset: %d", calculatedRecordBodyCRC, recordBodyCRC, recoverStartOffset) ); } } @@ -275,8 +277,8 @@ public WriteAheadLog start() throws IOException { StopWatch stopWatch = StopWatch.createStarted(); walChannel.open(channel -> Optional.ofNullable(tryReadWALHeader(walChannel)) - .map(WALHeader::getCapacity) - .orElse(null)); + .map(WALHeader::getCapacity) + .orElse(null)); WALHeader header = tryReadWALHeader(walChannel); if (null == header) { @@ -364,8 +366,8 @@ public void shutdownGracefully() { } boolean gracefulShutdown = Optional.ofNullable(slidingWindowService) - .map(s -> s.shutdown(1, TimeUnit.DAYS)) - .orElse(true); + .map(s -> s.shutdown(1, TimeUnit.DAYS)) + .orElse(true); try { flushWALHeader(gracefulShutdown ? ShutdownType.GRACEFULLY : ShutdownType.UNGRACEFULLY); } catch (IOException e) { @@ -432,11 +434,11 @@ private AppendResult append0(ByteBuf body, int crc) throws OverCapacityException private ByteBuf recordHeader(ByteBuf body, int crc, long start) { return new SlidingWindowService.RecordHeaderCoreData() - .setMagicCode(RECORD_HEADER_MAGIC_CODE) - .setRecordBodyLength(body.readableBytes()) - .setRecordBodyOffset(start + RECORD_HEADER_SIZE) - .setRecordBodyCRC(crc) - .marshal(); + .setMagicCode(RECORD_HEADER_MAGIC_CODE) + .setRecordBodyLength(body.readableBytes()) + .setRecordBodyOffset(start + RECORD_HEADER_SIZE) + .setRecordBodyCRC(crc) + .marshal(); } private ByteBuf record(ByteBuf body, int crc, long start) { @@ -562,17 +564,17 @@ public BlockWALServiceBuilder capacity(long capacity) { public BlockWALServiceBuilder config(Config config) { return this - .capacity(config.walCapacity()) - .initBufferSize(config.walInitBufferSize()) - .maxBufferSize(config.walMaxBufferSize()) - .ioThreadNums(config.walThread()) - .slidingWindowInitialSize(config.walWindowInitial()) - .slidingWindowScaleUnit(config.walWindowIncrement()) - .slidingWindowUpperLimit(config.walWindowMax()) - .blockSoftLimit(config.walBlockSoftLimit()) - .writeRateLimit(config.walWriteRateLimit()) - .nodeId(config.nodeId()) - .epoch(config.nodeEpoch()); + .capacity(config.walCapacity()) + .initBufferSize(config.walInitBufferSize()) + .maxBufferSize(config.walMaxBufferSize()) + .ioThreadNums(config.walThread()) + .slidingWindowInitialSize(config.walWindowInitial()) + .slidingWindowScaleUnit(config.walWindowIncrement()) + .slidingWindowUpperLimit(config.walWindowMax()) + .blockSoftLimit(config.walBlockSoftLimit()) + .writeRateLimit(config.walWriteRateLimit()) + .nodeId(config.nodeId()) + .epoch(config.nodeEpoch()); } public BlockWALServiceBuilder direct(boolean direct) { @@ -643,10 +645,10 @@ public BlockWALService build() { BlockWALService blockWALService = new BlockWALService(); WALChannel.WALChannelBuilder walChannelBuilder = WALChannel.builder(blockDevicePath) - .capacity(blockDeviceCapacityWant) - .initBufferSize(initBufferSize) - .maxBufferSize(maxBufferSize) - .recoveryMode(recoveryMode); + .capacity(blockDeviceCapacityWant) + .initBufferSize(initBufferSize) + .maxBufferSize(maxBufferSize) + .recoveryMode(recoveryMode); if (direct != null) { walChannelBuilder.direct(direct); } @@ -662,13 +664,13 @@ public BlockWALService build() { slidingWindowUpperLimit = Math.min(slidingWindowUpperLimit, blockDeviceCapacityWant - WAL_HEADER_TOTAL_CAPACITY); blockWALService.initialWindowSize = slidingWindowInitialSize; blockWALService.slidingWindowService = new SlidingWindowService( - blockWALService.walChannel, - ioThreadNums, - slidingWindowUpperLimit, - slidingWindowScaleUnit, - blockSoftLimit, - writeRateLimit, - blockWALService.flusher() + blockWALService.walChannel, + ioThreadNums, + slidingWindowUpperLimit, + slidingWindowScaleUnit, + blockSoftLimit, + writeRateLimit, + blockWALService.flusher() ); } @@ -687,21 +689,21 @@ public BlockWALService build() { @Override public String toString() { return "BlockWALServiceBuilder{" - + "blockDevicePath='" + blockDevicePath - + ", blockDeviceCapacityWant=" + blockDeviceCapacityWant - + ", direct=" + direct - + ", initBufferSize=" + initBufferSize - + ", maxBufferSize=" + maxBufferSize - + ", ioThreadNums=" + ioThreadNums - + ", slidingWindowInitialSize=" + slidingWindowInitialSize - + ", slidingWindowUpperLimit=" + slidingWindowUpperLimit - + ", slidingWindowScaleUnit=" + slidingWindowScaleUnit - + ", blockSoftLimit=" + blockSoftLimit - + ", writeRateLimit=" + writeRateLimit - + ", nodeId=" + nodeId - + ", epoch=" + epoch - + ", recoveryMode=" + recoveryMode - + '}'; + + "blockDevicePath='" + blockDevicePath + + ", blockDeviceCapacityWant=" + blockDeviceCapacityWant + + ", direct=" + direct + + ", initBufferSize=" + initBufferSize + + ", maxBufferSize=" + maxBufferSize + + ", ioThreadNums=" + ioThreadNums + + ", slidingWindowInitialSize=" + slidingWindowInitialSize + + ", slidingWindowUpperLimit=" + slidingWindowUpperLimit + + ", slidingWindowScaleUnit=" + slidingWindowScaleUnit + + ", blockSoftLimit=" + blockSoftLimit + + ", writeRateLimit=" + writeRateLimit + + ", nodeId=" + nodeId + + ", epoch=" + epoch + + ", recoveryMode=" + recoveryMode + + '}'; } } @@ -718,9 +720,9 @@ record RecoverResultImpl(ByteBuf record, long recordOffset) implements RecoverRe @Override public String toString() { return "RecoverResultImpl{" - + "record=" + record - + ", recordOffset=" + recordOffset - + '}'; + + "record=" + record + + ", recordOffset=" + recordOffset + + '}'; } } @@ -796,7 +798,7 @@ private boolean tryReadNextRecord() { if (firstInvalidOffset == -1 && WALUtil.isAligned(nextRecoverOffset) && nextRecoverOffset != skipRecordAtOffset) { // first invalid offset LOGGER.info("meet the first invalid offset during recovery. offset: {}, window: {}, detail: '{}'", - nextRecoverOffset, windowLength, e.getMessage()); + nextRecoverOffset, windowLength, e.getMessage()); firstInvalidOffset = nextRecoverOffset; } nextRecoverOffset = e.getJumpNextRecoverOffset(); diff --git a/s3stream/src/main/java/com/automq/stream/s3/wal/MemoryWriteAheadLog.java b/s3stream/src/main/java/com/automq/stream/s3/wal/MemoryWriteAheadLog.java index 9245b0a1c..a38b87987 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/wal/MemoryWriteAheadLog.java +++ b/s3stream/src/main/java/com/automq/stream/s3/wal/MemoryWriteAheadLog.java @@ -19,7 +19,6 @@ import com.automq.stream.s3.trace.context.TraceContext; import io.netty.buffer.ByteBuf; - import java.io.IOException; import java.util.Collections; import java.util.Iterator; diff --git a/s3stream/src/main/java/com/automq/stream/s3/wal/SlidingWindowService.java b/s3stream/src/main/java/com/automq/stream/s3/wal/SlidingWindowService.java index f7c798e6d..cc6f94a0d 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/wal/SlidingWindowService.java +++ b/s3stream/src/main/java/com/automq/stream/s3/wal/SlidingWindowService.java @@ -28,9 +28,6 @@ import com.automq.stream.utils.ThreadUtils; import com.automq.stream.utils.Threads; import io.netty.buffer.ByteBuf; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.util.Collection; import java.util.LinkedList; @@ -43,6 +40,8 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static com.automq.stream.s3.wal.BlockWALService.RECORD_HEADER_MAGIC_CODE; import static com.automq.stream.s3.wal.BlockWALService.RECORD_HEADER_SIZE; @@ -102,7 +101,8 @@ public class SlidingWindowService { */ private long lastWriteTimeNanos = 0; - public SlidingWindowService(WALChannel walChannel, int ioThreadNums, long upperLimit, long scaleUnit, long blockSoftLimit, int writeRateLimit, WALHeaderFlusher flusher) { + public SlidingWindowService(WALChannel walChannel, int ioThreadNums, long upperLimit, long scaleUnit, + long blockSoftLimit, int writeRateLimit, WALHeaderFlusher flusher) { this.walChannel = walChannel; this.ioThreadNums = ioThreadNums; this.upperLimit = upperLimit; @@ -120,9 +120,9 @@ public WindowCoreData getWindowCoreData() { public void start(AtomicLong windowMaxLength, long windowStartOffset) { this.windowCoreData = new WindowCoreData(windowMaxLength, windowStartOffset, windowStartOffset); this.ioExecutor = Threads.newFixedThreadPoolWithMonitor(ioThreadNums, - "block-wal-io-thread", false, LOGGER); + "block-wal-io-thread", false, LOGGER); ScheduledExecutorService pollBlockScheduler = Threads.newSingleThreadScheduledExecutor( - ThreadUtils.createThreadFactory("wal-poll-block-thread-%d", false), LOGGER); + ThreadUtils.createThreadFactory("wal-poll-block-thread-%d", false), LOGGER); pollBlockScheduler.scheduleAtFixedRate(this::tryWriteBlock, 0, minWriteIntervalNanos, TimeUnit.NANOSECONDS); initialized.set(true); } @@ -185,7 +185,8 @@ public Lock getBlockLock() { * - creates a new block, sets it as the current block and returns it * Note: this method is NOT thread safe, and it should be called with {@link #blockLock} locked. */ - public Block sealAndNewBlockLocked(Block previousBlock, long minSize, long trimOffset, long recordSectionCapacity) throws OverCapacityException { + public Block sealAndNewBlockLocked(Block previousBlock, long minSize, long trimOffset, + long recordSectionCapacity) throws OverCapacityException { assert initialized(); long startOffset = nextBlockStartOffset(previousBlock); @@ -197,9 +198,9 @@ public Block sealAndNewBlockLocked(Block previousBlock, long minSize, long trimO // Not enough space for this block if (startOffset + minSize - trimOffset > recordSectionCapacity) { LOGGER.warn("failed to allocate write offset as the ring buffer is full: startOffset: {}, minSize: {}, trimOffset: {}, recordSectionCapacity: {}", - startOffset, minSize, trimOffset, recordSectionCapacity); + startOffset, minSize, trimOffset, recordSectionCapacity); throw new OverCapacityException(String.format("failed to allocate write offset: ring buffer is full: startOffset: %d, minSize: %d, trimOffset: %d, recordSectionCapacity: %d", - startOffset, minSize, trimOffset, recordSectionCapacity)); + startOffset, minSize, trimOffset, recordSectionCapacity)); } long maxSize = upperLimit; @@ -362,7 +363,7 @@ private void makeWriteOffsetMatchWindow(long newWindowEndOffset) throws IOExcept } else { // the new window length is bigger than upper limit, reject this write request LOGGER.error("new windows size {} exceeds upper limit {}, reject this write request, window start offset: {}, new window end offset: {}", - newWindowMaxLength, upperLimit, windowStartOffset, newWindowEndOffset); + newWindowMaxLength, upperLimit, windowStartOffset, newWindowEndOffset); throw new OverCapacityException(String.format("new windows size exceeds upper limit %d", upperLimit)); } } @@ -436,12 +437,12 @@ public int getRecordHeaderCRC() { @Override public String toString() { return "RecordHeaderCoreData{" + - "magicCode=" + magicCode0 + - ", recordBodyLength=" + recordBodyLength1 + - ", recordBodyOffset=" + recordBodyOffset2 + - ", recordBodyCRC=" + recordBodyCRC3 + - ", recordHeaderCRC=" + recordHeaderCRC4 + - '}'; + "magicCode=" + magicCode0 + + ", recordBodyLength=" + recordBodyLength1 + + ", recordBodyOffset=" + recordBodyOffset2 + + ", recordBodyCRC=" + recordBodyCRC3 + + ", recordHeaderCRC=" + recordHeaderCRC4 + + '}'; } private ByteBuf marshalHeaderExceptCRC() { diff --git a/s3stream/src/main/java/com/automq/stream/s3/wal/WALHeader.java b/s3stream/src/main/java/com/automq/stream/s3/wal/WALHeader.java index 0c2442456..84f56e078 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/wal/WALHeader.java +++ b/s3stream/src/main/java/com/automq/stream/s3/wal/WALHeader.java @@ -20,7 +20,6 @@ import com.automq.stream.s3.DirectByteBufAlloc; import com.automq.stream.s3.wal.util.WALUtil; import io.netty.buffer.ByteBuf; - import java.util.concurrent.atomic.AtomicLong; /** @@ -54,14 +53,14 @@ class WALHeader { public static final int WAL_HEADER_MAGIC_CODE = 0x12345678; public static final int WAL_HEADER_SIZE = 4 // magic code - + 8 // capacity - + 8 // trim offset - + 8 // last write timestamp - + 8 // sliding window max length - + 4 // shutdown type - + 4 // node id - + 4 // node epoch - + 8; // crc + + 8 // capacity + + 8 // trim offset + + 8 // last write timestamp + + 8 // sliding window max length + + 4 // shutdown type + + 4 // node id + + 4 // node epoch + + 8; // crc public static final int WAL_HEADER_WITHOUT_CRC_SIZE = WAL_HEADER_SIZE - 4; private final AtomicLong trimOffset2 = new AtomicLong(-1); private final AtomicLong flushedTrimOffset = new AtomicLong(0); @@ -176,16 +175,16 @@ public WALHeader setEpoch(long epoch) { @Override public String toString() { return "WALHeader{" - + "magicCode=" + magicCode0 - + ", capacity=" + capacity1 - + ", trimOffset=" + trimOffset2 - + ", lastWriteTimestamp=" + lastWriteTimestamp3 - + ", slidingWindowMaxLength=" + slidingWindowMaxLength4 - + ", shutdownType=" + shutdownType5 - + ", nodeId=" + nodeId6 - + ", epoch=" + epoch7 - + ", crc=" + crc8 - + '}'; + + "magicCode=" + magicCode0 + + ", capacity=" + capacity1 + + ", trimOffset=" + trimOffset2 + + ", lastWriteTimestamp=" + lastWriteTimestamp3 + + ", slidingWindowMaxLength=" + slidingWindowMaxLength4 + + ", shutdownType=" + shutdownType5 + + ", nodeId=" + nodeId6 + + ", epoch=" + epoch7 + + ", crc=" + crc8 + + '}'; } private ByteBuf marshalHeaderExceptCRC() { diff --git a/s3stream/src/main/java/com/automq/stream/s3/wal/WriteAheadLog.java b/s3stream/src/main/java/com/automq/stream/s3/wal/WriteAheadLog.java index 4b301464a..a1fbedf45 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/wal/WriteAheadLog.java +++ b/s3stream/src/main/java/com/automq/stream/s3/wal/WriteAheadLog.java @@ -17,10 +17,8 @@ package com.automq.stream.s3.wal; - import com.automq.stream.s3.trace.context.TraceContext; import io.netty.buffer.ByteBuf; - import java.io.IOException; import java.util.Iterator; import java.util.concurrent.CompletableFuture; @@ -29,7 +27,6 @@ public interface WriteAheadLog { WriteAheadLog start() throws IOException; - void shutdownGracefully(); /** diff --git a/s3stream/src/main/java/com/automq/stream/s3/wal/benchmark/WriteBench.java b/s3stream/src/main/java/com/automq/stream/s3/wal/benchmark/WriteBench.java index 23c992e92..28e26393b 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/wal/benchmark/WriteBench.java +++ b/s3stream/src/main/java/com/automq/stream/s3/wal/benchmark/WriteBench.java @@ -25,12 +25,6 @@ import com.automq.stream.utils.Threads; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; -import net.sourceforge.argparse4j.ArgumentParsers; -import net.sourceforge.argparse4j.helper.HelpScreenException; -import net.sourceforge.argparse4j.inf.ArgumentParser; -import net.sourceforge.argparse4j.inf.ArgumentParserException; -import net.sourceforge.argparse4j.inf.Namespace; - import java.io.File; import java.io.IOException; import java.util.Iterator; @@ -44,6 +38,11 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.LockSupport; import java.util.concurrent.locks.ReentrantLock; +import net.sourceforge.argparse4j.ArgumentParsers; +import net.sourceforge.argparse4j.helper.HelpScreenException; +import net.sourceforge.argparse4j.inf.ArgumentParser; +import net.sourceforge.argparse4j.inf.ArgumentParserException; +import net.sourceforge.argparse4j.inf.Namespace; /** * WriteBench is a tool for benchmarking write performance of {@link BlockWALService} @@ -110,15 +109,15 @@ private static void resetWALHeader(String path) throws IOException { private static void logIt(Config config, Stat stat) { ScheduledExecutorService statExecutor = Threads.newSingleThreadScheduledExecutor( - ThreadUtils.createThreadFactory("stat-thread-%d", true), null); + ThreadUtils.createThreadFactory("stat-thread-%d", true), null); statExecutor.scheduleAtFixedRate(() -> { Stat.Result result = stat.reset(); if (0 != result.count()) { System.out.printf("Append task | Append Rate %d msg/s %d KB/s | Avg Latency %.3f ms | Max Latency %.3f ms\n", - TimeUnit.SECONDS.toNanos(1) * result.count() / result.elapsedTimeNanos(), - TimeUnit.SECONDS.toNanos(1) * (result.count() * config.recordSizeBytes) / result.elapsedTimeNanos() / 1024, - (double) result.costNanos() / TimeUnit.MILLISECONDS.toNanos(1) / result.count(), - (double) result.maxCostNanos() / TimeUnit.MILLISECONDS.toNanos(1)); + TimeUnit.SECONDS.toNanos(1) * result.count() / result.elapsedTimeNanos(), + TimeUnit.SECONDS.toNanos(1) * (result.count() * config.recordSizeBytes) / result.elapsedTimeNanos() / 1024, + (double) result.costNanos() / TimeUnit.MILLISECONDS.toNanos(1) / result.count(), + (double) result.maxCostNanos() / TimeUnit.MILLISECONDS.toNanos(1)); } }, LOG_INTERVAL_SECONDS, LOG_INTERVAL_SECONDS, TimeUnit.SECONDS); } @@ -127,7 +126,7 @@ private void run(Config config) { System.out.println("Starting benchmark"); ExecutorService executor = Threads.newFixedThreadPool( - config.threads, ThreadUtils.createThreadFactory("append-thread-%d", false), null); + config.threads, ThreadUtils.createThreadFactory("append-thread-%d", false), null); AppendTaskConfig appendTaskConfig = new AppendTaskConfig(config); Stat stat = new Stat(); runTrimTask(); @@ -158,7 +157,7 @@ private void run(Config config) { private void runTrimTask() { ScheduledExecutorService trimExecutor = Threads.newSingleThreadScheduledExecutor( - ThreadUtils.createThreadFactory("trim-thread-%d", true), null); + ThreadUtils.createThreadFactory("trim-thread-%d", true), null); trimExecutor.scheduleAtFixedRate(() -> { try { log.trim(trimOffset.get()); @@ -248,37 +247,37 @@ static class Config { static ArgumentParser parser() { ArgumentParser parser = ArgumentParsers - .newFor("WriteBench") - .build() - .defaultHelp(true) - .description("Benchmark write performance of BlockWALService"); + .newFor("WriteBench") + .build() + .defaultHelp(true) + .description("Benchmark write performance of BlockWALService"); parser.addArgument("-p", "--path") - .required(true) - .help("Path of the WAL file"); + .required(true) + .help("Path of the WAL file"); parser.addArgument("-c", "--capacity") - .type(Long.class) - .setDefault((long) 1 << 30) - .help("Capacity of the WAL in bytes"); + .type(Long.class) + .setDefault((long) 1 << 30) + .help("Capacity of the WAL in bytes"); parser.addArgument("-d", "--depth") - .type(Integer.class) - .help("IO depth of the WAL"); + .type(Integer.class) + .help("IO depth of the WAL"); parser.addArgument("--threads") - .type(Integer.class) - .setDefault(1) - .help("Number of threads to use to write"); + .type(Integer.class) + .setDefault(1) + .help("Number of threads to use to write"); parser.addArgument("--throughput") - .type(Integer.class) - .setDefault(1 << 20) - .help("Expected throughput in total in bytes per second"); + .type(Integer.class) + .setDefault(1 << 20) + .help("Expected throughput in total in bytes per second"); parser.addArgument("--record-size") - .dest("recordSize") - .type(Integer.class) - .setDefault(1 << 10) - .help("Size of each record in bytes"); + .dest("recordSize") + .type(Integer.class) + .setDefault(1 << 10) + .help("Size of each record in bytes"); parser.addArgument("--duration") - .type(Long.class) - .setDefault(60L) - .help("Duration of the benchmark in seconds"); + .type(Long.class) + .setDefault(60L) + .help("Duration of the benchmark in seconds"); return parser; } } diff --git a/s3stream/src/main/java/com/automq/stream/s3/wal/util/WALBlockDeviceChannel.java b/s3stream/src/main/java/com/automq/stream/s3/wal/util/WALBlockDeviceChannel.java index f1ecf40fc..3a421f92f 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/wal/util/WALBlockDeviceChannel.java +++ b/s3stream/src/main/java/com/automq/stream/s3/wal/util/WALBlockDeviceChannel.java @@ -23,13 +23,12 @@ import com.automq.stream.thirdparty.moe.cnkirito.kdio.DirectIOUtils; import com.automq.stream.thirdparty.moe.cnkirito.kdio.DirectRandomAccessFile; import io.netty.buffer.ByteBuf; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.util.concurrent.ExecutionException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static com.automq.stream.s3.Constants.CAPACITY_NOT_SET; @@ -68,7 +67,8 @@ public WALBlockDeviceChannel(String path, long capacityWant) { this(path, capacityWant, 0, 0, false); } - public WALBlockDeviceChannel(String path, long capacityWant, int initTempBufferSize, int maxTempBufferSize, boolean recoveryMode) { + public WALBlockDeviceChannel(String path, long capacityWant, int initTempBufferSize, int maxTempBufferSize, + boolean recoveryMode) { this.path = path; this.recoveryMode = recoveryMode; if (recoveryMode) { @@ -102,7 +102,7 @@ public static String checkAvailable(String path) { } if (!DirectIOUtils.allocatorAvailable()) { return "java.nio.DirectByteBuffer.(long, int) not available." + - " Add --add-opens=java.base/java.nio=ALL-UNNAMED and -Dio.netty.tryReflectionSetAccessible=true to JVM options may fix this."; + " Add --add-opens=java.base/java.nio=ALL-UNNAMED and -Dio.netty.tryReflectionSetAccessible=true to JVM options may fix this."; } if (!path.startsWith(DEVICE_PREFIX) && !tryOpenFileWithDirectIO(String.format(CHECK_DIRECT_IO_AVAILABLE_FORMAT, path))) { return "O_DIRECT not supported by the file system, path: " + path; diff --git a/s3stream/src/main/java/com/automq/stream/s3/wal/util/WALChannel.java b/s3stream/src/main/java/com/automq/stream/s3/wal/util/WALChannel.java index f9dc718ec..1fefc90eb 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/wal/util/WALChannel.java +++ b/s3stream/src/main/java/com/automq/stream/s3/wal/util/WALChannel.java @@ -20,11 +20,10 @@ import com.automq.stream.s3.wal.WALCapacityMismatchException; import com.automq.stream.s3.wal.WALNotInitializedException; import io.netty.buffer.ByteBuf; +import java.io.IOException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; - /** * There are two implementations of WALChannel: * 1. WALFileChannel based on file system, which calls fsync after each write to ensure data is flushed to disk. diff --git a/s3stream/src/main/java/com/automq/stream/s3/wal/util/WALFileChannel.java b/s3stream/src/main/java/com/automq/stream/s3/wal/util/WALFileChannel.java index 9abaed1f2..20c5148df 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/wal/util/WALFileChannel.java +++ b/s3stream/src/main/java/com/automq/stream/s3/wal/util/WALFileChannel.java @@ -20,7 +20,6 @@ import com.automq.stream.s3.wal.WALCapacityMismatchException; import com.automq.stream.s3.wal.WALNotInitializedException; import io.netty.buffer.ByteBuf; - import java.io.File; import java.io.IOException; import java.io.RandomAccessFile; diff --git a/s3stream/src/main/java/com/automq/stream/s3/wal/util/WALUtil.java b/s3stream/src/main/java/com/automq/stream/s3/wal/util/WALUtil.java index 802f0601d..c83c9d112 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/wal/util/WALUtil.java +++ b/s3stream/src/main/java/com/automq/stream/s3/wal/util/WALUtil.java @@ -20,7 +20,6 @@ import com.automq.stream.utils.CommandResult; import com.automq.stream.utils.CommandUtils; import io.netty.buffer.ByteBuf; - import java.io.File; import java.io.IOException; import java.io.RandomAccessFile; @@ -30,8 +29,8 @@ public class WALUtil { public static final int BLOCK_SIZE = Integer.parseInt(System.getProperty( - "automq.ebswal.blocksize", - "4096" + "automq.ebswal.blocksize", + "4096" )); /** @@ -108,7 +107,7 @@ public static void createFile(String path, long length) throws IOException { * Get the capacity of the given block device. */ public static long getBlockDeviceCapacity(String path) throws ExecutionException { - String[] cmd = new String[]{ + String[] cmd = new String[] { "lsblk", "--bytes", "--nodeps", diff --git a/s3stream/src/main/java/com/automq/stream/thirdparty/moe/cnkirito/kdio/DirectChannel.java b/s3stream/src/main/java/com/automq/stream/thirdparty/moe/cnkirito/kdio/DirectChannel.java index 33e4c88f3..17d83bb52 100755 --- a/s3stream/src/main/java/com/automq/stream/thirdparty/moe/cnkirito/kdio/DirectChannel.java +++ b/s3stream/src/main/java/com/automq/stream/thirdparty/moe/cnkirito/kdio/DirectChannel.java @@ -23,12 +23,8 @@ public interface DirectChannel extends Channel { /** * Writes from the src buffer into this channel at position. * - * @param src - * The {@link ByteBuffer} to write from - * - * @param position - * The position within the file at which to start writing - * + * @param src The {@link ByteBuffer} to write from + * @param position The position within the file at which to start writing * @return How many bytes were written from src into the file * @throws IOException */ @@ -37,12 +33,8 @@ public interface DirectChannel extends Channel { /** * Reads from this channel into the dst buffer from position. * - * @param dst - * The {@link ByteBuffer} to read into - * - * @param position - * The position within the file at which to start reading - * + * @param dst The {@link ByteBuffer} to read into + * @param position The position within the file at which to start reading * @return How many bytes were placed into dst * @throws IOException */ @@ -62,9 +54,7 @@ public interface DirectChannel extends Channel { * Truncates this file's length to fileLength. * * @param fileLength The length to which to truncate - * * @return This UnsafeByteAlignedChannel - * * @throws IOException */ DirectChannel truncate(long fileLength) throws IOException; diff --git a/s3stream/src/main/java/com/automq/stream/thirdparty/moe/cnkirito/kdio/DirectChannelImpl.java b/s3stream/src/main/java/com/automq/stream/thirdparty/moe/cnkirito/kdio/DirectChannelImpl.java index 7ba12863b..f8eb956c3 100755 --- a/s3stream/src/main/java/com/automq/stream/thirdparty/moe/cnkirito/kdio/DirectChannelImpl.java +++ b/s3stream/src/main/java/com/automq/stream/thirdparty/moe/cnkirito/kdio/DirectChannelImpl.java @@ -24,9 +24,9 @@ public class DirectChannelImpl implements DirectChannel { private final DirectIOLib lib; private final int fd; + private final boolean isReadOnly; private boolean isOpen; private long fileLength; - private final boolean isReadOnly; private DirectChannelImpl(DirectIOLib lib, int fd, long fileLength, boolean readOnly) { this.lib = lib; @@ -87,7 +87,7 @@ public DirectChannel truncate(final long length) throws IOException { ensureWritable(); if (DirectIOLib.ftruncate(fd, length) < 0) { throw new IOException("Error during truncate on descriptor " + fd + ": " + - DirectIOLib.getLastError()); + DirectIOLib.getLastError()); } fileLength = length; return this; @@ -103,7 +103,6 @@ public int getFD() { return fd; } - @Override public boolean isOpen() { return isOpen; @@ -122,7 +121,7 @@ public void close() throws IOException { isOpen = false; if (lib.close(fd) < 0) { throw new IOException("Error closing file with descriptor " + fd + ": " + - DirectIOLib.getLastError()); + DirectIOLib.getLastError()); } } } diff --git a/s3stream/src/main/java/com/automq/stream/thirdparty/moe/cnkirito/kdio/DirectIOLib.java b/s3stream/src/main/java/com/automq/stream/thirdparty/moe/cnkirito/kdio/DirectIOLib.java index 7002401df..2d2574cdd 100755 --- a/s3stream/src/main/java/com/automq/stream/thirdparty/moe/cnkirito/kdio/DirectIOLib.java +++ b/s3stream/src/main/java/com/automq/stream/thirdparty/moe/cnkirito/kdio/DirectIOLib.java @@ -21,13 +21,12 @@ import com.sun.jna.Pointer; import com.sun.jna.ptr.PointerByReference; import io.netty.util.internal.PlatformDependent; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Class containing native hooks and utility methods for performing direct I/O, using @@ -38,9 +37,9 @@ * when given a string for a file or directory. */ public class DirectIOLib { + static final int PC_REC_XFER_ALIGN = 0x11; private static final Logger logger = LoggerFactory.getLogger(DirectIOLib.class); public static boolean binit; - static final int PC_REC_XFER_ALIGN = 0x11; static { binit = false; @@ -97,7 +96,6 @@ public DirectIOLib(int fsBlockSize) { this.fsBlockNotMask = -((long) fsBlockSize); } - /** * Static method to register JNA hooks for doing direct I/O * @@ -108,7 +106,7 @@ public static DirectIOLib getLibForPath(String workingDir) { int fsBlockSize = initilizeSoftBlockSize(workingDir); if (fsBlockSize == -1) { logger.warn("O_DIRECT support non available on your version of Linux (" + System.getProperty("os.version") + "), " + - "please upgrade your kernel in order to use jaydio."); + "please upgrade your kernel in order to use jaydio."); return null; } return new DirectIOLib(fsBlockSize); @@ -156,7 +154,6 @@ private static int initilizeSoftBlockSize(String fileOrDir) { return fsBlockSize; } - // -- Java interfaces to native methods /** @@ -217,7 +214,6 @@ public static int lcm(long x, long y) { */ public static native int posix_memalign(PointerByReference memptr, NativeLong alignment, NativeLong size); - // -- alignment logic utility methods /** diff --git a/s3stream/src/main/java/com/automq/stream/thirdparty/moe/cnkirito/kdio/DirectIOUtils.java b/s3stream/src/main/java/com/automq/stream/thirdparty/moe/cnkirito/kdio/DirectIOUtils.java index a52043bb6..586dde91d 100755 --- a/s3stream/src/main/java/com/automq/stream/thirdparty/moe/cnkirito/kdio/DirectIOUtils.java +++ b/s3stream/src/main/java/com/automq/stream/thirdparty/moe/cnkirito/kdio/DirectIOUtils.java @@ -19,7 +19,6 @@ import com.sun.jna.Pointer; import com.sun.jna.ptr.PointerByReference; import io.netty.util.internal.PlatformDependent; - import java.nio.ByteBuffer; public class DirectIOUtils { @@ -38,7 +37,7 @@ public class DirectIOUtils { public static ByteBuffer allocateForDirectIO(DirectIOLib lib, int capacity) { if (capacity % lib.blockSize() > 0) { throw new IllegalArgumentException("Capacity (" + capacity + ") must be a multiple" - + "of the block size (" + lib.blockSize() + ")"); + + "of the block size (" + lib.blockSize() + ")"); } NativeLong blockSize = new NativeLong(lib.blockSize()); PointerByReference pointerToPointer = new PointerByReference(); diff --git a/s3stream/src/main/java/com/automq/stream/thirdparty/moe/cnkirito/kdio/DirectRandomAccessFile.java b/s3stream/src/main/java/com/automq/stream/thirdparty/moe/cnkirito/kdio/DirectRandomAccessFile.java index aabad015a..e77b280f6 100755 --- a/s3stream/src/main/java/com/automq/stream/thirdparty/moe/cnkirito/kdio/DirectRandomAccessFile.java +++ b/s3stream/src/main/java/com/automq/stream/thirdparty/moe/cnkirito/kdio/DirectRandomAccessFile.java @@ -24,22 +24,18 @@ /** * Class to emulate the behavior of {@link RandomAccessFile}, but using direct I/O. - * */ public class DirectRandomAccessFile implements Closeable { private final DirectChannel channel; - /** * @param file The file to open - * * @param mode Either "rw" or "r", depending on whether this file is read only - * * @throws IOException */ public DirectRandomAccessFile(File file, String mode) - throws IOException { + throws IOException { boolean readOnly = false; if ("r".equals(mode)) { @@ -60,7 +56,6 @@ public void close() throws IOException { channel.close(); } - public int write(ByteBuffer src, long position) throws IOException { return channel.write(src, position); } diff --git a/s3stream/src/main/java/com/automq/stream/utils/AsyncRateLimiter.java b/s3stream/src/main/java/com/automq/stream/utils/AsyncRateLimiter.java index a8f9e3e60..af0cff6e8 100644 --- a/s3stream/src/main/java/com/automq/stream/utils/AsyncRateLimiter.java +++ b/s3stream/src/main/java/com/automq/stream/utils/AsyncRateLimiter.java @@ -18,15 +18,14 @@ package com.automq.stream.utils; import com.google.common.util.concurrent.RateLimiter; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.Queue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; @SuppressWarnings("UnstableApiUsage") public class AsyncRateLimiter { diff --git a/s3stream/src/main/java/com/automq/stream/utils/ByteBufInputStream.java b/s3stream/src/main/java/com/automq/stream/utils/ByteBufInputStream.java index 61c809660..f2338316c 100644 --- a/s3stream/src/main/java/com/automq/stream/utils/ByteBufInputStream.java +++ b/s3stream/src/main/java/com/automq/stream/utils/ByteBufInputStream.java @@ -17,7 +17,6 @@ package com.automq.stream.utils; import io.netty.buffer.ByteBuf; - import java.io.InputStream; /** diff --git a/s3stream/src/main/java/com/automq/stream/utils/CloseableIterator.java b/s3stream/src/main/java/com/automq/stream/utils/CloseableIterator.java index 2e6bc4390..e3c5299fc 100644 --- a/s3stream/src/main/java/com/automq/stream/utils/CloseableIterator.java +++ b/s3stream/src/main/java/com/automq/stream/utils/CloseableIterator.java @@ -21,17 +21,16 @@ /** * Iterators that need to be closed in order to release resources should implement this interface. - * + *

    * Warning: before implementing this interface, consider if there are better options. The chance of misuse is * a bit high since people are used to iterating without closing. */ public interface CloseableIterator extends Iterator, Closeable { - void close(); - static CloseableIterator wrap(Iterator inner) { return new CloseableIterator() { @Override - public void close() {} + public void close() { + } @Override public boolean hasNext() { @@ -49,4 +48,6 @@ public void remove() { } }; } + + void close(); } diff --git a/s3stream/src/main/java/com/automq/stream/utils/CommandResult.java b/s3stream/src/main/java/com/automq/stream/utils/CommandResult.java index f7c8c6246..36267f598 100644 --- a/s3stream/src/main/java/com/automq/stream/utils/CommandResult.java +++ b/s3stream/src/main/java/com/automq/stream/utils/CommandResult.java @@ -50,9 +50,9 @@ public String stderr() { @Override public String toString() { return "CommandResult{" + - "code=" + code + - ", stdout='" + stdout + '\'' + - ", stderr='" + stderr + '\'' + - '}'; + "code=" + code + + ", stdout='" + stdout + '\'' + + ", stderr='" + stderr + '\'' + + '}'; } } diff --git a/s3stream/src/main/java/com/automq/stream/utils/FutureTicker.java b/s3stream/src/main/java/com/automq/stream/utils/FutureTicker.java index 1b4b8f090..6dee43183 100644 --- a/s3stream/src/main/java/com/automq/stream/utils/FutureTicker.java +++ b/s3stream/src/main/java/com/automq/stream/utils/FutureTicker.java @@ -44,8 +44,9 @@ public class FutureTicker { /** * Create a ticker with a delay and a executor - * @param delay the delay - * @param unit the time unit of the delay + * + * @param delay the delay + * @param unit the time unit of the delay * @param executor the executor, the {@link CompletableFuture} returned by {@link #tick()} will be completed by this executor */ public FutureTicker(long delay, TimeUnit unit, Executor executor) { @@ -67,7 +68,8 @@ public CompletableFuture tick() { private synchronized CompletableFuture maybeNextTick() { if (currentTick.isDone()) { // a future which will complete after delay - currentTick = CompletableFuture.runAsync(() -> { }, delayedExecutor); + currentTick = CompletableFuture.runAsync(() -> { + }, delayedExecutor); } return currentTick; } diff --git a/s3stream/src/main/java/com/automq/stream/utils/FutureUtil.java b/s3stream/src/main/java/com/automq/stream/utils/FutureUtil.java index 3017d5936..1ee5040d0 100644 --- a/s3stream/src/main/java/com/automq/stream/utils/FutureUtil.java +++ b/s3stream/src/main/java/com/automq/stream/utils/FutureUtil.java @@ -17,13 +17,12 @@ package com.automq.stream.utils; -import org.slf4j.Logger; - import java.util.Collection; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; import java.util.function.Supplier; +import org.slf4j.Logger; public class FutureUtil { public static CompletableFuture failedFuture(Throwable ex) { diff --git a/s3stream/src/main/java/com/automq/stream/utils/LogContext.java b/s3stream/src/main/java/com/automq/stream/utils/LogContext.java index 346be9afb..50bacc73a 100644 --- a/s3stream/src/main/java/com/automq/stream/utils/LogContext.java +++ b/s3stream/src/main/java/com/automq/stream/utils/LogContext.java @@ -148,14 +148,14 @@ public void trace(String message) { @Override public void trace(String format, Object arg) { if (logger.isTraceEnabled()) { - writeLog(null, LocationAwareLogger.TRACE_INT, format, new Object[]{arg}, null); + writeLog(null, LocationAwareLogger.TRACE_INT, format, new Object[] {arg}, null); } } @Override public void trace(String format, Object arg1, Object arg2) { if (logger.isTraceEnabled()) { - writeLog(null, LocationAwareLogger.TRACE_INT, format, new Object[]{arg1, arg2}, null); + writeLog(null, LocationAwareLogger.TRACE_INT, format, new Object[] {arg1, arg2}, null); } } @@ -183,14 +183,14 @@ public void trace(Marker marker, String msg) { @Override public void trace(Marker marker, String format, Object arg) { if (logger.isTraceEnabled()) { - writeLog(marker, LocationAwareLogger.TRACE_INT, format, new Object[]{arg}, null); + writeLog(marker, LocationAwareLogger.TRACE_INT, format, new Object[] {arg}, null); } } @Override public void trace(Marker marker, String format, Object arg1, Object arg2) { if (logger.isTraceEnabled()) { - writeLog(marker, LocationAwareLogger.TRACE_INT, format, new Object[]{arg1, arg2}, null); + writeLog(marker, LocationAwareLogger.TRACE_INT, format, new Object[] {arg1, arg2}, null); } } @@ -218,14 +218,14 @@ public void debug(String message) { @Override public void debug(String format, Object arg) { if (logger.isDebugEnabled()) { - writeLog(null, LocationAwareLogger.DEBUG_INT, format, new Object[]{arg}, null); + writeLog(null, LocationAwareLogger.DEBUG_INT, format, new Object[] {arg}, null); } } @Override public void debug(String format, Object arg1, Object arg2) { if (logger.isDebugEnabled()) { - writeLog(null, LocationAwareLogger.DEBUG_INT, format, new Object[]{arg1, arg2}, null); + writeLog(null, LocationAwareLogger.DEBUG_INT, format, new Object[] {arg1, arg2}, null); } } @@ -253,14 +253,14 @@ public void debug(Marker marker, String msg) { @Override public void debug(Marker marker, String format, Object arg) { if (logger.isDebugEnabled()) { - writeLog(marker, LocationAwareLogger.DEBUG_INT, format, new Object[]{arg}, null); + writeLog(marker, LocationAwareLogger.DEBUG_INT, format, new Object[] {arg}, null); } } @Override public void debug(Marker marker, String format, Object arg1, Object arg2) { if (logger.isDebugEnabled()) { - writeLog(marker, LocationAwareLogger.DEBUG_INT, format, new Object[]{arg1, arg2}, null); + writeLog(marker, LocationAwareLogger.DEBUG_INT, format, new Object[] {arg1, arg2}, null); } } @@ -285,12 +285,12 @@ public void warn(String message) { @Override public void warn(String format, Object arg) { - writeLog(null, LocationAwareLogger.WARN_INT, format, new Object[]{arg}, null); + writeLog(null, LocationAwareLogger.WARN_INT, format, new Object[] {arg}, null); } @Override public void warn(String message, Object arg1, Object arg2) { - writeLog(null, LocationAwareLogger.WARN_INT, message, new Object[]{arg1, arg2}, null); + writeLog(null, LocationAwareLogger.WARN_INT, message, new Object[] {arg1, arg2}, null); } @Override @@ -310,12 +310,12 @@ public void warn(Marker marker, String msg) { @Override public void warn(Marker marker, String format, Object arg) { - writeLog(marker, LocationAwareLogger.WARN_INT, format, new Object[]{arg}, null); + writeLog(marker, LocationAwareLogger.WARN_INT, format, new Object[] {arg}, null); } @Override public void warn(Marker marker, String format, Object arg1, Object arg2) { - writeLog(marker, LocationAwareLogger.WARN_INT, format, new Object[]{arg1, arg2}, null); + writeLog(marker, LocationAwareLogger.WARN_INT, format, new Object[] {arg1, arg2}, null); } @Override @@ -335,12 +335,12 @@ public void error(String message) { @Override public void error(String format, Object arg) { - writeLog(null, LocationAwareLogger.ERROR_INT, format, new Object[]{arg}, null); + writeLog(null, LocationAwareLogger.ERROR_INT, format, new Object[] {arg}, null); } @Override public void error(String format, Object arg1, Object arg2) { - writeLog(null, LocationAwareLogger.ERROR_INT, format, new Object[]{arg1, arg2}, null); + writeLog(null, LocationAwareLogger.ERROR_INT, format, new Object[] {arg1, arg2}, null); } @Override @@ -360,12 +360,12 @@ public void error(Marker marker, String msg) { @Override public void error(Marker marker, String format, Object arg) { - writeLog(marker, LocationAwareLogger.ERROR_INT, format, new Object[]{arg}, null); + writeLog(marker, LocationAwareLogger.ERROR_INT, format, new Object[] {arg}, null); } @Override public void error(Marker marker, String format, Object arg1, Object arg2) { - writeLog(marker, LocationAwareLogger.ERROR_INT, format, new Object[]{arg1, arg2}, null); + writeLog(marker, LocationAwareLogger.ERROR_INT, format, new Object[] {arg1, arg2}, null); } @Override @@ -385,12 +385,12 @@ public void info(String msg) { @Override public void info(String format, Object arg) { - writeLog(null, LocationAwareLogger.INFO_INT, format, new Object[]{arg}, null); + writeLog(null, LocationAwareLogger.INFO_INT, format, new Object[] {arg}, null); } @Override public void info(String format, Object arg1, Object arg2) { - writeLog(null, LocationAwareLogger.INFO_INT, format, new Object[]{arg1, arg2}, null); + writeLog(null, LocationAwareLogger.INFO_INT, format, new Object[] {arg1, arg2}, null); } @Override @@ -410,12 +410,12 @@ public void info(Marker marker, String msg) { @Override public void info(Marker marker, String format, Object arg) { - writeLog(marker, LocationAwareLogger.INFO_INT, format, new Object[]{arg}, null); + writeLog(marker, LocationAwareLogger.INFO_INT, format, new Object[] {arg}, null); } @Override public void info(Marker marker, String format, Object arg1, Object arg2) { - writeLog(marker, LocationAwareLogger.INFO_INT, format, new Object[]{arg1, arg2}, null); + writeLog(marker, LocationAwareLogger.INFO_INT, format, new Object[] {arg1, arg2}, null); } @Override diff --git a/s3stream/src/main/java/com/automq/stream/utils/S3Utils.java b/s3stream/src/main/java/com/automq/stream/utils/S3Utils.java index ea41f247f..6a2da01df 100644 --- a/s3stream/src/main/java/com/automq/stream/utils/S3Utils.java +++ b/s3stream/src/main/java/com/automq/stream/utils/S3Utils.java @@ -21,6 +21,17 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.CompositeByteBuf; import io.netty.buffer.Unpooled; +import java.net.URI; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Date; +import java.util.List; +import java.util.Objects; +import java.util.Random; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import org.apache.commons.lang3.StringUtils; import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; @@ -42,23 +53,12 @@ import software.amazon.awssdk.services.s3.model.UploadPartRequest; import software.amazon.awssdk.services.s3.model.UploadPartResponse; -import java.net.URI; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.Duration; -import java.util.ArrayList; -import java.util.Date; -import java.util.List; -import java.util.Objects; -import java.util.Random; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; - public class S3Utils { /** * Check s3 access with context. * This method is generally used to help users figure out problems in using S3. + * * @param context s3 context. */ public static void checkS3Access(S3Context context) { @@ -91,6 +91,36 @@ public static void checkS3Access(S3Context context) { } + private static String range(long start, long end) { + if (end == -1L) { + return "bytes=" + start + "-"; + } + return "bytes=" + start + "-" + end; + } + + private static S3AsyncClient newS3AsyncClient(String endpoint, String region, boolean forcePathStyle, + String accessKey, String secretKey) { + S3AsyncClientBuilder builder = S3AsyncClient.builder().region(Region.of(region)); + if (StringUtils.isNotBlank(endpoint)) { + builder.endpointOverride(URI.create(endpoint)); + } + builder.serviceConfiguration(c -> c.pathStyleAccessEnabled(forcePathStyle)); + builder.credentialsProvider(StaticCredentialsProvider.create(AwsBasicCredentials.create(accessKey, secretKey))); + builder.overrideConfiguration(b -> b.apiCallTimeout(Duration.ofMinutes(1)) + .apiCallAttemptTimeout(Duration.ofSeconds(30))); + return builder.build(); + } + + private static String hideSecret(String secret) { + if (secret == null) { + return null; + } + if (secret.length() < 6) { + return "*".repeat(secret.length()); + } + return secret.substring(0, 3) + "*".repeat(secret.length() - 6) + secret.substring(secret.length() - 3); + } + private static abstract class S3CheckTask implements AutoCloseable { protected final S3AsyncClient client; protected final String bucketName; @@ -102,6 +132,16 @@ public S3CheckTask(S3Context context, String taskName) { this.taskName = taskName; } + protected static void showErrorInfo(Exception e) { + if (e.getCause() instanceof S3Exception se) { + System.err.println("get S3 exception: "); + se.printStackTrace(); + } else { + System.err.println("get other exception: "); + e.printStackTrace(); + } + } + protected void run() { } @@ -115,23 +155,6 @@ public void close() { client.close(); } } - - protected static void showErrorInfo(Exception e) { - if (e.getCause() instanceof S3Exception se) { - System.err.println("get S3 exception: "); - se.printStackTrace(); - } else { - System.err.println("get other exception: "); - e.printStackTrace(); - } - } - } - - private static String range(long start, long end) { - if (end == -1L) { - return "bytes=" + start + "-"; - } - return "bytes=" + start + "-" + end; } private static class MultipartObjectOperationTask extends ObjectOperationTask { @@ -186,7 +209,8 @@ public void run() { } } - private CompletableFuture createMultipartUpload(S3AsyncClient writeS3Client, String bucketName, String path) { + private CompletableFuture createMultipartUpload(S3AsyncClient writeS3Client, String bucketName, + String path) { CompletableFuture cf = new CompletableFuture<>(); CreateMultipartUploadRequest request = CreateMultipartUploadRequest.builder().bucket(bucketName).key(path).build(); writeS3Client.createMultipartUpload(request).thenAccept(createMultipartUploadResponse -> { @@ -200,7 +224,8 @@ private CompletableFuture createMultipartUpload(S3AsyncClient writeS3Cli return cf; } - public CompletableFuture completeMultipartUpload(S3AsyncClient writeS3Client, String path, String bucket, String uploadId, List parts) { + public CompletableFuture completeMultipartUpload(S3AsyncClient writeS3Client, String path, String bucket, + String uploadId, List parts) { CompletableFuture cf = new CompletableFuture<>(); CompletedMultipartUpload multipartUpload = CompletedMultipartUpload.builder().parts(parts).build(); CompleteMultipartUploadRequest request = CompleteMultipartUploadRequest.builder().bucket(bucket).key(path).uploadId(uploadId).multipartUpload(multipartUpload).build(); @@ -216,16 +241,18 @@ public CompletableFuture completeMultipartUpload(S3AsyncClient writeS3Clie return cf; } - private CompletableFuture writePart(String uploadId, String path, String bucket, ByteBuf data, int partNum) { + private CompletableFuture writePart(String uploadId, String path, String bucket, ByteBuf data, + int partNum) { CompletableFuture cf = new CompletableFuture<>(); uploadPart(client, cf, path, uploadId, partNum, bucket, data); return cf; } - private void uploadPart(S3AsyncClient writeS3Client, CompletableFuture cf, String path, String uploadId, int partNumber, String bucket, ByteBuf part) { + private void uploadPart(S3AsyncClient writeS3Client, CompletableFuture cf, String path, + String uploadId, int partNumber, String bucket, ByteBuf part) { AsyncRequestBody body = AsyncRequestBody.fromByteBuffersUnsafe(part.nioBuffers()); UploadPartRequest request = UploadPartRequest.builder().bucket(bucket).key(path).uploadId(uploadId) - .partNumber(partNumber).build(); + .partNumber(partNumber).build(); CompletableFuture uploadPartCf = writeS3Client.uploadPart(request, body); uploadPartCf.thenAccept(uploadPartResponse -> { CompletedPart completedPart = CompletedPart.builder().partNumber(partNumber).eTag(uploadPartResponse.eTag()).build(); @@ -286,7 +313,8 @@ public void run() { } } - private void writeObject(S3AsyncClient writeS3Client, String path, ByteBuffer data, CompletableFuture cf, String bucket) { + private void writeObject(S3AsyncClient writeS3Client, String path, ByteBuffer data, CompletableFuture cf, + String bucket) { int objectSize = data.remaining(); PutObjectRequest request = PutObjectRequest.builder().bucket(bucket).key(path).build(); AsyncRequestBody body = AsyncRequestBody.fromByteBuffersUnsafe(data); @@ -300,24 +328,26 @@ private void writeObject(S3AsyncClient writeS3Client, String path, ByteBuffer da }); } - protected void readRange(S3AsyncClient readS3Client, String path, CompletableFuture cf, String bucket, long start, long end) { + protected void readRange(S3AsyncClient readS3Client, String path, CompletableFuture cf, String bucket, + long start, long end) { GetObjectRequest request = GetObjectRequest.builder().bucket(bucket).key(path).range(range(start, end)).build(); readS3Client.getObject(request, AsyncResponseTransformer.toPublisher()) - .thenAccept(responsePublisher -> { - CompositeByteBuf buf = DirectByteBufAlloc.compositeByteBuffer(); - responsePublisher.subscribe((bytes) -> { - // the aws client will copy DefaultHttpContent to heap ByteBuffer - buf.addComponent(true, Unpooled.wrappedBuffer(bytes)); - }).thenAccept(v -> { - cf.complete(buf); - }); - }).exceptionally(ex -> { - cf.completeExceptionally(ex); - return null; + .thenAccept(responsePublisher -> { + CompositeByteBuf buf = DirectByteBufAlloc.compositeByteBuffer(); + responsePublisher.subscribe((bytes) -> { + // the aws client will copy DefaultHttpContent to heap ByteBuffer + buf.addComponent(true, Unpooled.wrappedBuffer(bytes)); + }).thenAccept(v -> { + cf.complete(buf); }); + }).exceptionally(ex -> { + cf.completeExceptionally(ex); + return null; + }); } - protected void deleteObject(S3AsyncClient deleteS3Client, String path, CompletableFuture cf, String bucket) { + protected void deleteObject(S3AsyncClient deleteS3Client, String path, CompletableFuture cf, + String bucket) { DeleteObjectRequest request = DeleteObjectRequest.builder().bucket(bucket).key(path).build(); deleteS3Client.deleteObject(request).thenAccept(deleteObjectResponse -> { System.out.printf("deleted object %s%n", path); @@ -329,7 +359,6 @@ protected void deleteObject(S3AsyncClient deleteS3Client, String path, Completab }); } - @Override public void close() { System.out.println("3) Trying to delete object " + path + " ..."); @@ -352,48 +381,26 @@ public HelloS3Task(S3Context context) { super(context, HelloS3Task.class.getSimpleName()); } - @Override - public void run() { - System.out.println("Trying to list all buckets in your account ..."); - listBuckets(client); - } - private static void listBuckets(S3AsyncClient s3) { try { s3.listBuckets(ListBucketsRequest.builder().build()) - .thenAccept(response -> { - List bucketList = response.buckets(); - bucketList.forEach(bucket -> { - System.out.println("Bucket Name: " + bucket.name()); - }); - }).get(); + .thenAccept(response -> { + List bucketList = response.buckets(); + bucketList.forEach(bucket -> { + System.out.println("Bucket Name: " + bucket.name()); + }); + }).get(); } catch (ExecutionException | InterruptedException e) { showErrorInfo(e); throw new RuntimeException(e); } } - } - - private static S3AsyncClient newS3AsyncClient(String endpoint, String region, boolean forcePathStyle, String accessKey, String secretKey) { - S3AsyncClientBuilder builder = S3AsyncClient.builder().region(Region.of(region)); - if (StringUtils.isNotBlank(endpoint)) { - builder.endpointOverride(URI.create(endpoint)); - } - builder.serviceConfiguration(c -> c.pathStyleAccessEnabled(forcePathStyle)); - builder.credentialsProvider(StaticCredentialsProvider.create(AwsBasicCredentials.create(accessKey, secretKey))); - builder.overrideConfiguration(b -> b.apiCallTimeout(Duration.ofMinutes(1)) - .apiCallAttemptTimeout(Duration.ofSeconds(30))); - return builder.build(); - } - private static String hideSecret(String secret) { - if (secret == null) { - return null; - } - if (secret.length() < 6) { - return "*".repeat(secret.length()); + @Override + public void run() { + System.out.println("Trying to list all buckets in your account ..."); + listBuckets(client); } - return secret.substring(0, 3) + "*".repeat(secret.length() - 6) + secret.substring(secret.length() - 3); } public static class S3Context { @@ -404,7 +411,8 @@ public static class S3Context { private final String region; private final boolean forcePathStyle; - public S3Context(String endpoint, String accessKey, String secretKey, String bucketName, String region, boolean forcePathStyle) { + public S3Context(String endpoint, String accessKey, String secretKey, String bucketName, String region, + boolean forcePathStyle) { this.endpoint = endpoint; this.accessKey = accessKey; this.secretKey = secretKey; @@ -413,6 +421,10 @@ public S3Context(String endpoint, String accessKey, String secretKey, String buc this.forcePathStyle = forcePathStyle; } + public static Builder builder() { + return new Builder(); + } + public List advices() { List advises = new ArrayList<>(); if (StringUtils.isBlank(bucketName)) { @@ -454,17 +466,13 @@ public List advices() { @Override public String toString() { return "S3CheckContext{" + - "endpoint='" + endpoint + '\'' + - ", accessKey='" + hideSecret(accessKey) + '\'' + - ", secretKey='" + hideSecret(secretKey) + '\'' + - ", bucketName='" + bucketName + '\'' + - ", region='" + region + '\'' + - ", forcePathStyle=" + forcePathStyle + - '}'; - } - - public static Builder builder() { - return new Builder(); + "endpoint='" + endpoint + '\'' + + ", accessKey='" + hideSecret(accessKey) + '\'' + + ", secretKey='" + hideSecret(secretKey) + '\'' + + ", bucketName='" + bucketName + '\'' + + ", region='" + region + '\'' + + ", forcePathStyle=" + forcePathStyle + + '}'; } public static class Builder { diff --git a/s3stream/src/main/java/com/automq/stream/utils/ThreadUtils.java b/s3stream/src/main/java/com/automq/stream/utils/ThreadUtils.java index ce0284f1f..d049fed1b 100644 --- a/s3stream/src/main/java/com/automq/stream/utils/ThreadUtils.java +++ b/s3stream/src/main/java/com/automq/stream/utils/ThreadUtils.java @@ -27,14 +27,14 @@ public class ThreadUtils { /** * Create a new ThreadFactory. * - * @param pattern The pattern to use. If this contains %d, it will be - * replaced with a thread number. It should not contain more - * than one %d. - * @param daemon True if we want daemon threads. - * @return The new ThreadFactory. + * @param pattern The pattern to use. If this contains %d, it will be + * replaced with a thread number. It should not contain more + * than one %d. + * @param daemon True if we want daemon threads. + * @return The new ThreadFactory. */ public static ThreadFactory createThreadFactory(final String pattern, - final boolean daemon) { + final boolean daemon) { return new ThreadFactory() { private final AtomicLong threadEpoch = new AtomicLong(0); diff --git a/s3stream/src/main/java/com/automq/stream/utils/Threads.java b/s3stream/src/main/java/com/automq/stream/utils/Threads.java index fea413735..cc61f7f61 100644 --- a/s3stream/src/main/java/com/automq/stream/utils/Threads.java +++ b/s3stream/src/main/java/com/automq/stream/utils/Threads.java @@ -18,14 +18,13 @@ package com.automq.stream.utils; import com.automq.stream.utils.threads.S3StreamThreadPoolMonitor; -import org.slf4j.Logger; - import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import org.slf4j.Logger; public class Threads { @@ -41,7 +40,8 @@ protected void afterExecute(Runnable r, Throwable t) { }; } - public static ExecutorService newFixedThreadPoolWithMonitor(int nThreads, String namePrefix, boolean isDaemen, Logger logger) { + public static ExecutorService newFixedThreadPoolWithMonitor(int nThreads, String namePrefix, boolean isDaemen, + Logger logger) { return S3StreamThreadPoolMonitor.createAndMonitor(nThreads, nThreads, 0L, TimeUnit.MILLISECONDS, namePrefix, isDaemen, Integer.MAX_VALUE, throwable -> { if (throwable != null) { logger.error("[FATAL] Uncaught exception in executor thread {}", Thread.currentThread().getName(), throwable); @@ -50,15 +50,18 @@ public static ExecutorService newFixedThreadPoolWithMonitor(int nThreads, String }); } - public static ScheduledThreadPoolExecutor newSingleThreadScheduledExecutor(String name, boolean daemon, Logger logger) { + public static ScheduledThreadPoolExecutor newSingleThreadScheduledExecutor(String name, boolean daemon, + Logger logger) { return newSingleThreadScheduledExecutor(ThreadUtils.createThreadFactory(name, true), logger, false); } - public static ScheduledThreadPoolExecutor newSingleThreadScheduledExecutor(ThreadFactory threadFactory, Logger logger) { + public static ScheduledThreadPoolExecutor newSingleThreadScheduledExecutor(ThreadFactory threadFactory, + Logger logger) { return newSingleThreadScheduledExecutor(threadFactory, logger, false); } - public static ScheduledThreadPoolExecutor newSingleThreadScheduledExecutor(ThreadFactory threadFactory, Logger logger, boolean removeOnCancelPolicy) { + public static ScheduledThreadPoolExecutor newSingleThreadScheduledExecutor(ThreadFactory threadFactory, + Logger logger, boolean removeOnCancelPolicy) { ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1, threadFactory) { @Override protected void afterExecute(Runnable r, Throwable t) { diff --git a/s3stream/src/main/java/com/automq/stream/utils/Utils.java b/s3stream/src/main/java/com/automq/stream/utils/Utils.java index 3ba0b4296..d7e971a00 100644 --- a/s3stream/src/main/java/com/automq/stream/utils/Utils.java +++ b/s3stream/src/main/java/com/automq/stream/utils/Utils.java @@ -19,6 +19,7 @@ public class Utils { public static final String MAX_MERGE_READ_SPARSITY_RATE_NAME = "MERGE_READ_SPARSITY_RATE"; + public static float getMaxMergeReadSparsityRate() { float rate; try { diff --git a/s3stream/src/main/java/com/automq/stream/utils/biniarysearch/ComparableItem.java b/s3stream/src/main/java/com/automq/stream/utils/biniarysearch/ComparableItem.java index 10bc0f38a..529c86804 100644 --- a/s3stream/src/main/java/com/automq/stream/utils/biniarysearch/ComparableItem.java +++ b/s3stream/src/main/java/com/automq/stream/utils/biniarysearch/ComparableItem.java @@ -19,5 +19,6 @@ public interface ComparableItem { boolean isLessThan(T value); + boolean isGreaterThan(T value); } diff --git a/s3stream/src/main/java/com/automq/stream/utils/biniarysearch/IndexBlockOrderedBytes.java b/s3stream/src/main/java/com/automq/stream/utils/biniarysearch/IndexBlockOrderedBytes.java index e86546e77..0b17b0c16 100644 --- a/s3stream/src/main/java/com/automq/stream/utils/biniarysearch/IndexBlockOrderedBytes.java +++ b/s3stream/src/main/java/com/automq/stream/utils/biniarysearch/IndexBlockOrderedBytes.java @@ -46,7 +46,7 @@ public record TargetStreamOffset(long streamId, long offset) { } private record ComparableStreamRange(long streamId, long startOffset, int recordCount, int blockIndex) - implements ComparableItem { + implements ComparableItem { private static final int SIZE = 8 + 8 + 4 + 4; public long endOffset() { diff --git a/s3stream/src/main/java/com/automq/stream/utils/biniarysearch/StreamRecordBatchList.java b/s3stream/src/main/java/com/automq/stream/utils/biniarysearch/StreamRecordBatchList.java index 6a5f6143c..1a75679b9 100644 --- a/s3stream/src/main/java/com/automq/stream/utils/biniarysearch/StreamRecordBatchList.java +++ b/s3stream/src/main/java/com/automq/stream/utils/biniarysearch/StreamRecordBatchList.java @@ -18,7 +18,6 @@ package com.automq.stream.utils.biniarysearch; import com.automq.stream.s3.model.StreamRecordBatch; - import java.util.List; public class StreamRecordBatchList extends AbstractOrderedCollection { diff --git a/s3stream/src/main/java/com/automq/stream/utils/threads/S3StreamThreadPoolMonitor.java b/s3stream/src/main/java/com/automq/stream/utils/threads/S3StreamThreadPoolMonitor.java index afa4d9619..1ce81774e 100644 --- a/s3stream/src/main/java/com/automq/stream/utils/threads/S3StreamThreadPoolMonitor.java +++ b/s3stream/src/main/java/com/automq/stream/utils/threads/S3StreamThreadPoolMonitor.java @@ -18,9 +18,6 @@ package com.automq.stream.utils.threads; import com.automq.stream.utils.ThreadUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -31,14 +28,14 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.function.Function; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class S3StreamThreadPoolMonitor { - private static Logger waterMarkLogger = LoggerFactory.getLogger(S3StreamThreadPoolMonitor.class); - private static final List MONITOR_EXECUTOR = new CopyOnWriteArrayList<>(); private static final ScheduledExecutorService MONITOR_SCHEDULED = Executors.newSingleThreadScheduledExecutor( - ThreadUtils.createThreadFactory("ThreadPoolMonitor-%d", true)); - + ThreadUtils.createThreadFactory("ThreadPoolMonitor-%d", true)); + private static Logger waterMarkLogger = LoggerFactory.getLogger(S3StreamThreadPoolMonitor.class); private static volatile long threadPoolStatusPeriodTime = TimeUnit.SECONDS.toMillis(3); public static void config(Logger waterMarkLoggerConfig, long threadPoolStatusPeriodTimeConfig) { @@ -47,23 +44,23 @@ public static void config(Logger waterMarkLoggerConfig, long threadPoolStatusPer } public static ThreadPoolExecutor createAndMonitor(int corePoolSize, - int maximumPoolSize, - long keepAliveTime, - TimeUnit unit, - String name, - boolean isDaemon, - int queueCapacity) { + int maximumPoolSize, + long keepAliveTime, + TimeUnit unit, + String name, + boolean isDaemon, + int queueCapacity) { return createAndMonitor(corePoolSize, maximumPoolSize, keepAliveTime, unit, name, isDaemon, queueCapacity, throwable -> null); } public static ThreadPoolExecutor createAndMonitor(int corePoolSize, - int maximumPoolSize, - long keepAliveTime, - TimeUnit unit, - String name, - boolean isDaemon, - int queueCapacity, - Function afterExecutionHook) { + int maximumPoolSize, + long keepAliveTime, + TimeUnit unit, + String name, + boolean isDaemon, + int queueCapacity, + Function afterExecutionHook) { return createAndMonitor(corePoolSize, maximumPoolSize, keepAliveTime, unit, name, isDaemon, queueCapacity, afterExecutionHook, Collections.emptyList()); } diff --git a/s3stream/src/main/java/com/automq/stream/utils/threads/ThreadPoolWrapper.java b/s3stream/src/main/java/com/automq/stream/utils/threads/ThreadPoolWrapper.java index 471cec138..fbb881580 100644 --- a/s3stream/src/main/java/com/automq/stream/utils/threads/ThreadPoolWrapper.java +++ b/s3stream/src/main/java/com/automq/stream/utils/threads/ThreadPoolWrapper.java @@ -19,7 +19,6 @@ import com.google.common.base.MoreObjects; import com.google.common.base.Objects; - import java.util.List; import java.util.concurrent.ThreadPoolExecutor; @@ -35,41 +34,6 @@ public class ThreadPoolWrapper { this.statusPrinters = statusPrinters; } - public static class ThreadPoolWrapperBuilder { - private String name; - private ThreadPoolExecutor threadPoolExecutor; - private List statusPrinters; - - ThreadPoolWrapperBuilder() { - } - - public ThreadPoolWrapperBuilder name(final String name) { - this.name = name; - return this; - } - - public ThreadPoolWrapperBuilder threadPoolExecutor( - final ThreadPoolExecutor threadPoolExecutor) { - this.threadPoolExecutor = threadPoolExecutor; - return this; - } - - public ThreadPoolWrapperBuilder statusPrinters( - final List statusPrinters) { - this.statusPrinters = statusPrinters; - return this; - } - - public ThreadPoolWrapper build() { - return new ThreadPoolWrapper(this.name, this.threadPoolExecutor, this.statusPrinters); - } - - @Override - public String toString() { - return "ThreadPoolWrapper.ThreadPoolWrapperBuilder(name=" + this.name + ", threadPoolExecutor=" + this.threadPoolExecutor + ", statusPrinters=" + this.statusPrinters + ")"; - } - } - public static ThreadPoolWrapperBuilder builder() { return new ThreadPoolWrapperBuilder(); } @@ -78,22 +42,22 @@ public String getName() { return this.name; } - public ThreadPoolExecutor getThreadPoolExecutor() { - return this.threadPoolExecutor; - } - - public List getStatusPrinters() { - return this.statusPrinters; - } - public void setName(final String name) { this.name = name; } + public ThreadPoolExecutor getThreadPoolExecutor() { + return this.threadPoolExecutor; + } + public void setThreadPoolExecutor(final ThreadPoolExecutor threadPoolExecutor) { this.threadPoolExecutor = threadPoolExecutor; } + public List getStatusPrinters() { + return this.statusPrinters; + } + public void setStatusPrinters(final List statusPrinters) { this.statusPrinters = statusPrinters; } @@ -121,4 +85,39 @@ public String toString() { .add("statusPrinters", statusPrinters) .toString(); } + + public static class ThreadPoolWrapperBuilder { + private String name; + private ThreadPoolExecutor threadPoolExecutor; + private List statusPrinters; + + ThreadPoolWrapperBuilder() { + } + + public ThreadPoolWrapperBuilder name(final String name) { + this.name = name; + return this; + } + + public ThreadPoolWrapperBuilder threadPoolExecutor( + final ThreadPoolExecutor threadPoolExecutor) { + this.threadPoolExecutor = threadPoolExecutor; + return this; + } + + public ThreadPoolWrapperBuilder statusPrinters( + final List statusPrinters) { + this.statusPrinters = statusPrinters; + return this; + } + + public ThreadPoolWrapper build() { + return new ThreadPoolWrapper(this.name, this.threadPoolExecutor, this.statusPrinters); + } + + @Override + public String toString() { + return "ThreadPoolWrapper.ThreadPoolWrapperBuilder(name=" + this.name + ", threadPoolExecutor=" + this.threadPoolExecutor + ", statusPrinters=" + this.statusPrinters + ")"; + } + } } diff --git a/s3stream/src/test/java/com/automq/stream/s3/AsyncNetworkBandwidthLimiterTest.java b/s3stream/src/test/java/com/automq/stream/s3/AsyncNetworkBandwidthLimiterTest.java index a90823cb6..cfc59698d 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/AsyncNetworkBandwidthLimiterTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/AsyncNetworkBandwidthLimiterTest.java @@ -19,11 +19,10 @@ import com.automq.stream.s3.network.AsyncNetworkBandwidthLimiter; import com.automq.stream.s3.network.ThrottleStrategy; +import java.util.concurrent.CompletableFuture; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import java.util.concurrent.CompletableFuture; - public class AsyncNetworkBandwidthLimiterTest { @Test diff --git a/s3stream/src/test/java/com/automq/stream/s3/DefaultRecordBatch.java b/s3stream/src/test/java/com/automq/stream/s3/DefaultRecordBatch.java index 4999ab636..fd96fc8ee 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/DefaultRecordBatch.java +++ b/s3stream/src/test/java/com/automq/stream/s3/DefaultRecordBatch.java @@ -18,7 +18,6 @@ package com.automq.stream.s3; import com.automq.stream.api.RecordBatch; - import java.nio.ByteBuffer; import java.util.Collections; import java.util.Map; @@ -37,7 +36,6 @@ public static RecordBatch of(int count, int size) { return record; } - @Override public int count() { return count; diff --git a/s3stream/src/test/java/com/automq/stream/s3/DefaultRecordBatchWithContext.java b/s3stream/src/test/java/com/automq/stream/s3/DefaultRecordBatchWithContext.java index 1f193d237..025363e10 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/DefaultRecordBatchWithContext.java +++ b/s3stream/src/test/java/com/automq/stream/s3/DefaultRecordBatchWithContext.java @@ -19,7 +19,6 @@ import com.automq.stream.api.RecordBatch; import com.automq.stream.api.RecordBatchWithContext; - import java.nio.ByteBuffer; import java.util.Map; diff --git a/s3stream/src/test/java/com/automq/stream/s3/DefaultS3BlockCacheTest.java b/s3stream/src/test/java/com/automq/stream/s3/DefaultS3BlockCacheTest.java index 6d98d4515..102934bb4 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/DefaultS3BlockCacheTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/DefaultS3BlockCacheTest.java @@ -20,25 +20,24 @@ import com.automq.stream.s3.cache.CacheAccessType; import com.automq.stream.s3.cache.DefaultS3BlockCache; import com.automq.stream.s3.cache.ReadDataBlock; +import com.automq.stream.s3.metadata.ObjectUtils; +import com.automq.stream.s3.metadata.S3ObjectMetadata; +import com.automq.stream.s3.metadata.S3ObjectType; import com.automq.stream.s3.model.StreamRecordBatch; import com.automq.stream.s3.objects.ObjectManager; import com.automq.stream.s3.operator.MemoryS3Operator; import com.automq.stream.s3.operator.S3Operator; -import com.automq.stream.s3.metadata.ObjectUtils; -import com.automq.stream.s3.metadata.S3ObjectMetadata; -import com.automq.stream.s3.metadata.S3ObjectType; import com.automq.stream.utils.Threads; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Tag; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentMatchers; -import org.mockito.Mockito; - import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentMatchers; +import org.mockito.Mockito; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.ArgumentMatchers.eq; @@ -66,9 +65,9 @@ public void setup() { public void testRead() throws Exception { ObjectWriter objectWriter = ObjectWriter.writer(0, s3Operator, 1024, 1024); objectWriter.write(233, List.of( - newRecord(233, 10, 5, 512), - newRecord(233, 15, 10, 512), - newRecord(233, 25, 5, 512) + newRecord(233, 10, 5, 512), + newRecord(233, 15, 10, 512), + newRecord(233, 25, 5, 512) )); objectWriter.write(234, List.of(newRecord(234, 0, 5, 512))); objectWriter.close(); @@ -85,10 +84,10 @@ public void testRead() throws Exception { S3ObjectMetadata metadata3 = new S3ObjectMetadata(2, objectWriter.size(), S3ObjectType.STREAM_SET); when(objectManager.getObjects(eq(233L), eq(11L), eq(60L), eq(2))).thenReturn(CompletableFuture.completedFuture(List.of( - metadata1, metadata2 + metadata1, metadata2 ))); when(objectManager.getObjects(eq(233L), eq(40L), eq(60L), eq(2))).thenReturn(CompletableFuture.completedFuture(List.of( - metadata3 + metadata3 ))); ReadDataBlock rst = s3BlockCache.read(233L, 11L, 60L, 10000).get(3000, TimeUnit.SECONDS); @@ -106,8 +105,8 @@ public void testRead_readAhead() throws ExecutionException, InterruptedException ObjectWriter objectWriter = ObjectWriter.writer(0, s3Operator, 1024, 1024); objectWriter.write(233, List.of( - newRecord(233, 10, 5, 512), - newRecord(233, 15, 5, 4096) + newRecord(233, 10, 5, 512), + newRecord(233, 15, 5, 4096) )); objectWriter.close(); S3ObjectMetadata metadata1 = new S3ObjectMetadata(0, objectWriter.size(), S3ObjectType.STREAM_SET); @@ -144,5 +143,4 @@ StreamRecordBatch newRecord(long streamId, long offset, int count, int payloadSi return new StreamRecordBatch(streamId, 0, offset, count, TestUtils.random(payloadSize)); } - } diff --git a/s3stream/src/test/java/com/automq/stream/s3/DeltaWALUploadTaskTest.java b/s3stream/src/test/java/com/automq/stream/s3/DeltaWALUploadTaskTest.java index 40f4d21e9..5671580d0 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/DeltaWALUploadTaskTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/DeltaWALUploadTaskTest.java @@ -27,11 +27,6 @@ import com.automq.stream.s3.operator.MemoryS3Operator; import com.automq.stream.s3.operator.S3Operator; import com.automq.stream.utils.CloseableIterator; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Tag; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; - import java.util.HashMap; import java.util.List; import java.util.Map; @@ -40,6 +35,10 @@ import java.util.concurrent.ForkJoinPool; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicLong; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; import static com.automq.stream.s3.TestUtils.random; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -73,21 +72,21 @@ public void testUpload() throws Exception { Map> map = new HashMap<>(); map.put(233L, List.of( - new StreamRecordBatch(233, 0, 10, 2, random(512)), - new StreamRecordBatch(233, 0, 12, 2, random(128)), - new StreamRecordBatch(233, 0, 14, 2, random(512)) + new StreamRecordBatch(233, 0, 10, 2, random(512)), + new StreamRecordBatch(233, 0, 12, 2, random(128)), + new StreamRecordBatch(233, 0, 14, 2, random(512)) )); map.put(234L, List.of( - new StreamRecordBatch(234, 0, 20, 2, random(128)), - new StreamRecordBatch(234, 0, 22, 2, random(128)) + new StreamRecordBatch(234, 0, 20, 2, random(128)), + new StreamRecordBatch(234, 0, 22, 2, random(128)) )); Config config = new Config() - .objectBlockSize(16 * 1024 * 1024) - .objectPartSize(16 * 1024 * 1024) - .streamSplitSize(1000); + .objectBlockSize(16 * 1024 * 1024) + .objectPartSize(16 * 1024 * 1024) + .streamSplitSize(1000); deltaWALUploadTask = DeltaWALUploadTask.builder().config(config).streamRecordsMap(map).objectManager(objectManager) - .s3Operator(s3Operator).executor(ForkJoinPool.commonPool()).build(); + .s3Operator(s3Operator).executor(ForkJoinPool.commonPool()).build(); deltaWALUploadTask.prepare().get(); deltaWALUploadTask.upload().get(); @@ -119,7 +118,7 @@ public void testUpload() throws Exception { S3ObjectMetadata s3ObjectMetadata = new S3ObjectMetadata(request.getObjectId(), request.getObjectSize(), S3ObjectType.STREAM_SET); ObjectReader objectReader = new ObjectReader(s3ObjectMetadata, s3Operator); ObjectReader.DataBlockIndex blockIndex = objectReader.find(234, 20, 24).get() - .streamDataBlocks().get(0).dataBlockIndex(); + .streamDataBlocks().get(0).dataBlockIndex(); ObjectReader.DataBlock dataBlock = objectReader.read(blockIndex).get(); try (CloseableIterator it = dataBlock.iterator()) { StreamRecordBatch record = it.next(); @@ -134,7 +133,7 @@ record = it.next(); S3ObjectMetadata streamObjectMetadata = new S3ObjectMetadata(11, request.getStreamObjects().get(0).getObjectSize(), S3ObjectType.STREAM); ObjectReader objectReader = new ObjectReader(streamObjectMetadata, s3Operator); ObjectReader.DataBlockIndex blockIndex = objectReader.find(233, 10, 16).get() - .streamDataBlocks().get(0).dataBlockIndex(); + .streamDataBlocks().get(0).dataBlockIndex(); ObjectReader.DataBlock dataBlock = objectReader.read(blockIndex).get(); try (CloseableIterator it = dataBlock.iterator()) { StreamRecordBatch r1 = it.next(); @@ -158,16 +157,16 @@ public void testUpload_oneStream() throws Exception { Map> map = new HashMap<>(); map.put(233L, List.of( - new StreamRecordBatch(233, 0, 10, 2, random(512)), - new StreamRecordBatch(233, 0, 12, 2, random(128)), - new StreamRecordBatch(233, 0, 14, 2, random(512)) + new StreamRecordBatch(233, 0, 10, 2, random(512)), + new StreamRecordBatch(233, 0, 12, 2, random(128)), + new StreamRecordBatch(233, 0, 14, 2, random(512)) )); Config config = new Config() - .objectBlockSize(16 * 1024 * 1024) - .objectPartSize(16 * 1024 * 1024) - .streamSplitSize(16 * 1024 * 1024); + .objectBlockSize(16 * 1024 * 1024) + .objectPartSize(16 * 1024 * 1024) + .streamSplitSize(16 * 1024 * 1024); deltaWALUploadTask = DeltaWALUploadTask.builder().config(config).streamRecordsMap(map).objectManager(objectManager) - .s3Operator(s3Operator).executor(ForkJoinPool.commonPool()).build(); + .s3Operator(s3Operator).executor(ForkJoinPool.commonPool()).build(); deltaWALUploadTask.prepare().get(); deltaWALUploadTask.upload().get(); @@ -176,7 +175,6 @@ public void testUpload_oneStream() throws Exception { // Release all the buffers map.values().forEach(batches -> batches.forEach(StreamRecordBatch::release)); - ArgumentCaptor reqArg = ArgumentCaptor.forClass(CommitStreamSetObjectRequest.class); verify(objectManager, times(1)).commitStreamSetObject(reqArg.capture()); CommitStreamSetObjectRequest request = reqArg.getValue(); @@ -193,18 +191,18 @@ public void test_emptyWALData() throws ExecutionException, InterruptedException, Map> map = new HashMap<>(); map.put(233L, List.of( - new StreamRecordBatch(233, 0, 10, 2, random(512)) + new StreamRecordBatch(233, 0, 10, 2, random(512)) )); map.put(234L, List.of( - new StreamRecordBatch(234, 0, 20, 2, random(128)) + new StreamRecordBatch(234, 0, 20, 2, random(128)) )); Config config = new Config() - .objectBlockSize(16 * 1024 * 1024) - .objectPartSize(16 * 1024 * 1024) - .streamSplitSize(64); + .objectBlockSize(16 * 1024 * 1024) + .objectPartSize(16 * 1024 * 1024) + .streamSplitSize(64); deltaWALUploadTask = DeltaWALUploadTask.builder().config(config).streamRecordsMap(map).objectManager(objectManager) - .s3Operator(s3Operator).executor(ForkJoinPool.commonPool()).build(); + .s3Operator(s3Operator).executor(ForkJoinPool.commonPool()).build(); assertTrue(deltaWALUploadTask.forceSplit); } } diff --git a/s3stream/src/test/java/com/automq/stream/s3/ObjectReaderTest.java b/s3stream/src/test/java/com/automq/stream/s3/ObjectReaderTest.java index a9723541b..d4b35bcc1 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/ObjectReaderTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/ObjectReaderTest.java @@ -24,13 +24,12 @@ import com.automq.stream.s3.operator.S3Operator; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; +import java.util.List; +import java.util.concurrent.ExecutionException; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import org.mockito.Mockito; -import java.util.List; -import java.util.concurrent.ExecutionException; - import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -74,7 +73,6 @@ public void testIndexBlock() { blocks.writeInt(blockSize3); blocks.writeInt(recordCnt3); - ByteBuf streamRanges = Unpooled.buffer(3 * (8 + 8 + 4 + 4)); streamRanges.writeLong(streamId1); streamRanges.writeLong(0); diff --git a/s3stream/src/test/java/com/automq/stream/s3/ObjectWriterTest.java b/s3stream/src/test/java/com/automq/stream/s3/ObjectWriterTest.java index 41029be25..8459c8548 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/ObjectWriterTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/ObjectWriterTest.java @@ -17,18 +17,17 @@ package com.automq.stream.s3; +import com.automq.stream.s3.metadata.S3ObjectMetadata; +import com.automq.stream.s3.metadata.S3ObjectType; import com.automq.stream.s3.model.StreamRecordBatch; import com.automq.stream.s3.objects.ObjectStreamRange; import com.automq.stream.s3.operator.MemoryS3Operator; import com.automq.stream.s3.operator.S3Operator; -import com.automq.stream.s3.metadata.S3ObjectMetadata; -import com.automq.stream.s3.metadata.S3ObjectType; -import org.junit.jupiter.api.Tag; -import org.junit.jupiter.api.Test; - import java.util.Iterator; import java.util.List; import java.util.concurrent.ExecutionException; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -60,7 +59,6 @@ public void testWrite() throws ExecutionException, InterruptedException { assertEquals(0, streamRanges.get(1).getStartOffset()); assertEquals(5, streamRanges.get(1).getEndOffset()); - int objectSize = s3Operator.rangeRead(metadata.key(), 0L, objectWriter.size()).get().readableBytes(); assertEquals(objectSize, objectWriter.size()); diff --git a/s3stream/src/test/java/com/automq/stream/s3/S3StorageTest.java b/s3stream/src/test/java/com/automq/stream/s3/S3StorageTest.java index 878b3774b..433ce20e8 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/S3StorageTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/S3StorageTest.java @@ -34,14 +34,6 @@ import com.automq.stream.s3.wal.MemoryWriteAheadLog; import com.automq.stream.s3.wal.WriteAheadLog; import io.netty.buffer.ByteBuf; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Tag; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.ArgumentMatchers; -import org.mockito.Mockito; - import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -49,6 +41,13 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.ArgumentMatchers; +import org.mockito.Mockito; import static com.automq.stream.s3.TestUtils.random; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -70,6 +69,10 @@ public class S3StorageTest { S3Storage storage; Config config; + private static StreamRecordBatch newRecord(long streamId, long offset) { + return new StreamRecordBatch(streamId, 0, offset, 1, random(1)); + } + @BeforeEach public void setup() { config = new Config(); @@ -79,7 +82,7 @@ public void setup() { wal = spy(new MemoryWriteAheadLog()); s3Operator = new MemoryS3Operator(); storage = new S3Storage(config, wal, - streamManager, objectManager, new DefaultS3BlockCache(config, objectManager, s3Operator), s3Operator); + streamManager, objectManager, new DefaultS3BlockCache(config, objectManager, s3Operator), s3Operator); } @Test @@ -89,13 +92,13 @@ public void testAppend() throws Exception { Mockito.when(objectManager.commitStreamSetObject(any())).thenReturn(CompletableFuture.completedFuture(resp)); CompletableFuture cf1 = storage.append( - new StreamRecordBatch(233, 1, 10, 1, random(100)) + new StreamRecordBatch(233, 1, 10, 1, random(100)) ); CompletableFuture cf2 = storage.append( - new StreamRecordBatch(233, 1, 11, 2, random(100)) + new StreamRecordBatch(233, 1, 11, 2, random(100)) ); CompletableFuture cf3 = storage.append( - new StreamRecordBatch(234, 3, 100, 1, random(100)) + new StreamRecordBatch(234, 3, 100, 1, random(100)) ); cf1.get(3, TimeUnit.SECONDS); @@ -218,11 +221,11 @@ public void testUploadWALObject_sequence() throws ExecutionException, Interrupte @Test public void testRecoverContinuousRecords() { List recoverResults = List.of( - new TestRecoverResult(StreamRecordBatchCodec.encode(newRecord(233L, 10L))), - new TestRecoverResult(StreamRecordBatchCodec.encode(newRecord(233L, 11L))), - new TestRecoverResult(StreamRecordBatchCodec.encode(newRecord(233L, 12L))), - new TestRecoverResult(StreamRecordBatchCodec.encode(newRecord(233L, 15L))), - new TestRecoverResult(StreamRecordBatchCodec.encode(newRecord(234L, 20L))) + new TestRecoverResult(StreamRecordBatchCodec.encode(newRecord(233L, 10L))), + new TestRecoverResult(StreamRecordBatchCodec.encode(newRecord(233L, 11L))), + new TestRecoverResult(StreamRecordBatchCodec.encode(newRecord(233L, 12L))), + new TestRecoverResult(StreamRecordBatchCodec.encode(newRecord(233L, 15L))), + new TestRecoverResult(StreamRecordBatchCodec.encode(newRecord(234L, 20L))) ); List openingStreams = List.of(new StreamMetadata(233L, 0L, 0L, 11L, StreamState.OPENED)); @@ -234,10 +237,9 @@ public void testRecoverContinuousRecords() { assertEquals(11L, streamRecords.get(0).getBaseOffset()); assertEquals(12L, streamRecords.get(1).getBaseOffset()); - // openingStreams = List.of( - new StreamMetadata(233L, 0L, 0L, 5L, StreamState.OPENED)); + new StreamMetadata(233L, 0L, 0L, 5L, StreamState.OPENED)); boolean exception = false; try { storage.recoverContinuousRecords(recoverResults.iterator(), openingStreams); @@ -270,10 +272,6 @@ public void testWALOverCapacity() throws WriteAheadLog.OverCapacityException { assertEquals(12L, range.getEndOffset()); } - private static StreamRecordBatch newRecord(long streamId, long offset) { - return new StreamRecordBatch(streamId, 0, offset, 1, random(1)); - } - static class TestRecoverResult implements WriteAheadLog.RecoverResult { private final ByteBuf record; diff --git a/s3stream/src/test/java/com/automq/stream/s3/S3StreamMemoryTest.java b/s3stream/src/test/java/com/automq/stream/s3/S3StreamMemoryTest.java index fb80ad70d..b43816330 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/S3StreamMemoryTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/S3StreamMemoryTest.java @@ -17,7 +17,6 @@ package com.automq.stream.s3; - //import com.automq.stream.api.AppendResult; //import com.automq.stream.api.CreateStreamOptions; //import com.automq.stream.api.FetchResult; @@ -34,6 +33,7 @@ //import com.automq.stream.s3.streams.StreamManager; //import com.automq.stream.s3.wal.MemoryWriteAheadLog; //import org.junit.jupiter.api.BeforeEach; + import org.junit.jupiter.api.Tag; //import org.junit.jupiter.api.Test; //import org.slf4j.Logger; diff --git a/s3stream/src/test/java/com/automq/stream/s3/S3StreamTest.java b/s3stream/src/test/java/com/automq/stream/s3/S3StreamTest.java index 3779731be..d85d30952 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/S3StreamTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/S3StreamTest.java @@ -23,17 +23,16 @@ import com.automq.stream.s3.cache.ReadDataBlock; import com.automq.stream.s3.model.StreamRecordBatch; import com.automq.stream.s3.streams.StreamManager; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import org.mockito.Mockito; -import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; - import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; @@ -57,7 +56,7 @@ public void setup() { public void testFetch() throws Throwable { stream.confirmOffset.set(120L); Mockito.when(storage.read(any(), eq(233L), eq(110L), eq(120L), eq(100))) - .thenReturn(CompletableFuture.completedFuture(newReadDataBlock(110, 115, 110))); + .thenReturn(CompletableFuture.completedFuture(newReadDataBlock(110, 115, 110))); FetchResult rst = stream.fetch(110, 120, 100).get(1, TimeUnit.SECONDS); assertEquals(1, rst.recordBatchList().size()); assertEquals(110, rst.recordBatchList().get(0).baseOffset()); diff --git a/s3stream/src/test/java/com/automq/stream/s3/StreamObjectCopierTest.java b/s3stream/src/test/java/com/automq/stream/s3/StreamObjectCopierTest.java index d603ffa54..eebaefc70 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/StreamObjectCopierTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/StreamObjectCopierTest.java @@ -17,17 +17,16 @@ package com.automq.stream.s3; +import com.automq.stream.s3.metadata.S3ObjectMetadata; +import com.automq.stream.s3.metadata.S3ObjectType; import com.automq.stream.s3.model.StreamRecordBatch; import com.automq.stream.s3.operator.MemoryS3Operator; import com.automq.stream.s3.operator.S3Operator; -import com.automq.stream.s3.metadata.S3ObjectMetadata; -import com.automq.stream.s3.metadata.S3ObjectType; -import org.junit.jupiter.api.Tag; -import org.junit.jupiter.api.Test; - import java.util.Iterator; import java.util.List; import java.util.concurrent.ExecutionException; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -63,7 +62,6 @@ public void testCopy() throws ExecutionException, InterruptedException { long targetObjectSize = streamObjectCopyer.size(); S3ObjectMetadata metadata = new S3ObjectMetadata(targetObjectId, targetObjectSize, S3ObjectType.STREAM); - int objectSize = s3Operator.rangeRead(metadata.key(), 0L, targetObjectSize).get().readableBytes(); assertEquals(targetObjectSize, objectSize); diff --git a/s3stream/src/test/java/com/automq/stream/s3/StreamObjectsCompactionTaskTest.java b/s3stream/src/test/java/com/automq/stream/s3/StreamObjectsCompactionTaskTest.java index dd4072986..efffd0075 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/StreamObjectsCompactionTaskTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/StreamObjectsCompactionTaskTest.java @@ -17,24 +17,18 @@ package com.automq.stream.s3; +import com.automq.stream.s3.metadata.S3ObjectMetadata; +import com.automq.stream.s3.metadata.S3ObjectType; +import com.automq.stream.s3.metadata.S3StreamConstant; +import com.automq.stream.s3.metadata.StreamOffsetRange; import com.automq.stream.s3.model.StreamRecordBatch; -import com.automq.stream.s3.objects.CompactStreamObjectRequest; import com.automq.stream.s3.objects.CommitStreamSetObjectRequest; import com.automq.stream.s3.objects.CommitStreamSetObjectResponse; +import com.automq.stream.s3.objects.CompactStreamObjectRequest; import com.automq.stream.s3.objects.ObjectManager; import com.automq.stream.s3.objects.StreamObject; import com.automq.stream.s3.operator.MemoryS3Operator; import com.automq.stream.s3.operator.S3Operator; -import com.automq.stream.s3.metadata.S3ObjectMetadata; -import com.automq.stream.s3.metadata.S3ObjectType; -import com.automq.stream.s3.metadata.S3StreamConstant; -import com.automq.stream.s3.metadata.StreamOffsetRange; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Tag; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentMatchers; -import org.mockito.Mockito; - import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -46,6 +40,11 @@ import java.util.concurrent.ForkJoinPool; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentMatchers; +import org.mockito.Mockito; import static com.automq.stream.s3.operator.Writer.MAX_PART_SIZE; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -77,21 +76,21 @@ void setUp() { void testTriggerTask() throws ExecutionException, InterruptedException { // Prepare 4 stream objects. They should be compacted into 2 new stream objects. List> objectsDetails = List.of( - List.of(40L, 50L, 1000L), - List.of(50L, 60L, 1000L), - List.of(65L, 70L, 1000L), - List.of(70L, 80L, 1000L) + List.of(40L, 50L, 1000L), + List.of(50L, 60L, 1000L), + List.of(65L, 70L, 1000L), + List.of(70L, 80L, 1000L) ); List metadataList = prepareRawStreamObjects(10, stream.streamId(), objectsDetails); // two stream object groups should be handled when(objectManager.getStreamObjects(anyLong(), anyLong(), anyLong(), anyInt())) - .thenReturn(CompletableFuture.completedFuture(metadataList)); + .thenReturn(CompletableFuture.completedFuture(metadataList)); StreamObjectsCompactionTask task = new StreamObjectsCompactionTask(objectManager, s3Operator, stream, Long.MAX_VALUE, 0); AtomicLong objectIdAlloc = new AtomicLong(100); when(objectManager.prepareObject(anyInt(), anyLong())).thenAnswer( - invocation -> CompletableFuture.completedFuture(objectIdAlloc.getAndIncrement())); + invocation -> CompletableFuture.completedFuture(objectIdAlloc.getAndIncrement())); when(objectManager.compactStreamObject(ArgumentMatchers.any(CompactStreamObjectRequest.class))).thenReturn(CompletableFuture.completedFuture(null)); // trigger a stream object compaction task @@ -119,21 +118,21 @@ void testTriggerTask() throws ExecutionException, InterruptedException { void testTriggerTaskFailure() throws InterruptedException, ExecutionException { // 2 compaction groups will be handled. when(objectManager.getStreamObjects(anyLong(), anyLong(), anyLong(), anyInt())) - .thenReturn(CompletableFuture.completedFuture(List.of( - new S3ObjectMetadata(1, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 5, 10)), 0, 0, 150, - S3StreamConstant.INVALID_ORDER_ID), - new S3ObjectMetadata(2, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 10, 20)), 0, 0, 20, - S3StreamConstant.INVALID_ORDER_ID), - new S3ObjectMetadata(3, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 40, 50)), 0, 0, 20, - S3StreamConstant.INVALID_ORDER_ID), - new S3ObjectMetadata(4, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 50, 60)), 0, 0, 20, - S3StreamConstant.INVALID_ORDER_ID), - new S3ObjectMetadata(5, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 65, 70)), 0, 0, 20, - S3StreamConstant.INVALID_ORDER_ID), - new S3ObjectMetadata(6, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 70, 80)), 0, 0, 20, - S3StreamConstant.INVALID_ORDER_ID)))); + .thenReturn(CompletableFuture.completedFuture(List.of( + new S3ObjectMetadata(1, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 5, 10)), 0, 0, 150, + S3StreamConstant.INVALID_ORDER_ID), + new S3ObjectMetadata(2, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 10, 20)), 0, 0, 20, + S3StreamConstant.INVALID_ORDER_ID), + new S3ObjectMetadata(3, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 40, 50)), 0, 0, 20, + S3StreamConstant.INVALID_ORDER_ID), + new S3ObjectMetadata(4, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 50, 60)), 0, 0, 20, + S3StreamConstant.INVALID_ORDER_ID), + new S3ObjectMetadata(5, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 65, 70)), 0, 0, 20, + S3StreamConstant.INVALID_ORDER_ID), + new S3ObjectMetadata(6, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 70, 80)), 0, 0, 20, + S3StreamConstant.INVALID_ORDER_ID)))); when(objectManager.prepareObject(anyInt(), anyLong())).thenReturn( - CompletableFuture.failedFuture(new RuntimeException("halt compaction task"))); + CompletableFuture.failedFuture(new RuntimeException("halt compaction task"))); // The first group's compaction failed in prepareObject phase, the second group should not be handled. StreamObjectsCompactionTask task = new StreamObjectsCompactionTask(objectManager, s3Operator, stream, 100, 0); @@ -170,11 +169,11 @@ void testTriggerTaskFailure() throws InterruptedException, ExecutionException { * @throws InterruptedException when prepareObject or commitWALObject failed */ List prepareRawStreamObjects(long startObjectId, long streamId, - List> objectsDetails) throws ExecutionException, InterruptedException { + List> objectsDetails) throws ExecutionException, InterruptedException { AtomicLong objectIdAlloc = new AtomicLong(startObjectId); Stack commitStreamSetObjectRequests = new Stack<>(); Mockito.doAnswer(invocation -> CompletableFuture.completedFuture(objectIdAlloc.getAndIncrement())).when(objectManager) - .prepareObject(anyInt(), anyLong()); + .prepareObject(anyInt(), anyLong()); when(objectManager.commitStreamSetObject(ArgumentMatchers.any())).thenAnswer(invocation -> { commitStreamSetObjectRequests.push(invocation.getArgument(0)); return CompletableFuture.completedFuture(new CommitStreamSetObjectResponse()); @@ -189,11 +188,11 @@ List prepareRawStreamObjects(long startObjectId, long streamId int recordsSize = Math.toIntExact(objectsDetail.get(2)); Map> map = Map.of(streamId, - List.of(new StreamRecordBatch(streamId, 0, startOffset, Math.toIntExact(endOffset - startOffset), TestUtils.random(recordsSize)))); + List.of(new StreamRecordBatch(streamId, 0, startOffset, Math.toIntExact(endOffset - startOffset), TestUtils.random(recordsSize)))); Config config = new Config() - .objectBlockSize(16 * 1024 * 1024) - .objectPartSize(16 * 1024 * 1024) - .streamSplitSize(recordsSize - 1); + .objectBlockSize(16 * 1024 * 1024) + .objectPartSize(16 * 1024 * 1024) + .streamSplitSize(recordsSize - 1); DeltaWALUploadTask deltaWALUploadTask = new DeltaWALUploadTask(config, map, objectManager, s3Operator, ForkJoinPool.commonPool(), false, Integer.MAX_VALUE); deltaWALUploadTask.prepare().get(); @@ -211,9 +210,9 @@ List prepareRawStreamObjects(long startObjectId, long streamId assertEquals(endOffset, streamObject.getEndOffset()); metadataList.add( - new S3ObjectMetadata(streamObject.getObjectId(), S3ObjectType.STREAM, List.of(new StreamOffsetRange(streamObject.getStreamId(), - streamObject.getStartOffset(), streamObject.getEndOffset())), 0, System.currentTimeMillis(), streamObject.getObjectSize(), - S3StreamConstant.INVALID_ORDER_ID)); + new S3ObjectMetadata(streamObject.getObjectId(), S3ObjectType.STREAM, List.of(new StreamOffsetRange(streamObject.getStreamId(), + streamObject.getStartOffset(), streamObject.getEndOffset())), 0, System.currentTimeMillis(), streamObject.getObjectSize(), + S3StreamConstant.INVALID_ORDER_ID)); } return metadataList; } @@ -225,34 +224,34 @@ void testPrepareCompactGroups() throws ExecutionException, InterruptedException long currentTimestamp = System.currentTimeMillis(); when(objectManager.getStreamObjects(anyLong(), anyLong(), anyLong(), anyInt())) - .thenReturn(CompletableFuture.completedFuture(List.of( - new S3ObjectMetadata(1, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 5, 10)), 0, currentTimestamp, 150, - S3StreamConstant.INVALID_ORDER_ID), - new S3ObjectMetadata(2, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 10, 20)), 0, currentTimestamp, 20, - S3StreamConstant.INVALID_ORDER_ID), - new S3ObjectMetadata(3, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 40, 50)), 0, currentTimestamp, 20, - S3StreamConstant.INVALID_ORDER_ID), - new S3ObjectMetadata(4, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 50, 60)), 0, currentTimestamp, 20, - S3StreamConstant.INVALID_ORDER_ID), - new S3ObjectMetadata(5, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 65, 70)), 0, currentTimestamp, 20, - S3StreamConstant.INVALID_ORDER_ID), - new S3ObjectMetadata(6, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 70, 80)), 0, currentTimestamp, 20, - S3StreamConstant.INVALID_ORDER_ID)))); + .thenReturn(CompletableFuture.completedFuture(List.of( + new S3ObjectMetadata(1, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 5, 10)), 0, currentTimestamp, 150, + S3StreamConstant.INVALID_ORDER_ID), + new S3ObjectMetadata(2, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 10, 20)), 0, currentTimestamp, 20, + S3StreamConstant.INVALID_ORDER_ID), + new S3ObjectMetadata(3, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 40, 50)), 0, currentTimestamp, 20, + S3StreamConstant.INVALID_ORDER_ID), + new S3ObjectMetadata(4, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 50, 60)), 0, currentTimestamp, 20, + S3StreamConstant.INVALID_ORDER_ID), + new S3ObjectMetadata(5, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 65, 70)), 0, currentTimestamp, 20, + S3StreamConstant.INVALID_ORDER_ID), + new S3ObjectMetadata(6, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 70, 80)), 0, currentTimestamp, 20, + S3StreamConstant.INVALID_ORDER_ID)))); Queue> compactGroups = task1.prepareCompactGroups(0); assertEquals(2, compactGroups.size()); assertEquals(List.of( - new S3ObjectMetadata(3, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 40, 50)), 0, currentTimestamp, 20, - S3StreamConstant.INVALID_ORDER_ID), - new S3ObjectMetadata(4, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 50, 60)), 0, currentTimestamp, 20, - S3StreamConstant.INVALID_ORDER_ID)), - Objects.requireNonNull(compactGroups.poll()).stream().map(StreamObjectsCompactionTask.S3StreamObjectMetadataSplitWrapper::s3StreamObjectMetadata).collect(Collectors.toList())); + new S3ObjectMetadata(3, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 40, 50)), 0, currentTimestamp, 20, + S3StreamConstant.INVALID_ORDER_ID), + new S3ObjectMetadata(4, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 50, 60)), 0, currentTimestamp, 20, + S3StreamConstant.INVALID_ORDER_ID)), + Objects.requireNonNull(compactGroups.poll()).stream().map(StreamObjectsCompactionTask.S3StreamObjectMetadataSplitWrapper::s3StreamObjectMetadata).collect(Collectors.toList())); assertEquals(List.of( - new S3ObjectMetadata(5, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 65, 70)), 0, currentTimestamp, 20, - S3StreamConstant.INVALID_ORDER_ID), - new S3ObjectMetadata(6, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 70, 80)), 0, currentTimestamp, 20, - S3StreamConstant.INVALID_ORDER_ID)), - Objects.requireNonNull(compactGroups.poll()).stream().map(StreamObjectsCompactionTask.S3StreamObjectMetadataSplitWrapper::s3StreamObjectMetadata).collect(Collectors.toList())); + new S3ObjectMetadata(5, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 65, 70)), 0, currentTimestamp, 20, + S3StreamConstant.INVALID_ORDER_ID), + new S3ObjectMetadata(6, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 70, 80)), 0, currentTimestamp, 20, + S3StreamConstant.INVALID_ORDER_ID)), + Objects.requireNonNull(compactGroups.poll()).stream().map(StreamObjectsCompactionTask.S3StreamObjectMetadataSplitWrapper::s3StreamObjectMetadata).collect(Collectors.toList())); assertEquals(10, task1.getNextStartSearchingOffset()); // check if we can filter two groups with limit of timestamp @@ -260,36 +259,36 @@ void testPrepareCompactGroups() throws ExecutionException, InterruptedException currentTimestamp = System.currentTimeMillis(); when(objectManager.getStreamObjects(anyLong(), anyLong(), anyLong(), anyInt())) - .thenReturn(CompletableFuture.completedFuture(List.of( - new S3ObjectMetadata(1, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 5, 10)), 0, currentTimestamp - 20000, 60, - S3StreamConstant.INVALID_ORDER_ID), - new S3ObjectMetadata(2, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 10, 40)), 0, currentTimestamp, 20, - S3StreamConstant.INVALID_ORDER_ID), - new S3ObjectMetadata(3, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 40, 50)), 0, currentTimestamp - 20000, 20, - S3StreamConstant.INVALID_ORDER_ID), - new S3ObjectMetadata(4, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 50, 60)), 0, currentTimestamp, 20, - S3StreamConstant.INVALID_ORDER_ID), - new S3ObjectMetadata(5, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 60, 70)), 0, currentTimestamp - 30000, 20, - S3StreamConstant.INVALID_ORDER_ID), - new S3ObjectMetadata(6, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 70, 80)), 0, currentTimestamp - 30000, 20, - S3StreamConstant.INVALID_ORDER_ID), - new S3ObjectMetadata(7, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 80, 90)), 0, currentTimestamp - 30000, 20, - S3StreamConstant.INVALID_ORDER_ID), - new S3ObjectMetadata(8, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 90, 95)), 0, currentTimestamp - 30000, 80, - S3StreamConstant.INVALID_ORDER_ID), - new S3ObjectMetadata(9, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 95, 99)), 0, currentTimestamp, 20, - S3StreamConstant.INVALID_ORDER_ID)))); + .thenReturn(CompletableFuture.completedFuture(List.of( + new S3ObjectMetadata(1, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 5, 10)), 0, currentTimestamp - 20000, 60, + S3StreamConstant.INVALID_ORDER_ID), + new S3ObjectMetadata(2, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 10, 40)), 0, currentTimestamp, 20, + S3StreamConstant.INVALID_ORDER_ID), + new S3ObjectMetadata(3, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 40, 50)), 0, currentTimestamp - 20000, 20, + S3StreamConstant.INVALID_ORDER_ID), + new S3ObjectMetadata(4, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 50, 60)), 0, currentTimestamp, 20, + S3StreamConstant.INVALID_ORDER_ID), + new S3ObjectMetadata(5, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 60, 70)), 0, currentTimestamp - 30000, 20, + S3StreamConstant.INVALID_ORDER_ID), + new S3ObjectMetadata(6, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 70, 80)), 0, currentTimestamp - 30000, 20, + S3StreamConstant.INVALID_ORDER_ID), + new S3ObjectMetadata(7, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 80, 90)), 0, currentTimestamp - 30000, 20, + S3StreamConstant.INVALID_ORDER_ID), + new S3ObjectMetadata(8, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 90, 95)), 0, currentTimestamp - 30000, 80, + S3StreamConstant.INVALID_ORDER_ID), + new S3ObjectMetadata(9, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 95, 99)), 0, currentTimestamp, 20, + S3StreamConstant.INVALID_ORDER_ID)))); compactGroups = task2.prepareCompactGroups(0); assertEquals(1, compactGroups.size()); assertEquals(List.of( - new S3ObjectMetadata(5, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 60, 70)), 0, currentTimestamp - 30000, 20, - S3StreamConstant.INVALID_ORDER_ID), - new S3ObjectMetadata(6, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 70, 80)), 0, currentTimestamp - 30000, 20, - S3StreamConstant.INVALID_ORDER_ID), - new S3ObjectMetadata(7, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 80, 90)), 0, currentTimestamp - 30000, 20, - S3StreamConstant.INVALID_ORDER_ID)), - Objects.requireNonNull(compactGroups.poll()).stream().map(StreamObjectsCompactionTask.S3StreamObjectMetadataSplitWrapper::s3StreamObjectMetadata).collect(Collectors.toList())); + new S3ObjectMetadata(5, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 60, 70)), 0, currentTimestamp - 30000, 20, + S3StreamConstant.INVALID_ORDER_ID), + new S3ObjectMetadata(6, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 70, 80)), 0, currentTimestamp - 30000, 20, + S3StreamConstant.INVALID_ORDER_ID), + new S3ObjectMetadata(7, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 80, 90)), 0, currentTimestamp - 30000, 20, + S3StreamConstant.INVALID_ORDER_ID)), + Objects.requireNonNull(compactGroups.poll()).stream().map(StreamObjectsCompactionTask.S3StreamObjectMetadataSplitWrapper::s3StreamObjectMetadata).collect(Collectors.toList())); assertEquals(5, task2.getNextStartSearchingOffset()); // check if we can split big objects. @@ -297,14 +296,14 @@ void testPrepareCompactGroups() throws ExecutionException, InterruptedException currentTimestamp = System.currentTimeMillis(); when(objectManager.getStreamObjects(anyLong(), anyLong(), anyLong(), anyInt())) - .thenReturn(CompletableFuture.completedFuture(List.of( - new S3ObjectMetadata(1, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 5, 10)), 0, currentTimestamp, MAX_PART_SIZE + 10, - S3StreamConstant.INVALID_ORDER_ID), - new S3ObjectMetadata(2, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 10, 20)), 0, currentTimestamp, 2 * MAX_PART_SIZE, - S3StreamConstant.INVALID_ORDER_ID), - new S3ObjectMetadata(3, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 20, 30)), 0, currentTimestamp, MAX_PART_SIZE - 1, - S3StreamConstant.INVALID_ORDER_ID) - ))); + .thenReturn(CompletableFuture.completedFuture(List.of( + new S3ObjectMetadata(1, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 5, 10)), 0, currentTimestamp, MAX_PART_SIZE + 10, + S3StreamConstant.INVALID_ORDER_ID), + new S3ObjectMetadata(2, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 10, 20)), 0, currentTimestamp, 2 * MAX_PART_SIZE, + S3StreamConstant.INVALID_ORDER_ID), + new S3ObjectMetadata(3, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 20, 30)), 0, currentTimestamp, MAX_PART_SIZE - 1, + S3StreamConstant.INVALID_ORDER_ID) + ))); compactGroups = task3.prepareCompactGroups(0); assertEquals(1, compactGroups.size()); @@ -313,16 +312,16 @@ void testPrepareCompactGroups() throws ExecutionException, InterruptedException assertEquals(3, wrappers.size()); assertEquals(List.of( - new S3ObjectMetadata(1, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 5, 10)), 0, currentTimestamp, MAX_PART_SIZE + 10, - S3StreamConstant.INVALID_ORDER_ID), - new S3ObjectMetadata(2, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 10, 20)), 0, currentTimestamp, 2 * MAX_PART_SIZE, - S3StreamConstant.INVALID_ORDER_ID), - new S3ObjectMetadata(3, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 20, 30)), 0, currentTimestamp, MAX_PART_SIZE - 1, - S3StreamConstant.INVALID_ORDER_ID) - ), - Objects.requireNonNull(wrappers).stream().map(StreamObjectsCompactionTask.S3StreamObjectMetadataSplitWrapper::s3StreamObjectMetadata).collect(Collectors.toList())); + new S3ObjectMetadata(1, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 5, 10)), 0, currentTimestamp, MAX_PART_SIZE + 10, + S3StreamConstant.INVALID_ORDER_ID), + new S3ObjectMetadata(2, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 10, 20)), 0, currentTimestamp, 2 * MAX_PART_SIZE, + S3StreamConstant.INVALID_ORDER_ID), + new S3ObjectMetadata(3, S3ObjectType.STREAM, List.of(new StreamOffsetRange(1, 20, 30)), 0, currentTimestamp, MAX_PART_SIZE - 1, + S3StreamConstant.INVALID_ORDER_ID) + ), + Objects.requireNonNull(wrappers).stream().map(StreamObjectsCompactionTask.S3StreamObjectMetadataSplitWrapper::s3StreamObjectMetadata).collect(Collectors.toList())); assertEquals(List.of(2, 2, 1), - Objects.requireNonNull(wrappers).stream().map(StreamObjectsCompactionTask.S3StreamObjectMetadataSplitWrapper::splitCopyCount).collect(Collectors.toList())); + Objects.requireNonNull(wrappers).stream().map(StreamObjectsCompactionTask.S3StreamObjectMetadataSplitWrapper::splitCopyCount).collect(Collectors.toList())); assertEquals(5, task3.getNextStartSearchingOffset()); } } \ No newline at end of file diff --git a/s3stream/src/test/java/com/automq/stream/s3/TestUtils.java b/s3stream/src/test/java/com/automq/stream/s3/TestUtils.java index dd795f92c..b66de42c4 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/TestUtils.java +++ b/s3stream/src/test/java/com/automq/stream/s3/TestUtils.java @@ -20,7 +20,6 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.PooledByteBufAllocator; import io.netty.buffer.Unpooled; - import java.util.Random; import java.util.UUID; diff --git a/s3stream/src/test/java/com/automq/stream/s3/cache/BlockCacheTest.java b/s3stream/src/test/java/com/automq/stream/s3/cache/BlockCacheTest.java index 8a8b610ed..b07227d5d 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/cache/BlockCacheTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/cache/BlockCacheTest.java @@ -19,11 +19,10 @@ import com.automq.stream.s3.TestUtils; import com.automq.stream.s3.model.StreamRecordBatch; +import java.util.List; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; -import java.util.List; - import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNull; @@ -32,28 +31,32 @@ @Tag("S3Unit") public class BlockCacheTest { + private static StreamRecordBatch newRecord(long streamId, long offset, int count, int size) { + return new StreamRecordBatch(streamId, 0, offset, count, TestUtils.random(size)); + } + private BlockCache createBlockCache() { BlockCache blockCache = new BlockCache(1024 * 1024 * 1024); blockCache.put(233L, List.of( - newRecord(233L, 10L, 2, 1), - newRecord(233L, 12L, 2, 1) + newRecord(233L, 10L, 2, 1), + newRecord(233L, 12L, 2, 1) )); blockCache.put(233L, List.of( - newRecord(233L, 16L, 4, 1), - newRecord(233L, 20L, 2, 1) + newRecord(233L, 16L, 4, 1), + newRecord(233L, 20L, 2, 1) )); // overlap blockCache.put(233L, List.of( - newRecord(233L, 12L, 2, 1), - newRecord(233L, 14L, 1, 1), - newRecord(233L, 15L, 1, BlockCache.BLOCK_SIZE), - newRecord(233L, 16L, 4, 1), - newRecord(233L, 20L, 2, 1), - newRecord(233L, 22L, 1, 1), - newRecord(233L, 23L, 1, 1) + newRecord(233L, 12L, 2, 1), + newRecord(233L, 14L, 1, 1), + newRecord(233L, 15L, 1, BlockCache.BLOCK_SIZE), + newRecord(233L, 16L, 4, 1), + newRecord(233L, 20L, 2, 1), + newRecord(233L, 22L, 1, 1), + newRecord(233L, 23L, 1, 1) )); return blockCache; } @@ -90,8 +93,8 @@ public void testPutGet2() { public void testPutGet3() { BlockCache blockCache = createBlockCache(); blockCache.put(233L, 26L, 40L, List.of( - newRecord(233L, 26L, 4, 1), - newRecord(233L, 30L, 10, 4) + newRecord(233L, 26L, 4, 1), + newRecord(233L, 30L, 10, 4) )); BlockCache.GetCacheResult rst = blockCache.get(233L, 27L, 35L, BlockCache.BLOCK_SIZE * 2); @@ -107,8 +110,8 @@ public void testPutGet3() { public void testRangeCheck() { BlockCache blockCache = createBlockCache(); blockCache.put(233L, List.of( - newRecord(233L, 26L, 4, 1), - newRecord(233L, 30L, 10, 4) + newRecord(233L, 26L, 4, 1), + newRecord(233L, 30L, 10, 4) )); assertTrue(blockCache.checkRange(233, 10, 2)); @@ -122,15 +125,15 @@ public void testRangeCheck() { public void testEvict() { BlockCache blockCache = new BlockCache(4); blockCache.put(233L, List.of( - newRecord(233L, 10L, 2, 2), - newRecord(233L, 12L, 2, 1) + newRecord(233L, 10L, 2, 2), + newRecord(233L, 12L, 2, 1) )); assertEquals(2, blockCache.get(233L, 10L, 20L, 1000).getRecords().size()); blockCache.put(233L, List.of( - newRecord(233L, 16L, 4, 1), - newRecord(233L, 20L, 2, 1) + newRecord(233L, 16L, 4, 1), + newRecord(233L, 20L, 2, 1) )); assertEquals(0, blockCache.get(233L, 10L, 20L, 1000).getRecords().size()); assertEquals(2, blockCache.get(233L, 16, 21L, 1000).getRecords().size()); @@ -153,8 +156,8 @@ public void testLRU() { public void testReadAhead() { BlockCache blockCache = new BlockCache(16 * 1024 * 1024); blockCache.put(233L, 10L, 12L, List.of( - newRecord(233L, 10, 1, 1024 * 1024), - newRecord(233L, 11, 1, 1024) + newRecord(233L, 10, 1, 1024 * 1024), + newRecord(233L, 11, 1, 1024) )); // first read the block @@ -168,8 +171,4 @@ public void testReadAhead() { assertTrue(rst.getReadAheadRecords().isEmpty()); } - private static StreamRecordBatch newRecord(long streamId, long offset, int count, int size) { - return new StreamRecordBatch(streamId, 0, offset, count, TestUtils.random(size)); - } - } diff --git a/s3stream/src/test/java/com/automq/stream/s3/cache/DataBlockReadAccumulatorTest.java b/s3stream/src/test/java/com/automq/stream/s3/cache/DataBlockReadAccumulatorTest.java index ad970384f..33d7e3590 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/cache/DataBlockReadAccumulatorTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/cache/DataBlockReadAccumulatorTest.java @@ -22,16 +22,15 @@ import com.automq.stream.s3.TestUtils; import com.automq.stream.s3.model.StreamRecordBatch; import com.automq.stream.utils.CloseableIterator; -import org.apache.commons.lang3.tuple.ImmutablePair; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - import java.util.Iterator; import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.ArgumentMatchers.any; @@ -43,6 +42,10 @@ public class DataBlockReadAccumulatorTest { + private static StreamRecordBatch newRecord(long streamId, long offset, int count, int size) { + return new StreamRecordBatch(streamId, 0, offset, count, TestUtils.random(size)); + } + @Test public void test() throws ExecutionException, InterruptedException, TimeoutException { DataBlockReadAccumulator accumulator = new DataBlockReadAccumulator(); @@ -65,8 +68,8 @@ public void test() throws ExecutionException, InterruptedException, TimeoutExcep ObjectReader.DataBlock dataBlock = mock(ObjectReader.DataBlock.class); List records = List.of( - newRecord(10, 10, 2, 1), - newRecord(10, 12, 2, 1) + newRecord(10, 10, 2, 1), + newRecord(10, 12, 2, 1) ); when(dataBlock.recordCount()).thenReturn(2); when(dataBlock.iterator()).thenAnswer(args -> { @@ -111,8 +114,4 @@ public void close() { reserveResults3.get(0).cf().get().release(); } - private static StreamRecordBatch newRecord(long streamId, long offset, int count, int size) { - return new StreamRecordBatch(streamId, 0, offset, count, TestUtils.random(size)); - } - } diff --git a/s3stream/src/test/java/com/automq/stream/s3/cache/InflightReadThrottleTest.java b/s3stream/src/test/java/com/automq/stream/s3/cache/InflightReadThrottleTest.java index 99afebba0..59f9fc2e8 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/cache/InflightReadThrottleTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/cache/InflightReadThrottleTest.java @@ -18,11 +18,10 @@ package com.automq.stream.s3.cache; import com.automq.stream.utils.Threads; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - import java.util.UUID; import java.util.concurrent.CompletableFuture; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; public class InflightReadThrottleTest { @Test diff --git a/s3stream/src/test/java/com/automq/stream/s3/cache/LogCacheTest.java b/s3stream/src/test/java/com/automq/stream/s3/cache/LogCacheTest.java index 51f6e1946..68d0ade86 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/cache/LogCacheTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/cache/LogCacheTest.java @@ -19,16 +19,14 @@ import com.automq.stream.s3.TestUtils; import com.automq.stream.s3.model.StreamRecordBatch; -import org.junit.jupiter.api.Test; - import java.util.List; import java.util.Map; +import org.junit.jupiter.api.Test; import static org.junit.jupiter.api.Assertions.assertEquals; public class LogCacheTest { - @Test public void testPutGet() { LogCache logCache = new LogCache(1024 * 1024, 1024 * 1024); diff --git a/s3stream/src/test/java/com/automq/stream/s3/cache/ObjectReaderLRUCacheTest.java b/s3stream/src/test/java/com/automq/stream/s3/cache/ObjectReaderLRUCacheTest.java index 22201d7fc..e9c8e56e2 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/cache/ObjectReaderLRUCacheTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/cache/ObjectReaderLRUCacheTest.java @@ -25,12 +25,11 @@ import com.automq.stream.s3.model.StreamRecordBatch; import com.automq.stream.s3.operator.MemoryS3Operator; import com.automq.stream.s3.operator.S3Operator; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - import java.util.List; import java.util.Map; import java.util.concurrent.ExecutionException; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; public class ObjectReaderLRUCacheTest { diff --git a/s3stream/src/test/java/com/automq/stream/s3/cache/StreamCacheTest.java b/s3stream/src/test/java/com/automq/stream/s3/cache/StreamCacheTest.java index 57e1d831b..b213a9d82 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/cache/StreamCacheTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/cache/StreamCacheTest.java @@ -19,11 +19,10 @@ import com.automq.stream.s3.TestUtils; import com.automq.stream.s3.model.StreamRecordBatch; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - import java.util.List; import java.util.NavigableMap; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; public class StreamCacheTest { @@ -31,17 +30,17 @@ public class StreamCacheTest { public void testTailMap() { StreamCache streamCache = new StreamCache(); BlockCache.CacheBlock block1 = new BlockCache.CacheBlock(List.of( - new StreamRecordBatch(0, 0, 0, 10, TestUtils.random(8)), - new StreamRecordBatch(0, 0, 10, 20, TestUtils.random(8))), null); + new StreamRecordBatch(0, 0, 0, 10, TestUtils.random(8)), + new StreamRecordBatch(0, 0, 10, 20, TestUtils.random(8))), null); streamCache.put(block1); BlockCache.CacheBlock block2 = new BlockCache.CacheBlock(List.of( - new StreamRecordBatch(0, 0, 50, 20, TestUtils.random(8)), - new StreamRecordBatch(0, 0, 70, 30, TestUtils.random(8))), null); + new StreamRecordBatch(0, 0, 50, 20, TestUtils.random(8)), + new StreamRecordBatch(0, 0, 70, 30, TestUtils.random(8))), null); streamCache.put(block2); BlockCache.CacheBlock block3 = new BlockCache.CacheBlock(List.of( - new StreamRecordBatch(0, 0, 30, 20, TestUtils.random(8))), null); + new StreamRecordBatch(0, 0, 30, 20, TestUtils.random(8))), null); streamCache.put(block3); NavigableMap tailBlocks = streamCache.tailBlocks(5); @@ -57,15 +56,15 @@ public void testTailMap() { public void testRemove() { StreamCache streamCache = new StreamCache(); streamCache.put(new BlockCache.CacheBlock(List.of( - new StreamRecordBatch(0, 0, 0, 10, TestUtils.random(8)), - new StreamRecordBatch(0, 0, 10, 20, TestUtils.random(8))), null)); + new StreamRecordBatch(0, 0, 0, 10, TestUtils.random(8)), + new StreamRecordBatch(0, 0, 10, 20, TestUtils.random(8))), null)); streamCache.put(new BlockCache.CacheBlock(List.of( - new StreamRecordBatch(0, 0, 50, 20, TestUtils.random(8)), - new StreamRecordBatch(0, 0, 70, 30, TestUtils.random(8))), null)); + new StreamRecordBatch(0, 0, 50, 20, TestUtils.random(8)), + new StreamRecordBatch(0, 0, 70, 30, TestUtils.random(8))), null)); streamCache.put(new BlockCache.CacheBlock(List.of( - new StreamRecordBatch(0, 0, 30, 20, TestUtils.random(8))), null)); + new StreamRecordBatch(0, 0, 30, 20, TestUtils.random(8))), null)); streamCache.remove(30); diff --git a/s3stream/src/test/java/com/automq/stream/s3/cache/StreamReaderTest.java b/s3stream/src/test/java/com/automq/stream/s3/cache/StreamReaderTest.java index 01a5a37e1..bf5a988f8 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/cache/StreamReaderTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/cache/StreamReaderTest.java @@ -32,11 +32,6 @@ import com.automq.stream.s3.trace.context.TraceContext; import com.automq.stream.utils.CloseableIterator; import com.automq.stream.utils.Threads; -import org.apache.commons.lang3.tuple.ImmutablePair; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.Mockito; - import java.util.HashMap; import java.util.List; import java.util.Map; @@ -44,6 +39,10 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyLong; @@ -59,14 +58,14 @@ public void testGetDataBlockIndices() { ObjectManager objectManager = Mockito.mock(ObjectManager.class); ObjectWriter objectWriter = ObjectWriter.writer(0, s3Operator, 1024, 1024); objectWriter.write(233, List.of( - newRecord(233, 10, 5, 512), - newRecord(233, 15, 10, 512) + newRecord(233, 10, 5, 512), + newRecord(233, 15, 10, 512) )); objectWriter.close(); ObjectWriter objectWriter2 = ObjectWriter.writer(1, s3Operator, 1024, 1024); objectWriter2.write(233, List.of( - newRecord(233, 25, 5, 512), - newRecord(233, 30, 10, 512) + newRecord(233, 25, 5, 512), + newRecord(233, 30, 10, 512) )); objectWriter2.close(); @@ -74,7 +73,7 @@ public void testGetDataBlockIndices() { S3ObjectMetadata metadata2 = new S3ObjectMetadata(1, objectWriter2.size(), S3ObjectType.STREAM); doAnswer(invocation -> CompletableFuture.completedFuture(List.of(metadata1, metadata2))) - .when(objectManager).getObjects(eq(233L), eq(15L), eq(1024L), eq(2)); + .when(objectManager).getObjects(eq(233L), eq(15L), eq(1024L), eq(2)); StreamReader streamReader = new StreamReader(s3Operator, objectManager, Mockito.mock(BlockCache.class), new HashMap<>(), new InflightReadThrottle()); StreamReader.ReadContext context = new StreamReader.ReadContext(15L, 1024); @@ -107,13 +106,13 @@ public void testSyncReadAheadInflight() { long objectId = 1; S3ObjectMetadata metadata = new S3ObjectMetadata(objectId, -1, S3ObjectType.STREAM); doAnswer(invocation -> CompletableFuture.completedFuture(List.of(metadata))) - .when(objectManager).getObjects(eq(streamId), eq(startOffset), anyLong(), anyInt()); + .when(objectManager).getObjects(eq(streamId), eq(startOffset), anyLong(), anyInt()); ObjectReader reader = Mockito.mock(ObjectReader.class); ObjectReader.DataBlockIndex index1 = new ObjectReader.DataBlockIndex(0, 0, 256, 128); doReturn(reader).when(streamReader).getObjectReader(metadata); doAnswer(invocation -> CompletableFuture.completedFuture(new ObjectReader.FindIndexResult(true, -1, -1, - List.of(new StreamDataBlock(streamId, 64, 128, objectId, index1))))).when(reader).find(eq(streamId), eq(startOffset), anyLong(), eq(maxBytes)); + List.of(new StreamDataBlock(streamId, 64, 128, objectId, index1))))).when(reader).find(eq(streamId), eq(startOffset), anyLong(), eq(maxBytes)); doReturn(new CompletableFuture<>()).when(reader).read(index1); streamReader.syncReadAhead(TraceContext.DEFAULT, streamId, startOffset, endOffset, maxBytes, Mockito.mock(ReadAheadAgent.class), UUID.randomUUID()); @@ -139,8 +138,8 @@ public void testSyncReadAhead() { StreamReader.ReadContext context = new StreamReader.ReadContext(0, 256); ObjectReader.DataBlockIndex index1 = new ObjectReader.DataBlockIndex(0, 0, 256, 128); context.streamDataBlocksPair = List.of( - new ImmutablePair<>(1L, List.of( - new StreamDataBlock(233L, 0, 128, 1, index1)))); + new ImmutablePair<>(1L, List.of( + new StreamDataBlock(233L, 0, 128, 1, index1)))); ObjectReader reader = Mockito.mock(ObjectReader.class); ObjectReader.DataBlock dataBlock1 = Mockito.mock(ObjectReader.DataBlock.class); @@ -172,8 +171,8 @@ public StreamRecordBatch next() { }); Mockito.when(reader.read(index1)).thenReturn(CompletableFuture.completedFuture(dataBlock1)); context.objectReaderMap = new HashMap<>(Map.of(1L, reader)); - CompletableFuture> cf = streamReader.handleSyncReadAhead(TraceContext.DEFAULT,233L, 0, - 999, 64, Mockito.mock(ReadAheadAgent.class), UUID.randomUUID(), new TimerUtil(), context); + CompletableFuture> cf = streamReader.handleSyncReadAhead(TraceContext.DEFAULT, 233L, 0, + 999, 64, Mockito.mock(ReadAheadAgent.class), UUID.randomUUID(), new TimerUtil(), context); cf.whenComplete((rst, ex) -> { Assertions.assertNull(ex); @@ -198,8 +197,8 @@ public void testSyncReadAheadNotAlign() { StreamReader.ReadContext context = new StreamReader.ReadContext(startOffset, 256); ObjectReader.DataBlockIndex index1 = new ObjectReader.DataBlockIndex(0, 0, 256, 128); context.streamDataBlocksPair = List.of( - new ImmutablePair<>(1L, List.of( - new StreamDataBlock(233L, 0, 128, 1, index1)))); + new ImmutablePair<>(1L, List.of( + new StreamDataBlock(233L, 0, 128, 1, index1)))); ObjectReader reader = Mockito.mock(ObjectReader.class); ObjectReader.DataBlock dataBlock1 = Mockito.mock(ObjectReader.DataBlock.class); @@ -235,7 +234,7 @@ public StreamRecordBatch next() { context.taskKeySet.add(key); inflightReadAheadTasks.put(key, new DefaultS3BlockCache.ReadAheadTaskContext(new CompletableFuture<>(), DefaultS3BlockCache.ReadBlockCacheStatus.INIT)); CompletableFuture> cf = streamReader.handleSyncReadAhead(TraceContext.DEFAULT, 233L, startOffset, - 999, 64, Mockito.mock(ReadAheadAgent.class), UUID.randomUUID(), new TimerUtil(), context); + 999, 64, Mockito.mock(ReadAheadAgent.class), UUID.randomUUID(), new TimerUtil(), context); cf.whenComplete((rst, ex) -> { Assertions.assertNull(ex); @@ -260,9 +259,9 @@ public void testSyncReadAheadException() { ObjectReader.DataBlockIndex index1 = new ObjectReader.DataBlockIndex(0, 0, 256, 128); ObjectReader.DataBlockIndex index2 = new ObjectReader.DataBlockIndex(1, 256, 256, 128); context.streamDataBlocksPair = List.of( - new ImmutablePair<>(1L, List.of( - new StreamDataBlock(233L, 0, 128, 1, index1), - new StreamDataBlock(233L, 128, 256, 1, index2)))); + new ImmutablePair<>(1L, List.of( + new StreamDataBlock(233L, 0, 128, 1, index1), + new StreamDataBlock(233L, 128, 256, 1, index2)))); ObjectReader reader = Mockito.mock(ObjectReader.class); ObjectReader.DataBlock dataBlock1 = Mockito.mock(ObjectReader.DataBlock.class); @@ -296,7 +295,7 @@ public StreamRecordBatch next() { Mockito.when(reader.read(index2)).thenReturn(CompletableFuture.failedFuture(new RuntimeException("exception"))); context.objectReaderMap = new HashMap<>(Map.of(1L, reader)); CompletableFuture> cf = streamReader.handleSyncReadAhead(TraceContext.DEFAULT, 233L, 0, - 512, 1024, Mockito.mock(ReadAheadAgent.class), UUID.randomUUID(), new TimerUtil(), context); + 512, 1024, Mockito.mock(ReadAheadAgent.class), UUID.randomUUID(), new TimerUtil(), context); Threads.sleep(1000); @@ -326,8 +325,8 @@ public void testAsyncReadAhead() { StreamReader.ReadContext context = new StreamReader.ReadContext(0, 256); ObjectReader.DataBlockIndex index1 = new ObjectReader.DataBlockIndex(0, 0, 256, 128); context.streamDataBlocksPair = List.of( - new ImmutablePair<>(1L, List.of( - new StreamDataBlock(233L, 0, 128, 1, index1)))); + new ImmutablePair<>(1L, List.of( + new StreamDataBlock(233L, 0, 128, 1, index1)))); ObjectReader reader = Mockito.mock(ObjectReader.class); ObjectReader.DataBlock dataBlock1 = Mockito.mock(ObjectReader.DataBlock.class); @@ -382,9 +381,9 @@ public void testAsyncReadAheadException() { ObjectReader.DataBlockIndex index1 = new ObjectReader.DataBlockIndex(0, 0, 256, 128); ObjectReader.DataBlockIndex index2 = new ObjectReader.DataBlockIndex(1, 256, 256, 128); context.streamDataBlocksPair = List.of( - new ImmutablePair<>(1L, List.of( - new StreamDataBlock(233L, 0, 128, 1, index1), - new StreamDataBlock(233L, 128, 256, 1, index2)))); + new ImmutablePair<>(1L, List.of( + new StreamDataBlock(233L, 0, 128, 1, index1), + new StreamDataBlock(233L, 128, 256, 1, index2)))); ObjectReader reader = Mockito.mock(ObjectReader.class); ObjectReader.DataBlock dataBlock1 = Mockito.mock(ObjectReader.DataBlock.class); diff --git a/s3stream/src/test/java/com/automq/stream/s3/compact/CompactionAnalyzerTest.java b/s3stream/src/test/java/com/automq/stream/s3/compact/CompactionAnalyzerTest.java index def58a44a..37f49edc5 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/compact/CompactionAnalyzerTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/compact/CompactionAnalyzerTest.java @@ -17,22 +17,15 @@ package com.automq.stream.s3.compact; +import com.automq.stream.s3.StreamDataBlock; 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.StreamDataBlock; import com.automq.stream.s3.metadata.S3ObjectMetadata; import com.automq.stream.s3.metadata.S3ObjectType; import com.automq.stream.s3.metadata.StreamMetadata; import com.automq.stream.s3.metadata.StreamOffsetRange; import com.automq.stream.s3.metadata.StreamState; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Tag; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.Timeout; - import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; @@ -40,6 +33,12 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.when; @@ -48,6 +47,21 @@ @Tag("S3Unit") public class CompactionAnalyzerTest extends CompactionTestBase { + private static Map> generateStreamDataBlocks() { + return Map.of( + OBJECT_0, List.of( + new StreamDataBlock(STREAM_0, 0, 20, 0, OBJECT_0, -1, 20, 1), + new StreamDataBlock(STREAM_1, 30, 60, 1, OBJECT_0, -1, 30, 1), + new StreamDataBlock(STREAM_2, 30, 60, 2, OBJECT_0, -1, 30, 1)), + OBJECT_1, List.of( + new StreamDataBlock(STREAM_0, 20, 25, 0, OBJECT_1, -1, 5, 1), + new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, 60, 1)), + OBJECT_2, List.of( + new StreamDataBlock(STREAM_1, 400, 500, 0, OBJECT_2, -1, 100, 1), + new StreamDataBlock(STREAM_2, 230, 270, 1, OBJECT_2, -1, 40, 1)) + ); + } + @BeforeEach public void setUp() throws Exception { super.setUp(); @@ -63,39 +77,39 @@ public void testReadObjectIndices() { List streamMetadataList = this.streamManager.getStreams(Collections.emptyList()).join(); Map> streamDataBlocksMap = CompactionUtils.blockWaitObjectIndices(streamMetadataList, S3_WAL_OBJECT_METADATA_LIST, s3Operator, null); Map> expectedBlocksMap = Map.of( - OBJECT_0, List.of( - new StreamDataBlock(STREAM_0, 0, 15, 0, OBJECT_0, -1, -1, 1), - new StreamDataBlock(STREAM_1, 25, 30, 1, OBJECT_0, -1, -1, 1), - new StreamDataBlock(STREAM_1, 30, 60, 2, OBJECT_0, -1, -1, 1), - new StreamDataBlock(STREAM_2, 30, 60, 3, OBJECT_0, -1, -1, 1)), - OBJECT_1, List.of( - new StreamDataBlock(STREAM_0, 15, 20, 0, OBJECT_1, -1, -1, 1), - new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1)), - OBJECT_2, List.of( - new StreamDataBlock(STREAM_1, 400, 500, 0, OBJECT_2, -1, -1, 1), - new StreamDataBlock(STREAM_2, 230, 270, 1, OBJECT_2, -1, -1, 1))); + OBJECT_0, List.of( + new StreamDataBlock(STREAM_0, 0, 15, 0, OBJECT_0, -1, -1, 1), + new StreamDataBlock(STREAM_1, 25, 30, 1, OBJECT_0, -1, -1, 1), + new StreamDataBlock(STREAM_1, 30, 60, 2, OBJECT_0, -1, -1, 1), + new StreamDataBlock(STREAM_2, 30, 60, 3, OBJECT_0, -1, -1, 1)), + OBJECT_1, List.of( + new StreamDataBlock(STREAM_0, 15, 20, 0, OBJECT_1, -1, -1, 1), + new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1)), + OBJECT_2, List.of( + new StreamDataBlock(STREAM_1, 400, 500, 0, OBJECT_2, -1, -1, 1), + new StreamDataBlock(STREAM_2, 230, 270, 1, OBJECT_2, -1, -1, 1))); assertTrue(compare(streamDataBlocksMap, expectedBlocksMap)); } @Test public void testReadObjectIndicesWithTrimmedData() { when(streamManager.getStreams(Collections.emptyList())).thenReturn(CompletableFuture.completedFuture( - List.of(new StreamMetadata(STREAM_0, 0, 15, 20, StreamState.OPENED), - new StreamMetadata(STREAM_1, 0, 25, 500, StreamState.OPENED), - new StreamMetadata(STREAM_2, 0, 30, 270, StreamState.OPENED)))); + List.of(new StreamMetadata(STREAM_0, 0, 15, 20, StreamState.OPENED), + new StreamMetadata(STREAM_1, 0, 25, 500, StreamState.OPENED), + new StreamMetadata(STREAM_2, 0, 30, 270, StreamState.OPENED)))); List streamMetadataList = this.streamManager.getStreams(Collections.emptyList()).join(); Map> streamDataBlocksMap = CompactionUtils.blockWaitObjectIndices(streamMetadataList, S3_WAL_OBJECT_METADATA_LIST, s3Operator); Map> expectedBlocksMap = Map.of( - OBJECT_0, List.of( - new StreamDataBlock(STREAM_1, 25, 30, 1, OBJECT_0, -1, -1, 1), - new StreamDataBlock(STREAM_1, 30, 60, 2, OBJECT_0, -1, -1, 1), - new StreamDataBlock(STREAM_2, 30, 60, 3, OBJECT_0, -1, -1, 1)), - OBJECT_1, List.of( - new StreamDataBlock(STREAM_0, 15, 20, 0, OBJECT_1, -1, -1, 1), - new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1)), - OBJECT_2, List.of( - new StreamDataBlock(STREAM_1, 400, 500, 0, OBJECT_2, -1, -1, 1), - new StreamDataBlock(STREAM_2, 230, 270, 1, OBJECT_2, -1, -1, 1))); + OBJECT_0, List.of( + new StreamDataBlock(STREAM_1, 25, 30, 1, OBJECT_0, -1, -1, 1), + new StreamDataBlock(STREAM_1, 30, 60, 2, OBJECT_0, -1, -1, 1), + new StreamDataBlock(STREAM_2, 30, 60, 3, OBJECT_0, -1, -1, 1)), + OBJECT_1, List.of( + new StreamDataBlock(STREAM_0, 15, 20, 0, OBJECT_1, -1, -1, 1), + new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1)), + OBJECT_2, List.of( + new StreamDataBlock(STREAM_1, 400, 500, 0, OBJECT_2, -1, -1, 1), + new StreamDataBlock(STREAM_2, 230, 270, 1, OBJECT_2, -1, -1, 1))); assertTrue(compare(streamDataBlocksMap, expectedBlocksMap)); } @@ -112,24 +126,24 @@ public void testFilterBlocksToCompact() { public void testFilterBlocksToCompact2() { CompactionAnalyzer compactionAnalyzer = new CompactionAnalyzer(CACHE_SIZE, STREAM_SPLIT_SIZE, MAX_STREAM_NUM_IN_WAL, MAX_STREAM_OBJECT_NUM); Map> streamDataBlocksMap = Map.of( - OBJECT_0, List.of( - new StreamDataBlock(STREAM_0, 0, 20, 0, OBJECT_0, -1, -1, 1), - new StreamDataBlock(STREAM_1, 30, 60, 1, OBJECT_0, -1, -1, 1)), - OBJECT_1, List.of( - new StreamDataBlock(STREAM_0, 20, 25, 0, OBJECT_1, -1, -1, 1), - new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1)), - OBJECT_2, List.of( - new StreamDataBlock(STREAM_2, 230, 270, 1, OBJECT_2, -1, -1, 1)), - OBJECT_3, List.of( - new StreamDataBlock(STREAM_3, 0, 50, 1, OBJECT_3, -1, -1, 1))); + OBJECT_0, List.of( + new StreamDataBlock(STREAM_0, 0, 20, 0, OBJECT_0, -1, -1, 1), + new StreamDataBlock(STREAM_1, 30, 60, 1, OBJECT_0, -1, -1, 1)), + OBJECT_1, List.of( + new StreamDataBlock(STREAM_0, 20, 25, 0, OBJECT_1, -1, -1, 1), + new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1)), + OBJECT_2, List.of( + new StreamDataBlock(STREAM_2, 230, 270, 1, OBJECT_2, -1, -1, 1)), + OBJECT_3, List.of( + new StreamDataBlock(STREAM_3, 0, 50, 1, OBJECT_3, -1, -1, 1))); Map> result = compactionAnalyzer.filterBlocksToCompact(streamDataBlocksMap); Map> expectedBlocksMap = Map.of( - OBJECT_0, List.of( - new StreamDataBlock(STREAM_0, 0, 20, 0, OBJECT_0, -1, -1, 1), - new StreamDataBlock(STREAM_1, 30, 60, 1, OBJECT_0, -1, -1, 1)), - OBJECT_1, List.of( - new StreamDataBlock(STREAM_0, 20, 25, 0, OBJECT_1, -1, -1, 1), - new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1))); + OBJECT_0, List.of( + new StreamDataBlock(STREAM_0, 0, 20, 0, OBJECT_0, -1, -1, 1), + new StreamDataBlock(STREAM_1, 30, 60, 1, OBJECT_0, -1, -1, 1)), + OBJECT_1, List.of( + new StreamDataBlock(STREAM_0, 20, 25, 0, OBJECT_1, -1, -1, 1), + new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1))); assertTrue(compare(result, expectedBlocksMap)); } @@ -140,14 +154,14 @@ public void testSortStreamRangePositions() { Map> streamDataBlocksMap = CompactionUtils.blockWaitObjectIndices(streamMetadataList, S3_WAL_OBJECT_METADATA_LIST, s3Operator); List sortedStreamDataBlocks = compactionAnalyzer.sortStreamRangePositions(streamDataBlocksMap); List expectedBlocks = List.of( - new StreamDataBlock(STREAM_0, 0, 15, 0, OBJECT_0, -1, -1, 1), - new StreamDataBlock(STREAM_0, 15, 20, 0, OBJECT_1, -1, -1, 1), - new StreamDataBlock(STREAM_1, 25, 30, 1, OBJECT_0, -1, -1, 1), - new StreamDataBlock(STREAM_1, 30, 60, 2, OBJECT_0, -1, -1, 1), - new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1), - new StreamDataBlock(STREAM_1, 400, 500, 0, OBJECT_2, -1, -1, 1), - new StreamDataBlock(STREAM_2, 30, 60, 3, OBJECT_0, -1, -1, 1), - new StreamDataBlock(STREAM_2, 230, 270, 1, OBJECT_2, -1, -1, 1)); + new StreamDataBlock(STREAM_0, 0, 15, 0, OBJECT_0, -1, -1, 1), + new StreamDataBlock(STREAM_0, 15, 20, 0, OBJECT_1, -1, -1, 1), + new StreamDataBlock(STREAM_1, 25, 30, 1, OBJECT_0, -1, -1, 1), + new StreamDataBlock(STREAM_1, 30, 60, 2, OBJECT_0, -1, -1, 1), + new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1), + new StreamDataBlock(STREAM_1, 400, 500, 0, OBJECT_2, -1, -1, 1), + new StreamDataBlock(STREAM_2, 30, 60, 3, OBJECT_0, -1, -1, 1), + new StreamDataBlock(STREAM_2, 230, 270, 1, OBJECT_2, -1, -1, 1)); for (int i = 0; i < sortedStreamDataBlocks.size(); i++) { assertTrue(compare(sortedStreamDataBlocks.get(i), expectedBlocks.get(i))); } @@ -162,24 +176,24 @@ public void testGroupObjectWithLimit() { List compactedObjectBuilders = compactionAnalyzer.groupObjectWithLimits(streamDataBlocksMap, objectsToRemove); Assertions.assertTrue(objectsToRemove.isEmpty()); List expectedCompactedObject = List.of( - new CompactedObjectBuilder() - .setType(CompactionType.COMPACT) - .addStreamDataBlock(new StreamDataBlock(STREAM_0, 0, 15, 0, OBJECT_0, -1, -1, 1)) - .addStreamDataBlock(new StreamDataBlock(STREAM_0, 15, 20, 0, OBJECT_1, -1, -1, 1)), - new CompactedObjectBuilder() - .setType(CompactionType.SPLIT) - .addStreamDataBlock(new StreamDataBlock(STREAM_1, 25, 30, 1, OBJECT_0, -1, -1, 1)) - .addStreamDataBlock(new StreamDataBlock(STREAM_1, 30, 60, 2, OBJECT_0, -1, -1, 1)) - .addStreamDataBlock(new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1)), - new CompactedObjectBuilder() - .setType(CompactionType.SPLIT) - .addStreamDataBlock(new StreamDataBlock(STREAM_1, 400, 500, 0, OBJECT_2, -1, -1, 1)), - new CompactedObjectBuilder() - .setType(CompactionType.SPLIT) - .addStreamDataBlock(new StreamDataBlock(STREAM_2, 30, 60, 3, OBJECT_0, -1, -1, 1)), - new CompactedObjectBuilder() - .setType(CompactionType.COMPACT) - .addStreamDataBlock(new StreamDataBlock(STREAM_2, 230, 270, 1, OBJECT_2, -1, -1, 1))); + new CompactedObjectBuilder() + .setType(CompactionType.COMPACT) + .addStreamDataBlock(new StreamDataBlock(STREAM_0, 0, 15, 0, OBJECT_0, -1, -1, 1)) + .addStreamDataBlock(new StreamDataBlock(STREAM_0, 15, 20, 0, OBJECT_1, -1, -1, 1)), + new CompactedObjectBuilder() + .setType(CompactionType.SPLIT) + .addStreamDataBlock(new StreamDataBlock(STREAM_1, 25, 30, 1, OBJECT_0, -1, -1, 1)) + .addStreamDataBlock(new StreamDataBlock(STREAM_1, 30, 60, 2, OBJECT_0, -1, -1, 1)) + .addStreamDataBlock(new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1)), + new CompactedObjectBuilder() + .setType(CompactionType.SPLIT) + .addStreamDataBlock(new StreamDataBlock(STREAM_1, 400, 500, 0, OBJECT_2, -1, -1, 1)), + new CompactedObjectBuilder() + .setType(CompactionType.SPLIT) + .addStreamDataBlock(new StreamDataBlock(STREAM_2, 30, 60, 3, OBJECT_0, -1, -1, 1)), + new CompactedObjectBuilder() + .setType(CompactionType.COMPACT) + .addStreamDataBlock(new StreamDataBlock(STREAM_2, 230, 270, 1, OBJECT_2, -1, -1, 1))); for (int i = 0; i < compactedObjectBuilders.size(); i++) { assertTrue(compare(compactedObjectBuilders.get(i), expectedCompactedObject.get(i))); } @@ -194,24 +208,24 @@ public void testGroupObjectWithLimit2() { List compactedObjectBuilders = compactionAnalyzer.groupObjectWithLimits(streamDataBlocksMap, objectsToRemove); Assertions.assertTrue(objectsToRemove.isEmpty()); List expectedCompactedObject = List.of( - new CompactedObjectBuilder() - .setType(CompactionType.SPLIT) - .addStreamDataBlock(new StreamDataBlock(STREAM_0, 0, 15, 0, OBJECT_0, -1, -1, 1)) - .addStreamDataBlock(new StreamDataBlock(STREAM_0, 15, 20, 0, OBJECT_1, -1, -1, 1)), - new CompactedObjectBuilder() - .setType(CompactionType.SPLIT) - .addStreamDataBlock(new StreamDataBlock(STREAM_1, 25, 30, 1, OBJECT_0, -1, -1, 1)) - .addStreamDataBlock(new StreamDataBlock(STREAM_1, 30, 60, 2, OBJECT_0, -1, -1, 1)) - .addStreamDataBlock(new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1)), - new CompactedObjectBuilder() - .setType(CompactionType.SPLIT) - .addStreamDataBlock(new StreamDataBlock(STREAM_1, 400, 500, 0, OBJECT_2, -1, -1, 1)), - new CompactedObjectBuilder() - .setType(CompactionType.SPLIT) - .addStreamDataBlock(new StreamDataBlock(STREAM_2, 30, 60, 3, OBJECT_0, -1, -1, 1)), - new CompactedObjectBuilder() - .setType(CompactionType.SPLIT) - .addStreamDataBlock(new StreamDataBlock(STREAM_2, 230, 270, 1, OBJECT_2, -1, -1, 1))); + new CompactedObjectBuilder() + .setType(CompactionType.SPLIT) + .addStreamDataBlock(new StreamDataBlock(STREAM_0, 0, 15, 0, OBJECT_0, -1, -1, 1)) + .addStreamDataBlock(new StreamDataBlock(STREAM_0, 15, 20, 0, OBJECT_1, -1, -1, 1)), + new CompactedObjectBuilder() + .setType(CompactionType.SPLIT) + .addStreamDataBlock(new StreamDataBlock(STREAM_1, 25, 30, 1, OBJECT_0, -1, -1, 1)) + .addStreamDataBlock(new StreamDataBlock(STREAM_1, 30, 60, 2, OBJECT_0, -1, -1, 1)) + .addStreamDataBlock(new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1)), + new CompactedObjectBuilder() + .setType(CompactionType.SPLIT) + .addStreamDataBlock(new StreamDataBlock(STREAM_1, 400, 500, 0, OBJECT_2, -1, -1, 1)), + new CompactedObjectBuilder() + .setType(CompactionType.SPLIT) + .addStreamDataBlock(new StreamDataBlock(STREAM_2, 30, 60, 3, OBJECT_0, -1, -1, 1)), + new CompactedObjectBuilder() + .setType(CompactionType.SPLIT) + .addStreamDataBlock(new StreamDataBlock(STREAM_2, 230, 270, 1, OBJECT_2, -1, -1, 1))); for (int i = 0; i < compactedObjectBuilders.size(); i++) { assertTrue(compare(compactedObjectBuilders.get(i), expectedCompactedObject.get(i))); } @@ -226,18 +240,18 @@ public void testGroupObjectWithLimit3() { List compactedObjectBuilders = compactionAnalyzer.groupObjectWithLimits(streamDataBlocksMap, objectsToRemove); Assertions.assertEquals(Set.of(OBJECT_2), objectsToRemove); List expectedCompactedObject = List.of( - new CompactedObjectBuilder() - .setType(CompactionType.SPLIT) - .addStreamDataBlock(new StreamDataBlock(STREAM_0, 0, 15, 0, OBJECT_0, -1, -1, 1)) - .addStreamDataBlock(new StreamDataBlock(STREAM_0, 15, 20, 0, OBJECT_1, -1, -1, 1)), - new CompactedObjectBuilder() - .setType(CompactionType.SPLIT) - .addStreamDataBlock(new StreamDataBlock(STREAM_1, 25, 30, 1, OBJECT_0, -1, -1, 1)) - .addStreamDataBlock(new StreamDataBlock(STREAM_1, 30, 60, 2, OBJECT_0, -1, -1, 1)) - .addStreamDataBlock(new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1)), - new CompactedObjectBuilder() - .setType(CompactionType.COMPACT) - .addStreamDataBlock(new StreamDataBlock(STREAM_2, 30, 60, 3, OBJECT_0, -1, -1, 1))); + new CompactedObjectBuilder() + .setType(CompactionType.SPLIT) + .addStreamDataBlock(new StreamDataBlock(STREAM_0, 0, 15, 0, OBJECT_0, -1, -1, 1)) + .addStreamDataBlock(new StreamDataBlock(STREAM_0, 15, 20, 0, OBJECT_1, -1, -1, 1)), + new CompactedObjectBuilder() + .setType(CompactionType.SPLIT) + .addStreamDataBlock(new StreamDataBlock(STREAM_1, 25, 30, 1, OBJECT_0, -1, -1, 1)) + .addStreamDataBlock(new StreamDataBlock(STREAM_1, 30, 60, 2, OBJECT_0, -1, -1, 1)) + .addStreamDataBlock(new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1)), + new CompactedObjectBuilder() + .setType(CompactionType.COMPACT) + .addStreamDataBlock(new StreamDataBlock(STREAM_2, 30, 60, 3, OBJECT_0, -1, -1, 1))); for (int i = 0; i < compactedObjectBuilders.size(); i++) { assertTrue(compare(compactedObjectBuilders.get(i), expectedCompactedObject.get(i))); } @@ -250,17 +264,17 @@ public void testGroupObjectWithLimit4() { List compactedObjectBuilders = compactionAnalyzer.groupObjectWithLimits(generateStreamDataBlocks(), objectsToRemove); Assertions.assertEquals(Set.of(OBJECT_2), objectsToRemove); List expectedCompactedObject = List.of( - new CompactedObjectBuilder() - .setType(CompactionType.COMPACT) - .addStreamDataBlock(new StreamDataBlock(STREAM_0, 0, 20, 0, OBJECT_0, -1, -1, 1)) - .addStreamDataBlock(new StreamDataBlock(STREAM_0, 20, 25, 0, OBJECT_1, -1, -1, 1)), - new CompactedObjectBuilder() - .setType(CompactionType.SPLIT) - .addStreamDataBlock(new StreamDataBlock(STREAM_1, 30, 60, 1, OBJECT_0, -1, -1, 1)) - .addStreamDataBlock(new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1)), - new CompactedObjectBuilder() - .setType(CompactionType.COMPACT) - .addStreamDataBlock(new StreamDataBlock(STREAM_2, 30, 60, 3, OBJECT_0, -1, -1, 1))); + new CompactedObjectBuilder() + .setType(CompactionType.COMPACT) + .addStreamDataBlock(new StreamDataBlock(STREAM_0, 0, 20, 0, OBJECT_0, -1, -1, 1)) + .addStreamDataBlock(new StreamDataBlock(STREAM_0, 20, 25, 0, OBJECT_1, -1, -1, 1)), + new CompactedObjectBuilder() + .setType(CompactionType.SPLIT) + .addStreamDataBlock(new StreamDataBlock(STREAM_1, 30, 60, 1, OBJECT_0, -1, -1, 1)) + .addStreamDataBlock(new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1)), + new CompactedObjectBuilder() + .setType(CompactionType.COMPACT) + .addStreamDataBlock(new StreamDataBlock(STREAM_2, 30, 60, 3, OBJECT_0, -1, -1, 1))); for (int i = 0; i < compactedObjectBuilders.size(); i++) { assertTrue(compare(compactedObjectBuilders.get(i), expectedCompactedObject.get(i))); } @@ -282,17 +296,17 @@ public void testGroupObjectWithLimit6() { List compactedObjectBuilders = compactionAnalyzer.groupObjectWithLimits(generateStreamDataBlocks(), objectsToRemove); Assertions.assertEquals(Set.of(OBJECT_2), objectsToRemove); List expectedCompactedObject = List.of( - new CompactedObjectBuilder() - .setType(CompactionType.COMPACT) - .addStreamDataBlock(new StreamDataBlock(STREAM_0, 0, 20, 0, OBJECT_0, -1, -1, 1)) - .addStreamDataBlock(new StreamDataBlock(STREAM_0, 20, 25, 0, OBJECT_1, -1, -1, 1)), - new CompactedObjectBuilder() - .setType(CompactionType.SPLIT) - .addStreamDataBlock(new StreamDataBlock(STREAM_1, 30, 60, 1, OBJECT_0, -1, -1, 1)) - .addStreamDataBlock(new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1)), - new CompactedObjectBuilder() - .setType(CompactionType.SPLIT) - .addStreamDataBlock(new StreamDataBlock(STREAM_2, 30, 60, 3, OBJECT_0, -1, -1, 1))); + new CompactedObjectBuilder() + .setType(CompactionType.COMPACT) + .addStreamDataBlock(new StreamDataBlock(STREAM_0, 0, 20, 0, OBJECT_0, -1, -1, 1)) + .addStreamDataBlock(new StreamDataBlock(STREAM_0, 20, 25, 0, OBJECT_1, -1, -1, 1)), + new CompactedObjectBuilder() + .setType(CompactionType.SPLIT) + .addStreamDataBlock(new StreamDataBlock(STREAM_1, 30, 60, 1, OBJECT_0, -1, -1, 1)) + .addStreamDataBlock(new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1)), + new CompactedObjectBuilder() + .setType(CompactionType.SPLIT) + .addStreamDataBlock(new StreamDataBlock(STREAM_2, 30, 60, 3, OBJECT_0, -1, -1, 1))); for (int i = 0; i < compactedObjectBuilders.size(); i++) { assertTrue(compare(compactedObjectBuilders.get(i), expectedCompactedObject.get(i))); } @@ -307,21 +321,6 @@ public void testGroupObjectWithLimit7() { Assertions.assertTrue(compactedObjectBuilders.isEmpty()); } - private static Map> generateStreamDataBlocks() { - return Map.of( - OBJECT_0, List.of( - new StreamDataBlock(STREAM_0, 0, 20, 0, OBJECT_0, -1, 20, 1), - new StreamDataBlock(STREAM_1, 30, 60, 1, OBJECT_0, -1, 30, 1), - new StreamDataBlock(STREAM_2, 30, 60, 2, OBJECT_0, -1, 30, 1)), - OBJECT_1, List.of( - new StreamDataBlock(STREAM_0, 20, 25, 0, OBJECT_1, -1, 5, 1), - new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, 60, 1)), - OBJECT_2, List.of( - new StreamDataBlock(STREAM_1, 400, 500, 0, OBJECT_2, -1, 100, 1), - new StreamDataBlock(STREAM_2, 230, 270, 1, OBJECT_2, -1, 40, 1)) - ); - } - @Test public void testCompactionPlans1() { CompactionAnalyzer compactionAnalyzer = new CompactionAnalyzer(CACHE_SIZE, 100, MAX_STREAM_NUM_IN_WAL, MAX_STREAM_OBJECT_NUM); @@ -330,38 +329,38 @@ public void testCompactionPlans1() { List compactionPlans = compactionAnalyzer.analyze(streamDataBlocksMap, new HashSet<>()); Assertions.assertEquals(1, compactionPlans.size()); List expectCompactedObjects = List.of( - new CompactedObjectBuilder() - .setType(CompactionType.SPLIT) - .addStreamDataBlock(new StreamDataBlock(STREAM_1, 25, 30, 1, OBJECT_0, -1, -1, 1)) - .addStreamDataBlock(new StreamDataBlock(STREAM_1, 30, 60, 2, OBJECT_0, -1, -1, 1)) - .addStreamDataBlock(new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1)) - .build(), - new CompactedObjectBuilder() - .setType(CompactionType.SPLIT) - .addStreamDataBlock(new StreamDataBlock(STREAM_1, 400, 500, 0, OBJECT_2, -1, -1, 1)) - .build(), - new CompactedObjectBuilder() - .setType(CompactionType.SPLIT) - .addStreamDataBlock(new StreamDataBlock(STREAM_2, 30, 60, 3, OBJECT_0, -1, -1, 1)) - .build(), - new CompactedObjectBuilder() - .setType(CompactionType.COMPACT) - .addStreamDataBlock(new StreamDataBlock(STREAM_0, 0, 15, 0, OBJECT_0, -1, -1, 1)) - .addStreamDataBlock(new StreamDataBlock(STREAM_0, 15, 20, 0, OBJECT_1, -1, -1, 1)) - .addStreamDataBlock(new StreamDataBlock(STREAM_2, 230, 270, 1, OBJECT_2, -1, -1, 1)) - .build()); + new CompactedObjectBuilder() + .setType(CompactionType.SPLIT) + .addStreamDataBlock(new StreamDataBlock(STREAM_1, 25, 30, 1, OBJECT_0, -1, -1, 1)) + .addStreamDataBlock(new StreamDataBlock(STREAM_1, 30, 60, 2, OBJECT_0, -1, -1, 1)) + .addStreamDataBlock(new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1)) + .build(), + new CompactedObjectBuilder() + .setType(CompactionType.SPLIT) + .addStreamDataBlock(new StreamDataBlock(STREAM_1, 400, 500, 0, OBJECT_2, -1, -1, 1)) + .build(), + new CompactedObjectBuilder() + .setType(CompactionType.SPLIT) + .addStreamDataBlock(new StreamDataBlock(STREAM_2, 30, 60, 3, OBJECT_0, -1, -1, 1)) + .build(), + new CompactedObjectBuilder() + .setType(CompactionType.COMPACT) + .addStreamDataBlock(new StreamDataBlock(STREAM_0, 0, 15, 0, OBJECT_0, -1, -1, 1)) + .addStreamDataBlock(new StreamDataBlock(STREAM_0, 15, 20, 0, OBJECT_1, -1, -1, 1)) + .addStreamDataBlock(new StreamDataBlock(STREAM_2, 230, 270, 1, OBJECT_2, -1, -1, 1)) + .build()); Map> expectObjectStreamDataBlocks = Map.of( - OBJECT_0, List.of( - new StreamDataBlock(STREAM_0, 0, 15, 0, OBJECT_0, -1, -1, 1), - new StreamDataBlock(STREAM_1, 25, 30, 1, OBJECT_0, -1, -1, 1), - new StreamDataBlock(STREAM_1, 30, 60, 2, OBJECT_0, -1, -1, 1), - new StreamDataBlock(STREAM_2, 30, 60, 3, OBJECT_0, -1, -1, 1)), - OBJECT_1, List.of( - new StreamDataBlock(STREAM_0, 15, 20, 0, OBJECT_1, -1, -1, 1), - new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1)), - OBJECT_2, List.of( - new StreamDataBlock(STREAM_1, 400, 500, 0, OBJECT_2, -1, -1, 1), - new StreamDataBlock(STREAM_2, 230, 270, 1, OBJECT_2, -1, -1, 1))); + OBJECT_0, List.of( + new StreamDataBlock(STREAM_0, 0, 15, 0, OBJECT_0, -1, -1, 1), + new StreamDataBlock(STREAM_1, 25, 30, 1, OBJECT_0, -1, -1, 1), + new StreamDataBlock(STREAM_1, 30, 60, 2, OBJECT_0, -1, -1, 1), + new StreamDataBlock(STREAM_2, 30, 60, 3, OBJECT_0, -1, -1, 1)), + OBJECT_1, List.of( + new StreamDataBlock(STREAM_0, 15, 20, 0, OBJECT_1, -1, -1, 1), + new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1)), + OBJECT_2, List.of( + new StreamDataBlock(STREAM_1, 400, 500, 0, OBJECT_2, -1, -1, 1), + new StreamDataBlock(STREAM_2, 230, 270, 1, OBJECT_2, -1, -1, 1))); CompactionPlan compactionPlan = compactionPlans.get(0); for (int i = 0; i < compactionPlan.compactedObjects().size(); i++) { assertTrue(compare(compactionPlan.compactedObjects().get(i), expectCompactedObjects.get(i))); @@ -376,25 +375,25 @@ private void checkCompactionPlan2(List compactionPlans) { // first iteration List expectCompactedObjects = List.of( - new CompactedObjectBuilder() - .setType(CompactionType.SPLIT) - .addStreamDataBlock(new StreamDataBlock(STREAM_1, 25, 30, 1, OBJECT_0, -1, -1, 1)) - .addStreamDataBlock(new StreamDataBlock(STREAM_1, 30, 60, 2, OBJECT_0, -1, -1, 1)) - .addStreamDataBlock(new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1)) - .build(), - new CompactedObjectBuilder() - .setType(CompactionType.COMPACT) - .addStreamDataBlock(new StreamDataBlock(STREAM_0, 0, 15, 0, OBJECT_0, -1, -1, 1)) - .addStreamDataBlock(new StreamDataBlock(STREAM_0, 15, 20, 0, OBJECT_1, -1, -1, 1)) - .build()); + new CompactedObjectBuilder() + .setType(CompactionType.SPLIT) + .addStreamDataBlock(new StreamDataBlock(STREAM_1, 25, 30, 1, OBJECT_0, -1, -1, 1)) + .addStreamDataBlock(new StreamDataBlock(STREAM_1, 30, 60, 2, OBJECT_0, -1, -1, 1)) + .addStreamDataBlock(new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1)) + .build(), + new CompactedObjectBuilder() + .setType(CompactionType.COMPACT) + .addStreamDataBlock(new StreamDataBlock(STREAM_0, 0, 15, 0, OBJECT_0, -1, -1, 1)) + .addStreamDataBlock(new StreamDataBlock(STREAM_0, 15, 20, 0, OBJECT_1, -1, -1, 1)) + .build()); Map> expectObjectStreamDataBlocks = Map.of( - OBJECT_0, List.of( - new StreamDataBlock(STREAM_0, 0, 15, 0, OBJECT_0, -1, -1, 1), - new StreamDataBlock(STREAM_1, 25, 30, 1, OBJECT_0, -1, -1, 1), - new StreamDataBlock(STREAM_1, 30, 60, 2, OBJECT_0, -1, -1, 1)), - OBJECT_1, List.of( - new StreamDataBlock(STREAM_0, 15, 20, 0, OBJECT_1, -1, -1, 1), - new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1))); + OBJECT_0, List.of( + new StreamDataBlock(STREAM_0, 0, 15, 0, OBJECT_0, -1, -1, 1), + new StreamDataBlock(STREAM_1, 25, 30, 1, OBJECT_0, -1, -1, 1), + new StreamDataBlock(STREAM_1, 30, 60, 2, OBJECT_0, -1, -1, 1)), + OBJECT_1, List.of( + new StreamDataBlock(STREAM_0, 15, 20, 0, OBJECT_1, -1, -1, 1), + new StreamDataBlock(STREAM_1, 60, 120, 1, OBJECT_1, -1, -1, 1))); CompactionPlan compactionPlan = compactionPlans.get(0); for (int i = 0; i < compactionPlan.compactedObjects().size(); i++) { assertTrue(compare(compactionPlan.compactedObjects().get(i), expectCompactedObjects.get(i))); @@ -405,24 +404,24 @@ private void checkCompactionPlan2(List compactionPlans) { // second iteration expectCompactedObjects = List.of( - new CompactedObjectBuilder() - .setType(CompactionType.SPLIT) - .addStreamDataBlock(new StreamDataBlock(STREAM_1, 400, 500, 0, OBJECT_2, -1, -1, 1)) - .build(), - new CompactedObjectBuilder() - .setType(CompactionType.SPLIT) - .addStreamDataBlock(new StreamDataBlock(STREAM_2, 30, 60, 3, OBJECT_0, -1, -1, 1)) - .build(), - new CompactedObjectBuilder() - .setType(CompactionType.COMPACT) - .addStreamDataBlock(new StreamDataBlock(STREAM_2, 230, 270, 1, OBJECT_2, -1, -1, 1)) - .build()); + new CompactedObjectBuilder() + .setType(CompactionType.SPLIT) + .addStreamDataBlock(new StreamDataBlock(STREAM_1, 400, 500, 0, OBJECT_2, -1, -1, 1)) + .build(), + new CompactedObjectBuilder() + .setType(CompactionType.SPLIT) + .addStreamDataBlock(new StreamDataBlock(STREAM_2, 30, 60, 3, OBJECT_0, -1, -1, 1)) + .build(), + new CompactedObjectBuilder() + .setType(CompactionType.COMPACT) + .addStreamDataBlock(new StreamDataBlock(STREAM_2, 230, 270, 1, OBJECT_2, -1, -1, 1)) + .build()); expectObjectStreamDataBlocks = Map.of( - OBJECT_0, List.of( - new StreamDataBlock(STREAM_2, 30, 60, 3, OBJECT_0, -1, -1, 1)), - OBJECT_2, List.of( - new StreamDataBlock(STREAM_1, 400, 500, 0, OBJECT_2, -1, -1, 1), - new StreamDataBlock(STREAM_2, 230, 270, 1, OBJECT_2, -1, -1, 1))); + OBJECT_0, List.of( + new StreamDataBlock(STREAM_2, 30, 60, 3, OBJECT_0, -1, -1, 1)), + OBJECT_2, List.of( + new StreamDataBlock(STREAM_1, 400, 500, 0, OBJECT_2, -1, -1, 1), + new StreamDataBlock(STREAM_2, 230, 270, 1, OBJECT_2, -1, -1, 1))); compactionPlan = compactionPlans.get(1); for (int i = 0; i < compactionPlan.compactedObjects().size(); i++) { assertTrue(compare(compactionPlan.compactedObjects().get(i), expectCompactedObjects.get(i))); @@ -446,9 +445,9 @@ public void testCompactionPlansWithInvalidObject() { CompactionAnalyzer compactionAnalyzer = new CompactionAnalyzer(300, 100, MAX_STREAM_NUM_IN_WAL, MAX_STREAM_OBJECT_NUM); List s3ObjectMetadata = new ArrayList<>(S3_WAL_OBJECT_METADATA_LIST); s3ObjectMetadata.add( - new S3ObjectMetadata(100, S3ObjectType.STREAM_SET, - List.of(new StreamOffsetRange(STREAM_2, 1000, 1200)), System.currentTimeMillis(), - System.currentTimeMillis(), 512, 100)); + new S3ObjectMetadata(100, S3ObjectType.STREAM_SET, + List.of(new StreamOffsetRange(STREAM_2, 1000, 1200)), System.currentTimeMillis(), + System.currentTimeMillis(), 512, 100)); List streamMetadataList = this.streamManager.getStreams(Collections.emptyList()).join(); Map> streamDataBlocksMap = CompactionUtils.blockWaitObjectIndices(streamMetadataList, s3ObjectMetadata, s3Operator); List compactionPlans = compactionAnalyzer.analyze(streamDataBlocksMap, new HashSet<>()); diff --git a/s3stream/src/test/java/com/automq/stream/s3/compact/CompactionManagerTest.java b/s3stream/src/test/java/com/automq/stream/s3/compact/CompactionManagerTest.java index 15580a3ea..46e2137cb 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/compact/CompactionManagerTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/compact/CompactionManagerTest.java @@ -23,28 +23,17 @@ import com.automq.stream.s3.StreamDataBlock; import com.automq.stream.s3.TestUtils; import com.automq.stream.s3.compact.operator.DataBlockReader; +import com.automq.stream.s3.metadata.S3ObjectMetadata; +import com.automq.stream.s3.metadata.S3ObjectType; +import com.automq.stream.s3.metadata.S3StreamConstant; import com.automq.stream.s3.metadata.StreamMetadata; +import com.automq.stream.s3.metadata.StreamOffsetRange; import com.automq.stream.s3.metadata.StreamState; import com.automq.stream.s3.model.StreamRecordBatch; import com.automq.stream.s3.objects.CommitStreamSetObjectRequest; import com.automq.stream.s3.objects.ObjectStreamRange; import com.automq.stream.s3.objects.StreamObject; -import com.automq.stream.s3.metadata.S3ObjectMetadata; -import com.automq.stream.s3.metadata.S3ObjectType; -import com.automq.stream.s3.metadata.S3StreamConstant; -import com.automq.stream.s3.metadata.StreamOffsetRange; import com.automq.stream.s3.operator.DefaultS3Operator; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Tag; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.Timeout; -import org.mockito.Mockito; -import software.amazon.awssdk.core.async.AsyncRequestBody; -import software.amazon.awssdk.services.s3.S3AsyncClient; -import software.amazon.awssdk.services.s3.model.PutObjectRequest; - import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -57,6 +46,16 @@ import java.util.concurrent.CompletionException; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.mockito.Mockito; +import software.amazon.awssdk.core.async.AsyncRequestBody; +import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNull; @@ -76,6 +75,31 @@ public class CompactionManagerTest extends CompactionTestBase { private CompactionManager compactionManager; private Config config; + private static Map> getStreamDataBlockMap() { + StreamDataBlock block1 = new StreamDataBlock(STREAM_0, 0, 15, OBJECT_0, new ObjectReader.DataBlockIndex(0, 0, 15, 15)); + StreamDataBlock block2 = new StreamDataBlock(STREAM_1, 0, 20, OBJECT_0, new ObjectReader.DataBlockIndex(1, 15, 50, 20)); + + StreamDataBlock block3 = new StreamDataBlock(STREAM_0, 15, 27, OBJECT_1, new ObjectReader.DataBlockIndex(0, 0, 20, 12)); + StreamDataBlock block4 = new StreamDataBlock(STREAM_1, 20, 45, OBJECT_1, new ObjectReader.DataBlockIndex(1, 20, 60, 25)); + + StreamDataBlock block5 = new StreamDataBlock(STREAM_0, 27, 40, OBJECT_2, new ObjectReader.DataBlockIndex(0, 0, 20, 20)); + StreamDataBlock block6 = new StreamDataBlock(STREAM_3, 0, 30, OBJECT_2, new ObjectReader.DataBlockIndex(1, 20, 30, 30)); + return Map.of( + OBJECT_0, List.of( + block1, + block2 + ), + OBJECT_1, List.of( + block3, + block4 + ), + OBJECT_2, List.of( + block5, + block6 + ) + ); + } + @BeforeEach public void setUp() throws Exception { super.setUp(); @@ -91,7 +115,7 @@ public void setUp() throws Exception { when(config.maxStreamObjectNumPerCommit()).thenReturn(100); // when(config.networkInboundBaselineBandwidth()).thenReturn(1000L); compactionAnalyzer = new CompactionAnalyzer(config.streamSetObjectCompactionCacheSize(), config.streamSetObjectCompactionStreamSplitSize(), - config.maxStreamNumPerStreamSetObject(), config.maxStreamObjectNumPerCommit()); + config.maxStreamNumPerStreamSetObject(), config.maxStreamObjectNumPerCommit()); } @AfterEach @@ -189,9 +213,9 @@ public void testCompact() { @Test public void testCompactWithDataTrimmed() { when(streamManager.getStreams(Collections.emptyList())).thenReturn(CompletableFuture.completedFuture( - List.of(new StreamMetadata(STREAM_0, 0, 5, 20, StreamState.OPENED), - new StreamMetadata(STREAM_1, 0, 25, 500, StreamState.OPENED), - new StreamMetadata(STREAM_2, 0, 30, 270, StreamState.OPENED)))); + List.of(new StreamMetadata(STREAM_0, 0, 5, 20, StreamState.OPENED), + new StreamMetadata(STREAM_1, 0, 25, 500, StreamState.OPENED), + new StreamMetadata(STREAM_2, 0, 30, 270, StreamState.OPENED)))); compactionManager = new CompactionManager(config, objectManager, streamManager, s3Operator); List streamMetadataList = this.streamManager.getStreams(Collections.emptyList()).join(); CommitStreamSetObjectRequest request = compactionManager.buildCompactRequest(streamMetadataList, S3_WAL_OBJECT_METADATA_LIST); @@ -209,9 +233,9 @@ public void testCompactWithDataTrimmed() { @Test public void testCompactWithDataTrimmed2() { when(streamManager.getStreams(Collections.emptyList())).thenReturn(CompletableFuture.completedFuture( - List.of(new StreamMetadata(STREAM_0, 0, 15, 20, StreamState.OPENED), - new StreamMetadata(STREAM_1, 0, 25, 500, StreamState.OPENED), - new StreamMetadata(STREAM_2, 0, 30, 270, StreamState.OPENED)))); + List.of(new StreamMetadata(STREAM_0, 0, 15, 20, StreamState.OPENED), + new StreamMetadata(STREAM_1, 0, 25, 500, StreamState.OPENED), + new StreamMetadata(STREAM_2, 0, 30, 270, StreamState.OPENED)))); compactionManager = new CompactionManager(config, objectManager, streamManager, s3Operator); List streamMetadataList = this.streamManager.getStreams(Collections.emptyList()).join(); CommitStreamSetObjectRequest request = compactionManager.buildCompactRequest(streamMetadataList, S3_WAL_OBJECT_METADATA_LIST); @@ -238,19 +262,19 @@ public void testCompactionWithDataTrimmed3() { objectWriter.write(STREAM_3, List.of(r2, r3)); objectWriter.close().join(); List streamsIndices = List.of( - new StreamOffsetRange(STREAM_1, 500, 520), - new StreamOffsetRange(STREAM_3, 0, 20) + new StreamOffsetRange(STREAM_1, 500, 520), + new StreamOffsetRange(STREAM_3, 0, 20) ); S3ObjectMetadata objectMetadata = new S3ObjectMetadata(OBJECT_3, S3ObjectType.STREAM_SET, streamsIndices, System.currentTimeMillis(), - System.currentTimeMillis(), objectWriter.size(), OBJECT_3); + System.currentTimeMillis(), objectWriter.size(), OBJECT_3); S3_WAL_OBJECT_METADATA_LIST.add(objectMetadata); List.of(r1, r2, r3).forEach(StreamRecordBatch::release); }).join(); when(streamManager.getStreams(Collections.emptyList())).thenReturn(CompletableFuture.completedFuture( - List.of(new StreamMetadata(STREAM_0, 0, 0, 20, StreamState.OPENED), - new StreamMetadata(STREAM_1, 0, 25, 500, StreamState.OPENED), - new StreamMetadata(STREAM_2, 0, 30, 270, StreamState.OPENED), - new StreamMetadata(STREAM_3, 0, 10, 20, StreamState.OPENED)))); + List.of(new StreamMetadata(STREAM_0, 0, 0, 20, StreamState.OPENED), + new StreamMetadata(STREAM_1, 0, 25, 500, StreamState.OPENED), + new StreamMetadata(STREAM_2, 0, 30, 270, StreamState.OPENED), + new StreamMetadata(STREAM_3, 0, 10, 20, StreamState.OPENED)))); compactionManager = new CompactionManager(config, objectManager, streamManager, s3Operator); List streamMetadataList = this.streamManager.getStreams(Collections.emptyList()).join(); CommitStreamSetObjectRequest request = compactionManager.buildCompactRequest(streamMetadataList, S3_WAL_OBJECT_METADATA_LIST); @@ -269,19 +293,19 @@ public void testCompactionWithDataTrimmed4() { objectWriter.write(STREAM_3, List.of(r2, r3)); objectWriter.close().join(); List streamsIndices = List.of( - new StreamOffsetRange(STREAM_1, 500, 520), - new StreamOffsetRange(STREAM_3, 0, 20) + new StreamOffsetRange(STREAM_1, 500, 520), + new StreamOffsetRange(STREAM_3, 0, 20) ); S3ObjectMetadata objectMetadata = new S3ObjectMetadata(OBJECT_3, S3ObjectType.STREAM_SET, streamsIndices, System.currentTimeMillis(), - System.currentTimeMillis(), objectWriter.size(), OBJECT_3); + System.currentTimeMillis(), objectWriter.size(), OBJECT_3); S3_WAL_OBJECT_METADATA_LIST.add(objectMetadata); List.of(r1, r2, r3).forEach(StreamRecordBatch::release); }).join(); when(streamManager.getStreams(Collections.emptyList())).thenReturn(CompletableFuture.completedFuture( - List.of(new StreamMetadata(STREAM_0, 0, 0, 20, StreamState.OPENED), - new StreamMetadata(STREAM_1, 0, 25, 500, StreamState.OPENED), - new StreamMetadata(STREAM_2, 0, 30, 270, StreamState.OPENED), - new StreamMetadata(STREAM_3, 0, 10, 20, StreamState.OPENED)))); + List.of(new StreamMetadata(STREAM_0, 0, 0, 20, StreamState.OPENED), + new StreamMetadata(STREAM_1, 0, 25, 500, StreamState.OPENED), + new StreamMetadata(STREAM_2, 0, 30, 270, StreamState.OPENED), + new StreamMetadata(STREAM_3, 0, 10, 20, StreamState.OPENED)))); compactionManager = new CompactionManager(config, objectManager, streamManager, s3Operator); List streamMetadataList = this.streamManager.getStreams(Collections.emptyList()).join(); CommitStreamSetObjectRequest request = compactionManager.buildCompactRequest(streamMetadataList, S3_WAL_OBJECT_METADATA_LIST); @@ -299,8 +323,8 @@ public void testCompactionWithDataTrimmed4() { @Test public void testCompactWithNonExistStream() { when(streamManager.getStreams(Collections.emptyList())).thenReturn(CompletableFuture.completedFuture( - List.of(new StreamMetadata(STREAM_1, 0, 25, 500, StreamState.OPENED), - new StreamMetadata(STREAM_2, 0, 30, 270, StreamState.OPENED)))); + List.of(new StreamMetadata(STREAM_1, 0, 25, 500, StreamState.OPENED), + new StreamMetadata(STREAM_2, 0, 30, 270, StreamState.OPENED)))); compactionManager = new CompactionManager(config, objectManager, streamManager, s3Operator); List streamMetadataList = this.streamManager.getStreams(Collections.emptyList()).join(); CommitStreamSetObjectRequest request = compactionManager.buildCompactRequest(streamMetadataList, S3_WAL_OBJECT_METADATA_LIST); @@ -328,7 +352,7 @@ public void testCompactNoneExistObjects() { S3ObjectMetadata objectMetadata2 = new S3ObjectMetadata(OBJECT_2, 0, S3ObjectType.STREAM_SET); List s3ObjectMetadata = List.of(objectMetadata0, objectMetadata1, objectMetadata2); this.compactionAnalyzer = new CompactionAnalyzer(config.streamSetObjectCompactionCacheSize(), config.streamSetObjectCompactionStreamSplitSize(), - config.maxStreamNumPerStreamSetObject(), config.maxStreamObjectNumPerCommit()); + config.maxStreamNumPerStreamSetObject(), config.maxStreamObjectNumPerCommit()); List compactionPlans = this.compactionAnalyzer.analyze(streamDataBlockMap, new HashSet<>()); CommitStreamSetObjectRequest request = new CommitStreamSetObjectRequest(); @@ -342,7 +366,7 @@ public void testCompactNoneExistObjects() { CompactionManager compactionManager = new CompactionManager(config, objectManager, streamManager, s3Operator); Assertions.assertThrowsExactly(CompletionException.class, - () -> compactionManager.executeCompactionPlans(request, compactionPlans, s3ObjectMetadata)); + () -> compactionManager.executeCompactionPlans(request, compactionPlans, s3ObjectMetadata)); for (CompactionPlan plan : compactionPlans) { plan.streamDataBlocksMap().forEach((streamId, blocks) -> blocks.forEach(block -> { if (block.getObjectId() != OBJECT_2) { @@ -364,7 +388,7 @@ public void testCompactNoneExistObjects2() { S3ObjectMetadata objectMetadata2 = new S3ObjectMetadata(OBJECT_2, 0, S3ObjectType.STREAM_SET); List s3ObjectMetadata = List.of(objectMetadata0, objectMetadata1, objectMetadata2); this.compactionAnalyzer = new CompactionAnalyzer(config.streamSetObjectCompactionCacheSize(), config.streamSetObjectCompactionStreamSplitSize(), - config.maxStreamNumPerStreamSetObject(), config.maxStreamObjectNumPerCommit()); + config.maxStreamNumPerStreamSetObject(), config.maxStreamObjectNumPerCommit()); List compactionPlans = this.compactionAnalyzer.analyze(streamDataBlockMap, new HashSet<>()); CommitStreamSetObjectRequest request = new CommitStreamSetObjectRequest(); @@ -378,7 +402,7 @@ public void testCompactNoneExistObjects2() { CompactionManager compactionManager = new CompactionManager(config, objectManager, streamManager, s3Operator); Assertions.assertThrowsExactly(CompletionException.class, - () -> compactionManager.executeCompactionPlans(request, compactionPlans, s3ObjectMetadata)); + () -> compactionManager.executeCompactionPlans(request, compactionPlans, s3ObjectMetadata)); for (CompactionPlan plan : compactionPlans) { plan.streamDataBlocksMap().forEach((streamId, blocks) -> blocks.forEach(block -> { if (block.getObjectId() != OBJECT_1) { @@ -390,31 +414,6 @@ public void testCompactNoneExistObjects2() { } } - private static Map> getStreamDataBlockMap() { - StreamDataBlock block1 = new StreamDataBlock(STREAM_0, 0, 15, OBJECT_0, new ObjectReader.DataBlockIndex(0, 0, 15, 15)); - StreamDataBlock block2 = new StreamDataBlock(STREAM_1, 0, 20, OBJECT_0, new ObjectReader.DataBlockIndex(1, 15, 50, 20)); - - StreamDataBlock block3 = new StreamDataBlock(STREAM_0, 15, 27, OBJECT_1, new ObjectReader.DataBlockIndex(0, 0, 20, 12)); - StreamDataBlock block4 = new StreamDataBlock(STREAM_1, 20, 45, OBJECT_1, new ObjectReader.DataBlockIndex(1, 20, 60, 25)); - - StreamDataBlock block5 = new StreamDataBlock(STREAM_0, 27, 40, OBJECT_2, new ObjectReader.DataBlockIndex(0, 0, 20, 20)); - StreamDataBlock block6 = new StreamDataBlock(STREAM_3, 0, 30, OBJECT_2, new ObjectReader.DataBlockIndex(1, 20, 30, 30)); - return Map.of( - OBJECT_0, List.of( - block1, - block2 - ), - OBJECT_1, List.of( - block3, - block4 - ), - OBJECT_2, List.of( - block5, - block6 - ) - ); - } - @Test public void testCompactWithLimit() { when(config.streamSetObjectCompactionStreamSplitSize()).thenReturn(70L); @@ -437,32 +436,33 @@ public void testCompactWithLimit() { Assertions.assertTrue(checkDataIntegrity(streamMetadataList, s3ObjectMetadata, request)); } - private boolean checkDataIntegrity(List streamMetadataList, List s3ObjectMetadata, CommitStreamSetObjectRequest request) { + private boolean checkDataIntegrity(List streamMetadataList, List s3ObjectMetadata, + CommitStreamSetObjectRequest request) { Map s3WALObjectMetadataMap = s3ObjectMetadata.stream() - .collect(Collectors.toMap(S3ObjectMetadata::objectId, e -> e)); + .collect(Collectors.toMap(S3ObjectMetadata::objectId, e -> e)); Map> streamDataBlocks = CompactionUtils.blockWaitObjectIndices(streamMetadataList, s3ObjectMetadata, s3Operator); for (Map.Entry> entry : streamDataBlocks.entrySet()) { long objectId = entry.getKey(); DataBlockReader reader = new DataBlockReader(new S3ObjectMetadata(objectId, - s3WALObjectMetadataMap.get(objectId).objectSize(), S3ObjectType.STREAM_SET), s3Operator); + s3WALObjectMetadataMap.get(objectId).objectSize(), S3ObjectType.STREAM_SET), s3Operator); reader.readBlocks(entry.getValue()); } Map compactedObjectMap = new HashMap<>(); for (StreamObject streamObject : request.getStreamObjects()) { S3ObjectMetadata objectMetadata = new S3ObjectMetadata(streamObject.getObjectId(), S3ObjectType.STREAM, - List.of(new StreamOffsetRange(streamObject.getStreamId(), streamObject.getStartOffset(), streamObject.getEndOffset())), - System.currentTimeMillis(), System.currentTimeMillis(), streamObject.getObjectSize(), S3StreamConstant.INVALID_ORDER_ID); + List.of(new StreamOffsetRange(streamObject.getStreamId(), streamObject.getStartOffset(), streamObject.getEndOffset())), + System.currentTimeMillis(), System.currentTimeMillis(), streamObject.getObjectSize(), S3StreamConstant.INVALID_ORDER_ID); compactedObjectMap.put(streamObject.getObjectId(), objectMetadata); } List streamOffsetRanges = new ArrayList<>(); for (ObjectStreamRange objectStreamRange : request.getStreamRanges()) { streamOffsetRanges.add(new StreamOffsetRange(objectStreamRange.getStreamId(), - objectStreamRange.getStartOffset(), objectStreamRange.getEndOffset())); + objectStreamRange.getStartOffset(), objectStreamRange.getEndOffset())); } if (request.getObjectId() != -1) { S3ObjectMetadata metadata = new S3ObjectMetadata(request.getObjectId(), S3ObjectType.STREAM_SET, - streamOffsetRanges, System.currentTimeMillis(), System.currentTimeMillis(), request.getObjectSize(), request.getOrderId()); + streamOffsetRanges, System.currentTimeMillis(), System.currentTimeMillis(), request.getObjectSize(), request.getOrderId()); compactedObjectMap.put(request.getObjectId(), metadata); } @@ -470,7 +470,7 @@ private boolean checkDataIntegrity(List streamMetadataList, List for (Map.Entry> entry : compactedStreamDataBlocksMap.entrySet()) { long objectId = entry.getKey(); DataBlockReader reader = new DataBlockReader(new S3ObjectMetadata(objectId, - compactedObjectMap.get(objectId).objectSize(), S3ObjectType.STREAM_SET), s3Operator); + compactedObjectMap.get(objectId).objectSize(), S3ObjectType.STREAM_SET), s3Operator); reader.readBlocks(entry.getValue()); } List expectedStreamDataBlocks = streamDataBlocks.values().stream().flatMap(Collection::stream).toList(); diff --git a/s3stream/src/test/java/com/automq/stream/s3/compact/CompactionTestBase.java b/s3stream/src/test/java/com/automq/stream/s3/compact/CompactionTestBase.java index c31dce46a..87095f497 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/compact/CompactionTestBase.java +++ b/s3stream/src/test/java/com/automq/stream/s3/compact/CompactionTestBase.java @@ -18,26 +18,25 @@ package com.automq.stream.s3.compact; import com.automq.stream.s3.ObjectWriter; +import com.automq.stream.s3.StreamDataBlock; import com.automq.stream.s3.TestUtils; import com.automq.stream.s3.compact.objects.CompactedObject; import com.automq.stream.s3.compact.objects.CompactedObjectBuilder; -import com.automq.stream.s3.StreamDataBlock; import com.automq.stream.s3.memory.MemoryMetadataManager; +import com.automq.stream.s3.metadata.S3ObjectMetadata; +import com.automq.stream.s3.metadata.S3ObjectType; import com.automq.stream.s3.metadata.StreamMetadata; +import com.automq.stream.s3.metadata.StreamOffsetRange; import com.automq.stream.s3.metadata.StreamState; import com.automq.stream.s3.model.StreamRecordBatch; import com.automq.stream.s3.operator.MemoryS3Operator; import com.automq.stream.s3.operator.S3Operator; -import com.automq.stream.s3.metadata.S3ObjectMetadata; -import com.automq.stream.s3.metadata.S3ObjectType; -import com.automq.stream.s3.metadata.StreamOffsetRange; -import org.mockito.Mockito; - import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; +import org.mockito.Mockito; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -67,9 +66,9 @@ public class CompactionTestBase { public void setUp() throws Exception { streamManager = Mockito.mock(MemoryMetadataManager.class); when(streamManager.getStreams(Mockito.anyList())).thenReturn(CompletableFuture.completedFuture( - List.of(new StreamMetadata(STREAM_0, 0, 0, 20, StreamState.OPENED), - new StreamMetadata(STREAM_1, 0, 25, 500, StreamState.OPENED), - new StreamMetadata(STREAM_2, 0, 30, 270, StreamState.OPENED)))); + List.of(new StreamMetadata(STREAM_0, 0, 0, 20, StreamState.OPENED), + new StreamMetadata(STREAM_1, 0, 25, 500, StreamState.OPENED), + new StreamMetadata(STREAM_2, 0, 30, 270, StreamState.OPENED)))); objectManager = Mockito.spy(MemoryMetadataManager.class); s3Operator = new MemoryS3Operator(); @@ -87,13 +86,13 @@ public void setUp() throws Exception { objectWriter.write(STREAM_2, List.of(r4)); objectWriter.close().join(); List streamsIndices = List.of( - new StreamOffsetRange(STREAM_0, 0, 15), - new StreamOffsetRange(STREAM_1, 25, 30), - new StreamOffsetRange(STREAM_1, 30, 60), - new StreamOffsetRange(STREAM_2, 30, 60) + new StreamOffsetRange(STREAM_0, 0, 15), + new StreamOffsetRange(STREAM_1, 25, 30), + new StreamOffsetRange(STREAM_1, 30, 60), + new StreamOffsetRange(STREAM_2, 30, 60) ); S3ObjectMetadata objectMetadata = new S3ObjectMetadata(OBJECT_0, S3ObjectType.STREAM_SET, streamsIndices, System.currentTimeMillis(), - System.currentTimeMillis(), objectWriter.size(), OBJECT_0); + System.currentTimeMillis(), objectWriter.size(), OBJECT_0); S3_WAL_OBJECT_METADATA_LIST.add(objectMetadata); List.of(r1, r2, r3, r4).forEach(StreamRecordBatch::release); }).join(); @@ -108,11 +107,11 @@ public void setUp() throws Exception { objectWriter.write(STREAM_1, List.of(r6)); objectWriter.close().join(); List streamsIndices = List.of( - new StreamOffsetRange(STREAM_0, 15, 20), - new StreamOffsetRange(STREAM_1, 60, 120) + new StreamOffsetRange(STREAM_0, 15, 20), + new StreamOffsetRange(STREAM_1, 60, 120) ); S3ObjectMetadata objectMetadata = new S3ObjectMetadata(OBJECT_1, S3ObjectType.STREAM_SET, streamsIndices, System.currentTimeMillis(), - System.currentTimeMillis(), objectWriter.size(), OBJECT_1); + System.currentTimeMillis(), objectWriter.size(), OBJECT_1); S3_WAL_OBJECT_METADATA_LIST.add(objectMetadata); List.of(r5, r6).forEach(StreamRecordBatch::release); }).join(); @@ -127,11 +126,11 @@ public void setUp() throws Exception { objectWriter.write(STREAM_2, List.of(r9)); objectWriter.close().join(); List streamsIndices = List.of( - new StreamOffsetRange(STREAM_1, 400, 500), - new StreamOffsetRange(STREAM_2, 230, 270) + new StreamOffsetRange(STREAM_1, 400, 500), + new StreamOffsetRange(STREAM_2, 230, 270) ); S3ObjectMetadata objectMetadata = new S3ObjectMetadata(OBJECT_2, S3ObjectType.STREAM_SET, streamsIndices, System.currentTimeMillis(), - System.currentTimeMillis(), objectWriter.size(), OBJECT_2); + System.currentTimeMillis(), objectWriter.size(), OBJECT_2); S3_WAL_OBJECT_METADATA_LIST.add(objectMetadata); List.of(r8, r9).forEach(StreamRecordBatch::release); }).join(); @@ -144,9 +143,9 @@ public void tearDown() { protected boolean compare(StreamDataBlock block1, StreamDataBlock block2) { boolean attr = block1.getStreamId() == block2.getStreamId() && - block1.getStartOffset() == block2.getStartOffset() && - block1.getEndOffset() == block2.getEndOffset() && - block1.getRecordCount() == block2.getRecordCount(); + block1.getStartOffset() == block2.getStartOffset() && + block1.getEndOffset() == block2.getEndOffset() && + block1.getRecordCount() == block2.getRecordCount(); if (!attr) { return false; } @@ -173,7 +172,8 @@ protected boolean compare(List streamDataBlocks1, List> streamDataBlockMap1, Map> streamDataBlockMap2) { + protected boolean compare(Map> streamDataBlockMap1, + Map> streamDataBlockMap2) { if (streamDataBlockMap1.size() != streamDataBlockMap2.size()) { return false; } diff --git a/s3stream/src/test/java/com/automq/stream/s3/compact/CompactionUploaderTest.java b/s3stream/src/test/java/com/automq/stream/s3/compact/CompactionUploaderTest.java index e7c906626..13accbd53 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/compact/CompactionUploaderTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/compact/CompactionUploaderTest.java @@ -18,25 +18,24 @@ package com.automq.stream.s3.compact; import com.automq.stream.s3.Config; +import com.automq.stream.s3.StreamDataBlock; import com.automq.stream.s3.TestUtils; import com.automq.stream.s3.compact.objects.CompactedObject; import com.automq.stream.s3.compact.objects.CompactionType; -import com.automq.stream.s3.StreamDataBlock; import com.automq.stream.s3.compact.operator.DataBlockReader; import com.automq.stream.s3.memory.MemoryMetadataManager; -import com.automq.stream.s3.objects.StreamObject; -import com.automq.stream.s3.operator.MemoryS3Operator; import com.automq.stream.s3.metadata.S3ObjectMetadata; import com.automq.stream.s3.metadata.S3ObjectType; +import com.automq.stream.s3.objects.StreamObject; +import com.automq.stream.s3.operator.MemoryS3Operator; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.CompletableFuture; - import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -61,10 +60,10 @@ public void setUp() throws Exception { @Test public void testWriteWALObject() { List streamDataBlocks = List.of( - new StreamDataBlock(STREAM_0, 0, 20, 2, 1, 30, 20, 1), - new StreamDataBlock(STREAM_0, 20, 25, 3, 0, 10, 5, 1), - new StreamDataBlock(STREAM_2, 40, 120, 0, 2, 100, 80, 1), - new StreamDataBlock(STREAM_2, 120, 150, 1, 3, 0, 30, 1)); + new StreamDataBlock(STREAM_0, 0, 20, 2, 1, 30, 20, 1), + new StreamDataBlock(STREAM_0, 20, 25, 3, 0, 10, 5, 1), + new StreamDataBlock(STREAM_2, 40, 120, 0, 2, 100, 80, 1), + new StreamDataBlock(STREAM_2, 120, 150, 1, 3, 0, 30, 1)); CompactedObject compactedObject = new CompactedObject(CompactionType.COMPACT, streamDataBlocks); CompactionUploader uploader = new CompactionUploader(objectManager, s3Operator, config); CompletableFuture cf = uploader.chainWriteStreamSetObject(null, compactedObject); @@ -94,15 +93,15 @@ public void testWriteWALObject() { @Test public void testWriteWALObject2() { List streamDataBlocks1 = List.of( - new StreamDataBlock(STREAM_0, 0, 20, 2, 1, 30, 20, 1), - new StreamDataBlock(STREAM_0, 20, 25, 3, 0, 10, 5, 1), - new StreamDataBlock(STREAM_2, 40, 120, 0, 2, 100, 80, 1), - new StreamDataBlock(STREAM_2, 120, 150, 1, 3, 0, 30, 1)); + new StreamDataBlock(STREAM_0, 0, 20, 2, 1, 30, 20, 1), + new StreamDataBlock(STREAM_0, 20, 25, 3, 0, 10, 5, 1), + new StreamDataBlock(STREAM_2, 40, 120, 0, 2, 100, 80, 1), + new StreamDataBlock(STREAM_2, 120, 150, 1, 3, 0, 30, 1)); CompactedObject compactedObject = new CompactedObject(CompactionType.COMPACT, streamDataBlocks1); List streamDataBlocks2 = List.of( - new StreamDataBlock(STREAM_3, 0, 15, 2, 4, 0, 15, 1), - new StreamDataBlock(STREAM_3, 15, 20, 1, 5, 20, 5, 1)); + new StreamDataBlock(STREAM_3, 0, 15, 2, 4, 0, 15, 1), + new StreamDataBlock(STREAM_3, 15, 20, 1, 5, 20, 5, 1)); CompactedObject compactedObject2 = new CompactedObject(CompactionType.COMPACT, streamDataBlocks2); CompactionUploader uploader = new CompactionUploader(objectManager, s3Operator, config); @@ -142,8 +141,8 @@ public void testWriteWALObject2() { @Test public void testWriteStreamObject() { List streamDataBlocks = List.of( - new StreamDataBlock(STREAM_0, 0, 60, 1, 0, 23, 60, 1), - new StreamDataBlock(STREAM_0, 60, 120, 0, 1, 45, 60, 1)); + new StreamDataBlock(STREAM_0, 0, 60, 1, 0, 23, 60, 1), + new StreamDataBlock(STREAM_0, 60, 120, 0, 1, 45, 60, 1)); CompactedObject compactedObject = new CompactedObject(CompactionType.SPLIT, streamDataBlocks); CompactionUploader uploader = new CompactionUploader(objectManager, s3Operator, config); diff --git a/s3stream/src/test/java/com/automq/stream/s3/compact/CompactionUtilTest.java b/s3stream/src/test/java/com/automq/stream/s3/compact/CompactionUtilTest.java index e6372beb3..2c9d93d47 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/compact/CompactionUtilTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/compact/CompactionUtilTest.java @@ -17,17 +17,16 @@ package com.automq.stream.s3.compact; +import com.automq.stream.s3.StreamDataBlock; import com.automq.stream.s3.compact.objects.CompactedObject; import com.automq.stream.s3.compact.objects.CompactionType; -import com.automq.stream.s3.StreamDataBlock; import com.automq.stream.s3.objects.ObjectStreamRange; +import java.util.List; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.List; - import static org.junit.jupiter.api.Assertions.assertEquals; @Timeout(30) @@ -37,10 +36,10 @@ public class CompactionUtilTest extends CompactionTestBase { @Test public void testBuildObjectStreamRanges() { List streamDataBlocks = List.of( - new StreamDataBlock(STREAM_0, 0, 20, 2, 1, 0, 20, 1), - new StreamDataBlock(STREAM_0, 20, 25, 3, 0, 20, 5, 1), - new StreamDataBlock(STREAM_2, 40, 120, 0, 2, 25, 80, 1), - new StreamDataBlock(STREAM_2, 120, 150, 1, 3, 105, 30, 1)); + new StreamDataBlock(STREAM_0, 0, 20, 2, 1, 0, 20, 1), + new StreamDataBlock(STREAM_0, 20, 25, 3, 0, 20, 5, 1), + new StreamDataBlock(STREAM_2, 40, 120, 0, 2, 25, 80, 1), + new StreamDataBlock(STREAM_2, 120, 150, 1, 3, 105, 30, 1)); CompactedObject compactedObject = new CompactedObject(CompactionType.COMPACT, streamDataBlocks); List result = CompactionUtils.buildObjectStreamRange(compactedObject.streamDataBlocks()); assertEquals(2, result.size()); @@ -55,11 +54,11 @@ public void testBuildObjectStreamRanges() { @Test public void testMergeStreamDataBlocks() { List streamDataBlocks = List.of( - new StreamDataBlock(STREAM_0, 0, 15, 0, 1, 0, 20, 1), - new StreamDataBlock(STREAM_0, 15, 30, 1, 1, 20, 5, 1), - new StreamDataBlock(STREAM_0, 30, 100, 2, 1, 25, 80, 1), - new StreamDataBlock(STREAM_2, 40, 100, 3, 1, 105, 80, 1), - new StreamDataBlock(STREAM_2, 120, 150, 4, 1, 185, 30, 1)); + new StreamDataBlock(STREAM_0, 0, 15, 0, 1, 0, 20, 1), + new StreamDataBlock(STREAM_0, 15, 30, 1, 1, 20, 5, 1), + new StreamDataBlock(STREAM_0, 30, 100, 2, 1, 25, 80, 1), + new StreamDataBlock(STREAM_2, 40, 100, 3, 1, 105, 80, 1), + new StreamDataBlock(STREAM_2, 120, 150, 4, 1, 185, 30, 1)); List> result = CompactionUtils.groupStreamDataBlocks(streamDataBlocks); assertEquals(3, result.size()); Assertions.assertEquals(List.of(streamDataBlocks.get(0), streamDataBlocks.get(1), streamDataBlocks.get(2)), result.get(0)); diff --git a/s3stream/src/test/java/com/automq/stream/s3/failover/FailoverTest.java b/s3stream/src/test/java/com/automq/stream/s3/failover/FailoverTest.java index 9ca66d5c1..71a8b2a4a 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/failover/FailoverTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/failover/FailoverTest.java @@ -18,18 +18,17 @@ package com.automq.stream.s3.failover; import com.automq.stream.s3.wal.BlockWALService; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; - import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.Mockito.mock; diff --git a/s3stream/src/test/java/com/automq/stream/s3/operator/DefaultS3OperatorTest.java b/s3stream/src/test/java/com/automq/stream/s3/operator/DefaultS3OperatorTest.java index 598885797..5465184af 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/operator/DefaultS3OperatorTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/operator/DefaultS3OperatorTest.java @@ -19,6 +19,10 @@ import com.automq.stream.s3.TestUtils; import io.netty.buffer.ByteBuf; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; @@ -28,11 +32,6 @@ import software.amazon.awssdk.services.s3.model.DeleteObjectsResponse; import software.amazon.awssdk.services.s3.model.DeletedObject; -import java.util.Collections; -import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; - import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -64,17 +63,17 @@ void tearDown() { @Test void testDeleteObjectsSuccess() { when(s3.deleteObjects(any(DeleteObjectsRequest.class))) - .thenAnswer(invocation -> { - DeleteObjectsRequest request = invocation.getArgument(0); - DeleteObjectsResponse response = DeleteObjectsResponse.builder() - .deleted(request.delete().objects().stream() - .map(o -> DeletedObject.builder() - .key(o.key()) - .build()) - .toList()) - .build(); - return CompletableFuture.completedFuture(response); - }); + .thenAnswer(invocation -> { + DeleteObjectsRequest request = invocation.getArgument(0); + DeleteObjectsResponse response = DeleteObjectsResponse.builder() + .deleted(request.delete().objects().stream() + .map(o -> DeletedObject.builder() + .key(o.key()) + .build()) + .toList()) + .build(); + return CompletableFuture.completedFuture(response); + }); List keys = List.of("test1", "test2"); List deleted = operator.delete(keys).join(); assertEquals(keys, deleted); @@ -83,7 +82,7 @@ void testDeleteObjectsSuccess() { @Test void testDeleteObjectsFail() { when(s3.deleteObjects(any(DeleteObjectsRequest.class))) - .thenReturn(CompletableFuture.failedFuture(new RuntimeException("test"))); + .thenReturn(CompletableFuture.failedFuture(new RuntimeException("test"))); List keys = List.of("test1", "test2"); List deleted = operator.delete(keys).join(); assertEquals(Collections.emptyList(), deleted); @@ -92,7 +91,7 @@ void testDeleteObjectsFail() { @Test public void testMergeTask() { DefaultS3Operator.MergedReadTask mergedReadTask = new DefaultS3Operator.MergedReadTask( - new DefaultS3Operator.ReadTask("obj0", 0, 1024, new CompletableFuture<>()), 0); + new DefaultS3Operator.ReadTask("obj0", 0, 1024, new CompletableFuture<>()), 0); boolean ret = mergedReadTask.tryMerge(new DefaultS3Operator.ReadTask("obj0", 1024, 2048, new CompletableFuture<>())); assertTrue(ret); assertEquals(0, mergedReadTask.dataSparsityRate); @@ -108,7 +107,7 @@ public void testMergeTask() { @Test public void testMergeTask2() { DefaultS3Operator.MergedReadTask mergedReadTask = new DefaultS3Operator.MergedReadTask( - new DefaultS3Operator.ReadTask("obj0", 0, 1024, new CompletableFuture<>()), 0.5f); + new DefaultS3Operator.ReadTask("obj0", 0, 1024, new CompletableFuture<>()), 0.5f); boolean ret = mergedReadTask.tryMerge(new DefaultS3Operator.ReadTask("obj0", 2048, 4096, new CompletableFuture<>())); assertTrue(ret); assertEquals(0.25, mergedReadTask.dataSparsityRate, 0.01); diff --git a/s3stream/src/test/java/com/automq/stream/s3/operator/MultiPartWriterTest.java b/s3stream/src/test/java/com/automq/stream/s3/operator/MultiPartWriterTest.java index 2800221d6..64685a0f0 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/operator/MultiPartWriterTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/operator/MultiPartWriterTest.java @@ -19,6 +19,13 @@ import com.automq.stream.s3.TestUtils; import io.netty.buffer.ByteBuf; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; @@ -37,14 +44,6 @@ import software.amazon.awssdk.services.s3.model.UploadPartRequest; import software.amazon.awssdk.services.s3.model.UploadPartResponse; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.stream.Collectors; - import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; @@ -87,20 +86,20 @@ void testWrite() throws NoSuchMethodException, InvocationTargetException, Illega writer.uploadIdCf.get(); List payloads = List.of( - // case 2 - TestUtils.random(120), - // case 1 - TestUtils.random(20), - // case 3 - TestUtils.random(40), - // case 4 - TestUtils.random(60), - // case 1 - TestUtils.random(80), - // case 5 - TestUtils.random(200), - // last part - TestUtils.random(10) + // case 2 + TestUtils.random(120), + // case 1 + TestUtils.random(20), + // case 3 + TestUtils.random(40), + // case 4 + TestUtils.random(60), + // case 1 + TestUtils.random(80), + // case 5 + TestUtils.random(200), + // last part + TestUtils.random(10) ); payloads.forEach(writer::write); @@ -109,8 +108,8 @@ void testWrite() throws NoSuchMethodException, InvocationTargetException, Illega assertEquals("unit-test-bucket", requests.get(0).bucket()); assertEquals("test-path", requests.get(0).key()); assertEquals(List.of(1, 2, 3, 4), requests.stream() - .map(UploadPartRequest::partNumber) - .collect(Collectors.toList())); + .map(UploadPartRequest::partNumber) + .collect(Collectors.toList())); assertEquals(List.of(120L, 120L, 280L, 10L), contentLengths); } @@ -137,7 +136,6 @@ void testCopyWrite() throws NoSuchMethodException, InvocationTargetException, Il method.setAccessible(true); method.invoke(copyBuilder, copyResultBuilder); - when(s3.uploadPart(any(UploadPartRequest.class), any(AsyncRequestBody.class))).thenAnswer(invocation -> { UploadPartRequest request = invocation.getArgument(0); uploadPartRequests.add(request); @@ -161,7 +159,7 @@ void testCopyWrite() throws NoSuchMethodException, InvocationTargetException, Il GetObjectResponse.Builder responseBuilder = GetObjectResponse.builder(); software.amazon.awssdk.core.async.ResponsePublisher responsePublisher - = new ResponsePublisher<>(responseBuilder.build(), AsyncRequestBody.fromByteBuffer(TestUtils.random((int) (end - start + 1)).nioBuffer())); + = new ResponsePublisher<>(responseBuilder.build(), AsyncRequestBody.fromByteBuffer(TestUtils.random((int) (end - start + 1)).nioBuffer())); return CompletableFuture.completedFuture(responsePublisher); }); @@ -198,15 +196,15 @@ void testCopyWrite() throws NoSuchMethodException, InvocationTargetException, Il assertEquals("unit-test-bucket", uploadPartCopyRequests.get(0).sourceBucket()); assertEquals("unit-test-bucket", uploadPartCopyRequests.get(0).destinationBucket()); assertEquals(List.of("path-1"), uploadPartCopyRequests.stream() - .map(UploadPartCopyRequest::sourceKey) - .collect(Collectors.toList())); + .map(UploadPartCopyRequest::sourceKey) + .collect(Collectors.toList())); assertEquals("test-path-2", uploadPartCopyRequests.get(0).destinationKey()); assertEquals(List.of(1), uploadPartCopyRequests.stream() - .map(UploadPartCopyRequest::partNumber) - .collect(Collectors.toList())); + .map(UploadPartCopyRequest::partNumber) + .collect(Collectors.toList())); assertEquals(List.of("bytes=0-119"), uploadPartCopyRequests.stream() - .map(UploadPartCopyRequest::copySourceRange) - .collect(Collectors.toList())); + .map(UploadPartCopyRequest::copySourceRange) + .collect(Collectors.toList())); } } \ No newline at end of file diff --git a/s3stream/src/test/java/com/automq/stream/s3/operator/ProxyWriterTest.java b/s3stream/src/test/java/com/automq/stream/s3/operator/ProxyWriterTest.java index 793561d0e..94078ca15 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/operator/ProxyWriterTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/operator/ProxyWriterTest.java @@ -20,14 +20,13 @@ import com.automq.stream.s3.TestUtils; import com.automq.stream.s3.network.ThrottleStrategy; import io.netty.buffer.ByteBuf; +import java.util.concurrent.CompletableFuture; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.mockito.ArgumentCaptor; import software.amazon.awssdk.services.s3.model.CompletedPart; -import java.util.concurrent.CompletableFuture; - import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; @@ -52,7 +51,6 @@ public void setup() { writer = new ProxyWriter(operator, "testpath", null); } - @Test public void testWrite_onePart() { writer.write(TestUtils.random(15 * 1024 * 1024)); @@ -88,7 +86,7 @@ public void testWrite_dataLargerThanMaxUploadSize() { public void testWrite_copyWrite() { when(operator.createMultipartUpload(eq("testpath"))).thenReturn(CompletableFuture.completedFuture("test_upload_id")); when(operator.uploadPartCopy(eq("test_src_path"), eq("testpath"), eq(0L), eq(15L * 1024 * 1024), eq("test_upload_id"), eq(1))) - .thenReturn(CompletableFuture.completedFuture(CompletedPart.builder().partNumber(1).eTag("etag1").build())); + .thenReturn(CompletableFuture.completedFuture(CompletedPart.builder().partNumber(1).eTag("etag1").build())); when(operator.completeMultipartUpload(eq("testpath"), eq("test_upload_id"), any())).thenReturn(CompletableFuture.completedFuture(null)); writer.copyWrite("test_src_path", 0, 15 * 1024 * 1024); diff --git a/s3stream/src/test/java/com/automq/stream/s3/utils/AsyncRateLimiterTest.java b/s3stream/src/test/java/com/automq/stream/s3/utils/AsyncRateLimiterTest.java index b6a6194a6..a9c1c13a0 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/utils/AsyncRateLimiterTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/utils/AsyncRateLimiterTest.java @@ -18,12 +18,11 @@ package com.automq.stream.s3.utils; import com.automq.stream.utils.AsyncRateLimiter; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; public class AsyncRateLimiterTest { diff --git a/s3stream/src/test/java/com/automq/stream/s3/wal/BlockWALServiceTest.java b/s3stream/src/test/java/com/automq/stream/s3/wal/BlockWALServiceTest.java index f16265406..82df39972 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/wal/BlockWALServiceTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/wal/BlockWALServiceTest.java @@ -26,17 +26,6 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.CompositeByteBuf; import io.netty.buffer.Unpooled; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Tag; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.condition.EnabledOnOs; -import org.junit.jupiter.api.condition.OS; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.CsvSource; -import org.junit.jupiter.params.provider.MethodSource; -import org.junit.jupiter.params.provider.ValueSource; - import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -50,6 +39,16 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; import java.util.stream.Stream; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.condition.EnabledOnOs; +import org.junit.jupiter.api.condition.OS; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.CsvSource; +import org.junit.jupiter.params.provider.MethodSource; +import org.junit.jupiter.params.provider.ValueSource; import static com.automq.stream.s3.wal.BlockWALService.RECORD_HEADER_SIZE; import static com.automq.stream.s3.wal.BlockWALService.WAL_HEADER_TOTAL_CAPACITY; @@ -67,20 +66,8 @@ class BlockWALServiceTest { static final String TEST_BLOCK_DEVICE = System.getenv("WAL_TEST_BLOCK_DEVICE"); - @ParameterizedTest(name = "Test {index}: mergeWrite={0}") - @ValueSource(booleans = {false, true}) - public void testSingleThreadAppendBasic(boolean mergeWrite) throws IOException, OverCapacityException { - testSingleThreadAppendBasic0(mergeWrite, false); - } - - @ParameterizedTest(name = "Test {index}: mergeWrite={0}") - @ValueSource(booleans = {false, true}) - @EnabledOnOs(OS.LINUX) - public void testSingleThreadAppendBasicDirectIO(boolean mergeWrite) throws IOException, OverCapacityException { - testSingleThreadAppendBasic0(mergeWrite, true); - } - - private static void testSingleThreadAppendBasic0(boolean mergeWrite, boolean directIO) throws IOException, OverCapacityException { + private static void testSingleThreadAppendBasic0(boolean mergeWrite, + boolean directIO) throws IOException, OverCapacityException { final int recordSize = 4096 + 1; final int recordCount = 100; final long blockDeviceCapacity = WALUtil.alignLargeByBlockSize(recordSize) * recordCount + WAL_HEADER_TOTAL_CAPACITY; @@ -92,9 +79,9 @@ private static void testSingleThreadAppendBasic0(boolean mergeWrite, boolean dir } BlockWALService.BlockWALServiceBuilder builder = BlockWALService.builder(path, blockDeviceCapacity) - .direct(directIO) - .slidingWindowInitialSize(0) - .slidingWindowScaleUnit(4096); + .direct(directIO) + .slidingWindowInitialSize(0) + .slidingWindowScaleUnit(4096); if (!mergeWrite) { builder.blockSoftLimit(0); } @@ -133,23 +120,11 @@ private static void testSingleThreadAppendBasic0(boolean mergeWrite, boolean dir wal.shutdownGracefully(); } assertTrue(maxFlushedOffset.get() > maxRecordOffset.get(), - "maxFlushedOffset should be greater than maxRecordOffset. maxFlushedOffset: " + maxFlushedOffset.get() + ", maxRecordOffset: " + maxRecordOffset.get()); - } - - @ParameterizedTest(name = "Test {index}: mergeWrite={0}") - @ValueSource(booleans = {false, true}) - public void testSingleThreadAppendWhenOverCapacity(boolean mergeWrite) throws IOException { - testSingleThreadAppendWhenOverCapacity0(mergeWrite, false); - } - - @ParameterizedTest(name = "Test {index}: mergeWrite={0}") - @ValueSource(booleans = {false, true}) - @EnabledOnOs(OS.LINUX) - public void testSingleThreadAppendWhenOverCapacityDirectIO(boolean mergeWrite) throws IOException { - testSingleThreadAppendWhenOverCapacity0(mergeWrite, true); + "maxFlushedOffset should be greater than maxRecordOffset. maxFlushedOffset: " + maxFlushedOffset.get() + ", maxRecordOffset: " + maxRecordOffset.get()); } - private static void testSingleThreadAppendWhenOverCapacity0(boolean mergeWrite, boolean directIO) throws IOException { + private static void testSingleThreadAppendWhenOverCapacity0(boolean mergeWrite, + boolean directIO) throws IOException { final int recordSize = 4096 + 1; final int recordCount = 100; long blockDeviceCapacity; @@ -167,9 +142,9 @@ private static void testSingleThreadAppendWhenOverCapacity0(boolean mergeWrite, } BlockWALService.BlockWALServiceBuilder builder = BlockWALService.builder(path, blockDeviceCapacity) - .direct(directIO) - .slidingWindowInitialSize(0) - .slidingWindowScaleUnit(4096); + .direct(directIO) + .slidingWindowInitialSize(0) + .slidingWindowScaleUnit(4096); if (!mergeWrite) { builder.blockSoftLimit(0); } @@ -222,23 +197,11 @@ private static void testSingleThreadAppendWhenOverCapacity0(boolean mergeWrite, wal.shutdownGracefully(); } assertTrue(maxFlushedOffset.get() > maxRecordOffset.get(), - "maxFlushedOffset should be greater than maxRecordOffset. maxFlushedOffset: " + maxFlushedOffset.get() + ", maxRecordOffset: " + maxRecordOffset.get()); - } - - @ParameterizedTest(name = "Test {index}: mergeWrite={0}") - @ValueSource(booleans = {false, true}) - public void testMultiThreadAppend(boolean mergeWrite) throws InterruptedException, IOException { - testMultiThreadAppend0(mergeWrite, false); - } - - @ParameterizedTest(name = "Test {index}: mergeWrite={0}") - @ValueSource(booleans = {false, true}) - @EnabledOnOs(OS.LINUX) - public void testMultiThreadAppendDirectIO(boolean mergeWrite) throws InterruptedException, IOException { - testMultiThreadAppend0(mergeWrite, true); + "maxFlushedOffset should be greater than maxRecordOffset. maxFlushedOffset: " + maxFlushedOffset.get() + ", maxRecordOffset: " + maxRecordOffset.get()); } - private static void testMultiThreadAppend0(boolean mergeWrite, boolean directIO) throws IOException, InterruptedException { + private static void testMultiThreadAppend0(boolean mergeWrite, + boolean directIO) throws IOException, InterruptedException { final int recordSize = 4096 + 1; final int recordCount = 10; final int threadCount = 8; @@ -251,7 +214,7 @@ private static void testMultiThreadAppend0(boolean mergeWrite, boolean directIO) } BlockWALService.BlockWALServiceBuilder builder = BlockWALService.builder(path, blockDeviceCapacity) - .direct(directIO); + .direct(directIO); if (!mergeWrite) { builder.blockSoftLimit(0); } @@ -296,7 +259,420 @@ private static void testMultiThreadAppend0(boolean mergeWrite, boolean directIO) wal.shutdownGracefully(); } assertTrue(maxFlushedOffset.get() > maxRecordOffset.get(), - "maxFlushedOffset should be greater than maxRecordOffset. maxFlushedOffset: " + maxFlushedOffset.get() + ", maxRecordOffset: " + maxRecordOffset.get()); + "maxFlushedOffset should be greater than maxRecordOffset. maxFlushedOffset: " + maxFlushedOffset.get() + ", maxRecordOffset: " + maxRecordOffset.get()); + } + + private static void testRecoverAfterMergeWrite0(boolean shutdown, boolean overCapacity, + boolean directIO) throws IOException { + final int recordSize = 1024 + 1; + final int recordCount = 100; + long blockDeviceCapacity; + if (overCapacity) { + blockDeviceCapacity = recordSize * recordCount + WAL_HEADER_TOTAL_CAPACITY; + } else { + blockDeviceCapacity = WALUtil.alignLargeByBlockSize(recordSize) * recordCount + WAL_HEADER_TOTAL_CAPACITY; + } + String path = TestUtils.tempFilePath(); + + if (directIO && TEST_BLOCK_DEVICE != null) { + path = TEST_BLOCK_DEVICE; + blockDeviceCapacity = WALUtil.alignLargeByBlockSize(blockDeviceCapacity); + resetBlockDevice(path, blockDeviceCapacity); + } + + // Append records + final WriteAheadLog previousWAL = BlockWALService.builder(path, blockDeviceCapacity) + .direct(directIO) + .build() + .start(); + recoverAndReset(previousWAL); + List appended = appendAsync(previousWAL, recordSize, recordCount); + if (shutdown) { + previousWAL.shutdownGracefully(); + } + + // Recover records + final WriteAheadLog wal = BlockWALService.builder(path, blockDeviceCapacity) + .direct(directIO) + .build() + .start(); + try { + Iterator recover = wal.recover(); + assertNotNull(recover); + + List recovered = new ArrayList<>(recordCount); + while (recover.hasNext()) { + RecoverResult next = recover.next(); + next.record().release(); + recovered.add(next.recordOffset()); + } + assertEquals(appended, recovered); + wal.reset().join(); + } finally { + wal.shutdownGracefully(); + } + } + + private static List appendAsync(WriteAheadLog wal, int recordSize, int recordCount) { + List appended = new ArrayList<>(recordCount); + List> appendFutures = new LinkedList<>(); + WriteBench.TrimOffset trimOffset = new WriteBench.TrimOffset(); + for (int i = 0; i < recordCount; i++) { + ByteBuf data = TestUtils.random(recordSize); + AppendResult appendResult; + try { + appendResult = wal.append(data.retainedDuplicate()); + } catch (OverCapacityException e) { + long offset = trimOffset.get(); + wal.trim(offset).join(); + appended = appended.stream() + .filter(recordOffset -> recordOffset > offset) + .collect(Collectors.toList()); + i--; + continue; + } + appended.add(appendResult.recordOffset()); + trimOffset.appended(appendResult.recordOffset()); + appendFutures.add(appendResult.future().whenComplete((callbackResult, throwable) -> { + assertNull(throwable); + assertEquals(0, callbackResult.flushedOffset() % WALUtil.BLOCK_SIZE); + trimOffset.flushed(callbackResult.flushedOffset()); + }).whenComplete((callbackResult, throwable) -> { + if (null != throwable) { + throwable.printStackTrace(); + System.exit(1); + } + }).thenApply(ignored -> null)); + } + CompletableFuture.allOf(appendFutures.toArray(new CompletableFuture[0])).join(); + return appended; + } + + public static Stream testRecoverFromDisasterData() { + return Stream.of( + new RecoverFromDisasterParam( + WALUtil.BLOCK_SIZE + 1, + 100L, + -1L, + 50L, + Arrays.asList(0L, 2L, 4L), + Arrays.asList(0L, 2L, 4L), + WALUtil.BLOCK_SIZE + ).toArguments("base"), + new RecoverFromDisasterParam( + WALUtil.BLOCK_SIZE + 1, + 100L, + 0L, + 50L, + Arrays.asList(0L, 2L, 4L), + Arrays.asList(2L, 4L), + WALUtil.BLOCK_SIZE + ).toArguments("trimmed at zero"), + new RecoverFromDisasterParam( + WALUtil.BLOCK_SIZE + 1, + 100L, + 2L, + 50L, + Arrays.asList(0L, 2L, 4L, 6L), + Arrays.asList(4L, 6L), + WALUtil.BLOCK_SIZE + ).toArguments("trimmed"), + new RecoverFromDisasterParam( + WALUtil.BLOCK_SIZE + 1, + 100L, + 2L, + 50L, + Arrays.asList(0L, 2L, 4L, 6L, 8L, 10L, 12L, 14L, 16L, 18L, 20L), + Arrays.asList(4L, 6L, 8L, 10L, 12L, 14L, 16L, 18L, 20L), + WALUtil.BLOCK_SIZE + ).toArguments("WAL header flushed slow"), + new RecoverFromDisasterParam( + WALUtil.BLOCK_SIZE + 1, + 100L, + 2L, + 50L, + Arrays.asList(0L, 2L, 8L, 10L, 14L, 20L), + Arrays.asList(8L, 10L, 14L, 20L), + WALUtil.BLOCK_SIZE + ).toArguments("many invalid records"), + new RecoverFromDisasterParam( + WALUtil.BLOCK_SIZE + 1, + 100L, + 2L, + 50L, + Arrays.asList(14L, 8L, 10L, 20L, 0L, 2L), + Arrays.asList(8L, 10L, 14L, 20L), + WALUtil.BLOCK_SIZE + ).toArguments("write in random order"), + new RecoverFromDisasterParam( + WALUtil.BLOCK_SIZE + 1, + 100L, + 20230920L, + 50L, + Arrays.asList(20230900L, 20230910L, 20230916L, 20230920L, 20230930L, 20230940L, 20230950L, 20230960L, 20230970L, 20230980L), + Arrays.asList(20230930L, 20230940L, 20230950L, 20230960L, 20230970L), + WALUtil.BLOCK_SIZE + ).toArguments("big logic offset"), + new RecoverFromDisasterParam( + WALUtil.BLOCK_SIZE + 1, + 100L, + 180L, + 50L, + Arrays.asList(150L, 160L, 170L, 180L, 190L, 200L, 202L, 210L, 220L, 230L, 240L), + Arrays.asList(190L, 200L, 202L, 210L, 220L, 230L), + WALUtil.BLOCK_SIZE + ).toArguments("round robin"), + new RecoverFromDisasterParam( + WALUtil.BLOCK_SIZE + 1, + 100L, + 210L, + 50L, + Arrays.asList(111L, 113L, 115L, 117L, 119L, 120L, 130L, + 210L, 215L, 220L, 230L, 240L, 250L, 260L, 270L, 280L, 290L), + Arrays.asList(215L, 220L, 230L, 240L, 250L, 260L), + WALUtil.BLOCK_SIZE + ).toArguments("overwrite"), + new RecoverFromDisasterParam( + WALUtil.BLOCK_SIZE + 1, + 100L, + -1L, + 1L, + Arrays.asList(0L, 2L, 5L, 7L), + List.of(0L, 2L), + WALUtil.BLOCK_SIZE + ).toArguments("small window - record size not aligned"), + new RecoverFromDisasterParam( + WALUtil.BLOCK_SIZE + 1, + 100L, + 10L, + 3L, + Arrays.asList(10L, 12L, 15L, 17L, 19L), + List.of(12L, 15L), + WALUtil.BLOCK_SIZE + ).toArguments("invalid record in window - record size not aligned"), + new RecoverFromDisasterParam( + WALUtil.BLOCK_SIZE + 1, + 100L, + 10L, + 9L, + Arrays.asList(9L, 14L, 18L, 20L), + List.of(14L, 18L), + WALUtil.BLOCK_SIZE + ).toArguments("trim at an invalid record - record size not aligned"), + new RecoverFromDisasterParam( + WALUtil.BLOCK_SIZE, + 100L, + -1L, + 1L, + Arrays.asList(0L, 1L, 3L, 4L, 5L), + List.of(0L, 1L), + WALUtil.BLOCK_SIZE + ).toArguments("small window - record size aligned"), + new RecoverFromDisasterParam( + WALUtil.BLOCK_SIZE, + 100L, + 10L, + 3L, + Arrays.asList(10L, 11L, 13L, 14L, 15L, 16L), + List.of(11L, 13L, 14L), + WALUtil.BLOCK_SIZE + ).toArguments("invalid record in window - record size aligned"), + new RecoverFromDisasterParam( + WALUtil.BLOCK_SIZE, + 100L, + 10L, + 5L, + Arrays.asList(9L, 11L, 13L, 15L, 16L, 17L), + List.of(11L, 13L, 15L, 16L), + WALUtil.BLOCK_SIZE + ).toArguments("trim at an invalid record - record size aligned"), + new RecoverFromDisasterParam( + WALUtil.BLOCK_SIZE, + 100L, + 10L, + 0L, + Arrays.asList(10L, 11L, 12L, 14L), + List.of(11L, 12L), + WALUtil.BLOCK_SIZE + ).toArguments("zero window"), + new RecoverFromDisasterParam( + 42, + 8192L, + -1L, + 8192L, + Arrays.asList(0L, 42L, 84L), + Arrays.asList(0L, 42L, 84L), + 1 + ).toArguments("merge write - base"), + new RecoverFromDisasterParam( + 42, + 8192L, + 42L, + 8192L, + Arrays.asList(0L, 42L, 84L, 126L), + Arrays.asList(84L, 126L), + 1 + ).toArguments("merge write - trimmed"), + new RecoverFromDisasterParam( + 42, + 8192L, + 42L, + 8192L, + Arrays.asList(0L, 42L, 42 * 2L, 42 * 4L, 4096L, 4096L + 42L, 4096L + 42 * 3L), + Arrays.asList(42 * 2L, 4096L, 4096L + 42L), + 1 + ).toArguments("merge write - some invalid records"), + new RecoverFromDisasterParam( + 42, + 8192L, + 42L, + 8192L, + Arrays.asList(42L, 42 * 4L, 42 * 2L, 4096L + 42 * 3L, 0L, 4096L, 4096L + 42L), + Arrays.asList(42 * 2L, 4096L, 4096L + 42L), + 1 + ).toArguments("merge write - random order"), + new RecoverFromDisasterParam( + 1000, + 8192L, + 2000L, + 8192L, + Arrays.asList(0L, 1000L, 2000L, 3000L, 4000L, 5000L, 7000L), + Arrays.asList(3000L, 4000L, 5000L), + 1 + ).toArguments("merge write - record in the middle"), + new RecoverFromDisasterParam( + 42, + 8192L, + 8192L + 4096L + 42L, + 8192L, + Arrays.asList(8192L + 4096L, 8192L + 4096L + 42L, 8192L + 4096L + 42 * 2L, 8192L + 4096L + 42 * 4L, 16384L, 16384L + 42L, 16384L + 42 * 3L), + Arrays.asList(8192L + 4096L + 42 * 2L, 16384L, 16384L + 42L), + 1 + ).toArguments("merge write - round robin"), + new RecoverFromDisasterParam( + 1000, + 8192L, + 12000L, + 8192L, + Arrays.asList(1000L, 2000L, 3000L, 4000L, 5000L, 6000L, 7000L, + 9000L, 10000L, 11000L, 12000L, 13000L, 14000L, 15000L), + Arrays.asList(13000L, 14000L, 15000L), + 1 + ).toArguments("merge write - overwrite"), + new RecoverFromDisasterParam( + 42, + 4096L * 20, + -1L, + 4096L, + Arrays.asList(0L, 42L, 42 * 3L, 4096L, 4096L + 42L, 4096L + 42 * 3L, 12288L, 12288L + 42L, 12288L + 42 * 3L, 16384L), + Arrays.asList(0L, 42L, 4096L, 4096L + 42L), + 1 + ).toArguments("merge write - small window"), + new RecoverFromDisasterParam( + 42, + 4096L * 20, + 4096L * 2, + 4096L * 4, + Arrays.asList(4096L * 2, 4096L * 2 + 42L, 4096L * 2 + 42 * 3L, + 4096L * 4, 4096L * 4 + 42L, 4096L * 4 + 42 * 3L, + 4096L * 5, 4096L * 5 + 42L, 4096L * 5 + 42 * 3L, + 4096L * 6, 4096L * 6 + 42L, 4096L * 6 + 42 * 3L, + 4096L * 7, 4096L * 7 + 42L, 4096L * 7 + 42 * 3L, + 4096L * 8), + Arrays.asList(4096L * 2 + 42L, 4096L * 4, 4096L * 4 + 42L, + 4096L * 5, 4096L * 5 + 42L, 4096L * 6, 4096L * 6 + 42L), + 1 + ).toArguments("merge write - invalid record in window"), + new RecoverFromDisasterParam( + 42, + 4096L * 20, + 4096L * 2 + 42 * 2L, + 4096L * 2, + Arrays.asList(4096L * 2, 4096L * 2 + 42L, 4096L * 2 + 42 * 3L, + 4096L * 3, 4096L * 3 + 42L, 4096L * 3 + 42 * 3L, + 4096L * 5, 4096L * 5 + 42L, 4096L * 5 + 42 * 3L, + 4096L * 6, 4096L * 6 + 42L, 4096L * 6 + 42 * 3L, + 4096L * 7), + Arrays.asList(4096L * 3, 4096L * 3 + 42L, 4096L * 5, 4096L * 5 + 42L), + 1 + ).toArguments("merge write - trim at an invalid record"), + new RecoverFromDisasterParam( + 42, + 4096L * 20, + 4096L * 2, + 0L, + Arrays.asList(4096L * 2, 4096L * 2 + 42L, 4096L * 2 + 42 * 3L, + 4096L * 3, 4096L * 3 + 42L, 4096L * 3 + 42 * 3L, + 4096L * 5, 4096L * 5 + 42L, 4096L * 5 + 42 * 3L, + 4096L * 6), + Arrays.asList(4096L * 2 + 42L, 4096L * 3, 4096L * 3 + 42L), + 1 + ).toArguments("merge write - zero window") + ); + } + + /** + * Call {@link WriteAheadLog#recover()} {@link WriteAheadLog#reset()} and drop all records. + */ + private static void recoverAndReset(WriteAheadLog wal) { + for (Iterator it = wal.recover(); it.hasNext(); ) { + it.next().record().release(); + } + wal.reset().join(); + } + + /** + * Write "0"s to the block device to reset it. + */ + private static void resetBlockDevice(String path, long capacity) throws IOException { + WALChannel channel = WALChannel.builder(path) + .capacity(capacity) + .direct(true) + .build(); + channel.open(); + ByteBuf buf = Unpooled.buffer((int) capacity); + buf.writeZero((int) capacity); + channel.write(buf, 0); + channel.close(); + } + + @ParameterizedTest(name = "Test {index}: mergeWrite={0}") + @ValueSource(booleans = {false, true}) + public void testSingleThreadAppendBasic(boolean mergeWrite) throws IOException, OverCapacityException { + testSingleThreadAppendBasic0(mergeWrite, false); + } + + @ParameterizedTest(name = "Test {index}: mergeWrite={0}") + @ValueSource(booleans = {false, true}) + @EnabledOnOs(OS.LINUX) + public void testSingleThreadAppendBasicDirectIO(boolean mergeWrite) throws IOException, OverCapacityException { + testSingleThreadAppendBasic0(mergeWrite, true); + } + + @ParameterizedTest(name = "Test {index}: mergeWrite={0}") + @ValueSource(booleans = {false, true}) + public void testSingleThreadAppendWhenOverCapacity(boolean mergeWrite) throws IOException { + testSingleThreadAppendWhenOverCapacity0(mergeWrite, false); + } + + @ParameterizedTest(name = "Test {index}: mergeWrite={0}") + @ValueSource(booleans = {false, true}) + @EnabledOnOs(OS.LINUX) + public void testSingleThreadAppendWhenOverCapacityDirectIO(boolean mergeWrite) throws IOException { + testSingleThreadAppendWhenOverCapacity0(mergeWrite, true); + } + + @ParameterizedTest(name = "Test {index}: mergeWrite={0}") + @ValueSource(booleans = {false, true}) + public void testMultiThreadAppend(boolean mergeWrite) throws InterruptedException, IOException { + testMultiThreadAppend0(mergeWrite, false); + } + + @ParameterizedTest(name = "Test {index}: mergeWrite={0}") + @ValueSource(booleans = {false, true}) + @EnabledOnOs(OS.LINUX) + public void testMultiThreadAppendDirectIO(boolean mergeWrite) throws InterruptedException, IOException { + testMultiThreadAppend0(mergeWrite, true); } private long append(WriteAheadLog wal, int recordSize) throws OverCapacityException { @@ -322,15 +698,16 @@ private List append(WriteAheadLog wal, int recordSize, int recordCount) { wal.trim(offset).join(); final long trimmedOffset = offset; recordOffsets = recordOffsets.stream() - .filter(recordOffset -> recordOffset > trimmedOffset) - .collect(Collectors.toList()); + .filter(recordOffset -> recordOffset > trimmedOffset) + .collect(Collectors.toList()); i--; } } return recordOffsets; } - private List appendWithoutTrim(WriteAheadLog wal, int recordSize, int recordCount) throws OverCapacityException { + private List appendWithoutTrim(WriteAheadLog wal, int recordSize, + int recordCount) throws OverCapacityException { List recordOffsets = new ArrayList<>(recordCount); for (int i = 0; i < recordCount; i++) { long offset = append(wal, recordSize); @@ -368,11 +745,13 @@ public void testSingleThreadRecover(boolean shutdown, boolean overCapacity, int "false, true, 11", }) @EnabledOnOs(OS.LINUX) - public void testSingleThreadRecoverDirectIO(boolean shutdown, boolean overCapacity, int recordCount) throws IOException { + public void testSingleThreadRecoverDirectIO(boolean shutdown, boolean overCapacity, + int recordCount) throws IOException { testSingleThreadRecover0(shutdown, overCapacity, recordCount, true); } - private void testSingleThreadRecover0(boolean shutdown, boolean overCapacity, int recordCount, boolean directIO) throws IOException { + private void testSingleThreadRecover0(boolean shutdown, boolean overCapacity, int recordCount, + boolean directIO) throws IOException { final int recordSize = 4096 + 1; long blockDeviceCapacity; if (overCapacity) { @@ -390,9 +769,9 @@ private void testSingleThreadRecover0(boolean shutdown, boolean overCapacity, in // Append records final WriteAheadLog previousWAL = BlockWALService.builder(path, blockDeviceCapacity) - .direct(directIO) - .build() - .start(); + .direct(directIO) + .build() + .start(); recoverAndReset(previousWAL); List appended = append(previousWAL, recordSize, recordCount); if (shutdown) { @@ -401,9 +780,9 @@ private void testSingleThreadRecover0(boolean shutdown, boolean overCapacity, in // Recover records final WriteAheadLog wal = BlockWALService.builder(path, blockDeviceCapacity) - .direct(directIO) - .build() - .start(); + .direct(directIO) + .build() + .start(); try { Iterator recover = wal.recover(); assertNotNull(recover); @@ -444,91 +823,6 @@ public void testRecoverAfterMergeWriteDirectIO(boolean shutdown, boolean overCap testRecoverAfterMergeWrite0(shutdown, overCapacity, true); } - private static void testRecoverAfterMergeWrite0(boolean shutdown, boolean overCapacity, boolean directIO) throws IOException { - final int recordSize = 1024 + 1; - final int recordCount = 100; - long blockDeviceCapacity; - if (overCapacity) { - blockDeviceCapacity = recordSize * recordCount + WAL_HEADER_TOTAL_CAPACITY; - } else { - blockDeviceCapacity = WALUtil.alignLargeByBlockSize(recordSize) * recordCount + WAL_HEADER_TOTAL_CAPACITY; - } - String path = TestUtils.tempFilePath(); - - if (directIO && TEST_BLOCK_DEVICE != null) { - path = TEST_BLOCK_DEVICE; - blockDeviceCapacity = WALUtil.alignLargeByBlockSize(blockDeviceCapacity); - resetBlockDevice(path, blockDeviceCapacity); - } - - // Append records - final WriteAheadLog previousWAL = BlockWALService.builder(path, blockDeviceCapacity) - .direct(directIO) - .build() - .start(); - recoverAndReset(previousWAL); - List appended = appendAsync(previousWAL, recordSize, recordCount); - if (shutdown) { - previousWAL.shutdownGracefully(); - } - - // Recover records - final WriteAheadLog wal = BlockWALService.builder(path, blockDeviceCapacity) - .direct(directIO) - .build() - .start(); - try { - Iterator recover = wal.recover(); - assertNotNull(recover); - - List recovered = new ArrayList<>(recordCount); - while (recover.hasNext()) { - RecoverResult next = recover.next(); - next.record().release(); - recovered.add(next.recordOffset()); - } - assertEquals(appended, recovered); - wal.reset().join(); - } finally { - wal.shutdownGracefully(); - } - } - - private static List appendAsync(WriteAheadLog wal, int recordSize, int recordCount) { - List appended = new ArrayList<>(recordCount); - List> appendFutures = new LinkedList<>(); - WriteBench.TrimOffset trimOffset = new WriteBench.TrimOffset(); - for (int i = 0; i < recordCount; i++) { - ByteBuf data = TestUtils.random(recordSize); - AppendResult appendResult; - try { - appendResult = wal.append(data.retainedDuplicate()); - } catch (OverCapacityException e) { - long offset = trimOffset.get(); - wal.trim(offset).join(); - appended = appended.stream() - .filter(recordOffset -> recordOffset > offset) - .collect(Collectors.toList()); - i--; - continue; - } - appended.add(appendResult.recordOffset()); - trimOffset.appended(appendResult.recordOffset()); - appendFutures.add(appendResult.future().whenComplete((callbackResult, throwable) -> { - assertNull(throwable); - assertEquals(0, callbackResult.flushedOffset() % WALUtil.BLOCK_SIZE); - trimOffset.flushed(callbackResult.flushedOffset()); - }).whenComplete((callbackResult, throwable) -> { - if (null != throwable) { - throwable.printStackTrace(); - System.exit(1); - } - }).thenApply(ignored -> null)); - } - CompletableFuture.allOf(appendFutures.toArray(new CompletableFuture[0])).join(); - return appended; - } - @Test public void testAppendAfterRecover() throws IOException, OverCapacityException { testAppendAfterRecover0(false); @@ -550,9 +844,9 @@ private void testAppendAfterRecover0(boolean directIO) throws IOException, OverC } final WriteAheadLog previousWAL = BlockWALService.builder(path, 1 << 20) - .direct(directIO) - .build() - .start(); + .direct(directIO) + .build() + .start(); recoverAndReset(previousWAL); // Append 2 records long appended0 = append(previousWAL, recordSize); @@ -561,9 +855,9 @@ private void testAppendAfterRecover0(boolean directIO) throws IOException, OverC assertEquals(WALUtil.alignLargeByBlockSize(recordSize), appended1); final WriteAheadLog wal = BlockWALService.builder(path, 1 << 20) - .direct(directIO) - .build() - .start(); + .direct(directIO) + .build() + .start(); try { // Recover records Iterator recover = wal.recover(); @@ -589,14 +883,13 @@ private void testAppendAfterRecover0(boolean directIO) throws IOException, OverC } } - private ByteBuf recordHeader(ByteBuf body, long offset) { return new SlidingWindowService.RecordHeaderCoreData() - .setMagicCode(BlockWALService.RECORD_HEADER_MAGIC_CODE) - .setRecordBodyLength(body.readableBytes()) - .setRecordBodyOffset(offset + BlockWALService.RECORD_HEADER_SIZE) - .setRecordBodyCRC(WALUtil.crc32(body)) - .marshal(); + .setMagicCode(BlockWALService.RECORD_HEADER_MAGIC_CODE) + .setRecordBodyLength(body.readableBytes()) + .setRecordBodyOffset(offset + BlockWALService.RECORD_HEADER_SIZE) + .setRecordBodyCRC(WALUtil.crc32(body)) + .marshal(); } private void write(WALChannel walChannel, long logicOffset, int recordSize) throws IOException { @@ -612,316 +905,21 @@ private void write(WALChannel walChannel, long logicOffset, int recordSize) thro private void writeWALHeader(WALChannel walChannel, long trimOffset, long maxLength) throws IOException { ByteBuf header = new WALHeader(walChannel.capacity(), maxLength) - .updateTrimOffset(trimOffset) - .marshal(); + .updateTrimOffset(trimOffset) + .marshal(); walChannel.writeAndFlush(header, 0); } - private static class RecoverFromDisasterParam { - int recordSize; - long capacity; - // WAL header - long trimOffset; - long maxLength; - // WAL records - List writeOffsets; - List recoveredOffsets; - - public RecoverFromDisasterParam( - int recordSize, - long capacity, - long trimOffset, - long maxLength, - List writeOffsets, - List recoveredOffsets, - int unit - ) { - this.recordSize = recordSize; - this.capacity = capacity * unit + WAL_HEADER_TOTAL_CAPACITY; - this.trimOffset = trimOffset * unit; - this.maxLength = maxLength * unit; - this.writeOffsets = writeOffsets.stream().map(offset -> offset * unit).collect(Collectors.toList()); - this.recoveredOffsets = recoveredOffsets.stream().map(offset -> offset * unit).collect(Collectors.toList()); - } - - public Arguments toArguments(String name) { - return Arguments.of(name, recordSize, capacity, trimOffset, maxLength, writeOffsets, recoveredOffsets); - } - } - - public static Stream testRecoverFromDisasterData() { - return Stream.of( - new RecoverFromDisasterParam( - WALUtil.BLOCK_SIZE + 1, - 100L, - -1L, - 50L, - Arrays.asList(0L, 2L, 4L), - Arrays.asList(0L, 2L, 4L), - WALUtil.BLOCK_SIZE - ).toArguments("base"), - new RecoverFromDisasterParam( - WALUtil.BLOCK_SIZE + 1, - 100L, - 0L, - 50L, - Arrays.asList(0L, 2L, 4L), - Arrays.asList(2L, 4L), - WALUtil.BLOCK_SIZE - ).toArguments("trimmed at zero"), - new RecoverFromDisasterParam( - WALUtil.BLOCK_SIZE + 1, - 100L, - 2L, - 50L, - Arrays.asList(0L, 2L, 4L, 6L), - Arrays.asList(4L, 6L), - WALUtil.BLOCK_SIZE - ).toArguments("trimmed"), - new RecoverFromDisasterParam( - WALUtil.BLOCK_SIZE + 1, - 100L, - 2L, - 50L, - Arrays.asList(0L, 2L, 4L, 6L, 8L, 10L, 12L, 14L, 16L, 18L, 20L), - Arrays.asList(4L, 6L, 8L, 10L, 12L, 14L, 16L, 18L, 20L), - WALUtil.BLOCK_SIZE - ).toArguments("WAL header flushed slow"), - new RecoverFromDisasterParam( - WALUtil.BLOCK_SIZE + 1, - 100L, - 2L, - 50L, - Arrays.asList(0L, 2L, 8L, 10L, 14L, 20L), - Arrays.asList(8L, 10L, 14L, 20L), - WALUtil.BLOCK_SIZE - ).toArguments("many invalid records"), - new RecoverFromDisasterParam( - WALUtil.BLOCK_SIZE + 1, - 100L, - 2L, - 50L, - Arrays.asList(14L, 8L, 10L, 20L, 0L, 2L), - Arrays.asList(8L, 10L, 14L, 20L), - WALUtil.BLOCK_SIZE - ).toArguments("write in random order"), - new RecoverFromDisasterParam( - WALUtil.BLOCK_SIZE + 1, - 100L, - 20230920L, - 50L, - Arrays.asList(20230900L, 20230910L, 20230916L, 20230920L, 20230930L, 20230940L, 20230950L, 20230960L, 20230970L, 20230980L), - Arrays.asList(20230930L, 20230940L, 20230950L, 20230960L, 20230970L), - WALUtil.BLOCK_SIZE - ).toArguments("big logic offset"), - new RecoverFromDisasterParam( - WALUtil.BLOCK_SIZE + 1, - 100L, - 180L, - 50L, - Arrays.asList(150L, 160L, 170L, 180L, 190L, 200L, 202L, 210L, 220L, 230L, 240L), - Arrays.asList(190L, 200L, 202L, 210L, 220L, 230L), - WALUtil.BLOCK_SIZE - ).toArguments("round robin"), - new RecoverFromDisasterParam( - WALUtil.BLOCK_SIZE + 1, - 100L, - 210L, - 50L, - Arrays.asList(111L, 113L, 115L, 117L, 119L, 120L, 130L, - 210L, 215L, 220L, 230L, 240L, 250L, 260L, 270L, 280L, 290L), - Arrays.asList(215L, 220L, 230L, 240L, 250L, 260L), - WALUtil.BLOCK_SIZE - ).toArguments("overwrite"), - new RecoverFromDisasterParam( - WALUtil.BLOCK_SIZE + 1, - 100L, - -1L, - 1L, - Arrays.asList(0L, 2L, 5L, 7L), - List.of(0L, 2L), - WALUtil.BLOCK_SIZE - ).toArguments("small window - record size not aligned"), - new RecoverFromDisasterParam( - WALUtil.BLOCK_SIZE + 1, - 100L, - 10L, - 3L, - Arrays.asList(10L, 12L, 15L, 17L, 19L), - List.of(12L, 15L), - WALUtil.BLOCK_SIZE - ).toArguments("invalid record in window - record size not aligned"), - new RecoverFromDisasterParam( - WALUtil.BLOCK_SIZE + 1, - 100L, - 10L, - 9L, - Arrays.asList(9L, 14L, 18L, 20L), - List.of(14L, 18L), - WALUtil.BLOCK_SIZE - ).toArguments("trim at an invalid record - record size not aligned"), - new RecoverFromDisasterParam( - WALUtil.BLOCK_SIZE, - 100L, - -1L, - 1L, - Arrays.asList(0L, 1L, 3L, 4L, 5L), - List.of(0L, 1L), - WALUtil.BLOCK_SIZE - ).toArguments("small window - record size aligned"), - new RecoverFromDisasterParam( - WALUtil.BLOCK_SIZE, - 100L, - 10L, - 3L, - Arrays.asList(10L, 11L, 13L, 14L, 15L, 16L), - List.of(11L, 13L, 14L), - WALUtil.BLOCK_SIZE - ).toArguments("invalid record in window - record size aligned"), - new RecoverFromDisasterParam( - WALUtil.BLOCK_SIZE, - 100L, - 10L, - 5L, - Arrays.asList(9L, 11L, 13L, 15L, 16L, 17L), - List.of(11L, 13L, 15L, 16L), - WALUtil.BLOCK_SIZE - ).toArguments("trim at an invalid record - record size aligned"), - new RecoverFromDisasterParam( - WALUtil.BLOCK_SIZE, - 100L, - 10L, - 0L, - Arrays.asList(10L, 11L, 12L, 14L), - List.of(11L, 12L), - WALUtil.BLOCK_SIZE - ).toArguments("zero window"), - new RecoverFromDisasterParam( - 42, - 8192L, - -1L, - 8192L, - Arrays.asList(0L, 42L, 84L), - Arrays.asList(0L, 42L, 84L), - 1 - ).toArguments("merge write - base"), - new RecoverFromDisasterParam( - 42, - 8192L, - 42L, - 8192L, - Arrays.asList(0L, 42L, 84L, 126L), - Arrays.asList(84L, 126L), - 1 - ).toArguments("merge write - trimmed"), - new RecoverFromDisasterParam( - 42, - 8192L, - 42L, - 8192L, - Arrays.asList(0L, 42L, 42 * 2L, 42 * 4L, 4096L, 4096L + 42L, 4096L + 42 * 3L), - Arrays.asList(42 * 2L, 4096L, 4096L + 42L), - 1 - ).toArguments("merge write - some invalid records"), - new RecoverFromDisasterParam( - 42, - 8192L, - 42L, - 8192L, - Arrays.asList(42L, 42 * 4L, 42 * 2L, 4096L + 42 * 3L, 0L, 4096L, 4096L + 42L), - Arrays.asList(42 * 2L, 4096L, 4096L + 42L), - 1 - ).toArguments("merge write - random order"), - new RecoverFromDisasterParam( - 1000, - 8192L, - 2000L, - 8192L, - Arrays.asList(0L, 1000L, 2000L, 3000L, 4000L, 5000L, 7000L), - Arrays.asList(3000L, 4000L, 5000L), - 1 - ).toArguments("merge write - record in the middle"), - new RecoverFromDisasterParam( - 42, - 8192L, - 8192L + 4096L + 42L, - 8192L, - Arrays.asList(8192L + 4096L, 8192L + 4096L + 42L, 8192L + 4096L + 42 * 2L, 8192L + 4096L + 42 * 4L, 16384L, 16384L + 42L, 16384L + 42 * 3L), - Arrays.asList(8192L + 4096L + 42 * 2L, 16384L, 16384L + 42L), - 1 - ).toArguments("merge write - round robin"), - new RecoverFromDisasterParam( - 1000, - 8192L, - 12000L, - 8192L, - Arrays.asList(1000L, 2000L, 3000L, 4000L, 5000L, 6000L, 7000L, - 9000L, 10000L, 11000L, 12000L, 13000L, 14000L, 15000L), - Arrays.asList(13000L, 14000L, 15000L), - 1 - ).toArguments("merge write - overwrite"), - new RecoverFromDisasterParam( - 42, - 4096L * 20, - -1L, - 4096L, - Arrays.asList(0L, 42L, 42 * 3L, 4096L, 4096L + 42L, 4096L + 42 * 3L, 12288L, 12288L + 42L, 12288L + 42 * 3L, 16384L), - Arrays.asList(0L, 42L, 4096L, 4096L + 42L), - 1 - ).toArguments("merge write - small window"), - new RecoverFromDisasterParam( - 42, - 4096L * 20, - 4096L * 2, - 4096L * 4, - Arrays.asList(4096L * 2, 4096L * 2 + 42L, 4096L * 2 + 42 * 3L, - 4096L * 4, 4096L * 4 + 42L, 4096L * 4 + 42 * 3L, - 4096L * 5, 4096L * 5 + 42L, 4096L * 5 + 42 * 3L, - 4096L * 6, 4096L * 6 + 42L, 4096L * 6 + 42 * 3L, - 4096L * 7, 4096L * 7 + 42L, 4096L * 7 + 42 * 3L, - 4096L * 8), - Arrays.asList(4096L * 2 + 42L, 4096L * 4, 4096L * 4 + 42L, - 4096L * 5, 4096L * 5 + 42L, 4096L * 6, 4096L * 6 + 42L), - 1 - ).toArguments("merge write - invalid record in window"), - new RecoverFromDisasterParam( - 42, - 4096L * 20, - 4096L * 2 + 42 * 2L, - 4096L * 2, - Arrays.asList(4096L * 2, 4096L * 2 + 42L, 4096L * 2 + 42 * 3L, - 4096L * 3, 4096L * 3 + 42L, 4096L * 3 + 42 * 3L, - 4096L * 5, 4096L * 5 + 42L, 4096L * 5 + 42 * 3L, - 4096L * 6, 4096L * 6 + 42L, 4096L * 6 + 42 * 3L, - 4096L * 7), - Arrays.asList(4096L * 3, 4096L * 3 + 42L, 4096L * 5, 4096L * 5 + 42L), - 1 - ).toArguments("merge write - trim at an invalid record"), - new RecoverFromDisasterParam( - 42, - 4096L * 20, - 4096L * 2, - 0L, - Arrays.asList(4096L * 2, 4096L * 2 + 42L, 4096L * 2 + 42 * 3L, - 4096L * 3, 4096L * 3 + 42L, 4096L * 3 + 42 * 3L, - 4096L * 5, 4096L * 5 + 42L, 4096L * 5 + 42 * 3L, - 4096L * 6), - Arrays.asList(4096L * 2 + 42L, 4096L * 3, 4096L * 3 + 42L), - 1 - ).toArguments("merge write - zero window") - ); - } - @ParameterizedTest(name = "Test {index} {0}") @MethodSource("testRecoverFromDisasterData") public void testRecoverFromDisaster( - String name, - int recordSize, - long capacity, - long trimOffset, - long maxLength, - List writeOffsets, - List recoveredOffsets + String name, + int recordSize, + long capacity, + long trimOffset, + long maxLength, + List writeOffsets, + List recoveredOffsets ) throws IOException { testRecoverFromDisaster0(name, recordSize, capacity, trimOffset, maxLength, writeOffsets, recoveredOffsets, false); } @@ -930,26 +928,26 @@ public void testRecoverFromDisaster( @MethodSource("testRecoverFromDisasterData") @EnabledOnOs({OS.LINUX}) public void testRecoverFromDisasterDirectIO( - String name, - int recordSize, - long capacity, - long trimOffset, - long maxLength, - List writeOffsets, - List recoveredOffsets + String name, + int recordSize, + long capacity, + long trimOffset, + long maxLength, + List writeOffsets, + List recoveredOffsets ) throws IOException { testRecoverFromDisaster0(name, recordSize, capacity, trimOffset, maxLength, writeOffsets, recoveredOffsets, true); } private void testRecoverFromDisaster0( - String name, - int recordSize, - long capacity, - long trimOffset, - long maxLength, - List writeOffsets, - List recoveredOffsets, - boolean directIO + String name, + int recordSize, + long capacity, + long trimOffset, + long maxLength, + List writeOffsets, + List recoveredOffsets, + boolean directIO ) throws IOException { String path = TestUtils.tempFilePath(); if (directIO && TEST_BLOCK_DEVICE != null) { @@ -965,9 +963,9 @@ private void testRecoverFromDisaster0( walChannel = blockDeviceChannel; } else { walChannel = WALChannel.builder(path) - .capacity(capacity) - .direct(false) - .build(); + .capacity(capacity) + .direct(false) + .build(); } // Simulate disaster @@ -979,9 +977,9 @@ private void testRecoverFromDisaster0( walChannel.close(); final WriteAheadLog wal = BlockWALService.builder(path, capacity) - .direct(directIO) - .build() - .start(); + .direct(directIO) + .build() + .start(); try { // Recover records Iterator recover = wal.recover(); @@ -1024,17 +1022,17 @@ private void testRecoverAfterReset0(boolean directIO) throws IOException, OverCa // 1. append and force shutdown final WriteAheadLog wal1 = BlockWALService.builder(path, blockDeviceCapacity) - .direct(directIO) - .build() - .start(); + .direct(directIO) + .build() + .start(); recoverAndReset(wal1); List appended1 = appendWithoutTrim(wal1, recordSize, recordCount); // 2. recover and reset final WriteAheadLog wal2 = BlockWALService.builder(path, blockDeviceCapacity) - .direct(directIO) - .build() - .start(); + .direct(directIO) + .build() + .start(); Iterator recover = wal2.recover(); assertNotNull(recover); List recovered1 = new ArrayList<>(recordCount); @@ -1051,9 +1049,9 @@ private void testRecoverAfterReset0(boolean directIO) throws IOException, OverCa // 4. recover again final WriteAheadLog wal3 = BlockWALService.builder(path, blockDeviceCapacity) - .direct(directIO) - .build() - .start(); + .direct(directIO) + .build() + .start(); recover = wal3.recover(); assertNotNull(recover); List recovered2 = new ArrayList<>(recordCount); @@ -1068,8 +1066,8 @@ private void testRecoverAfterReset0(boolean directIO) throws IOException, OverCa @Test public void testTrimInvalidOffset() throws IOException, OverCapacityException { final WriteAheadLog wal = BlockWALService.builder(TestUtils.tempFilePath(), 16384) - .build() - .start(); + .build() + .start(); recoverAndReset(wal); try { long appended = append(wal, 42); @@ -1082,9 +1080,9 @@ public void testTrimInvalidOffset() throws IOException, OverCapacityException { @Test public void testWindowGreaterThanCapacity() throws IOException, OverCapacityException { final WriteAheadLog wal = BlockWALService.builder(TestUtils.tempFilePath(), WALUtil.BLOCK_SIZE * 3L) - .slidingWindowUpperLimit(WALUtil.BLOCK_SIZE * 4L) - .build() - .start(); + .slidingWindowUpperLimit(WALUtil.BLOCK_SIZE * 4L) + .build() + .start(); recoverAndReset(wal); try { append(wal, 42); @@ -1118,19 +1116,19 @@ private void testRecoveryMode0(boolean directIO) throws IOException, OverCapacit // simulate a crash WriteAheadLog wal1 = BlockWALService.builder(path, capacity) - .nodeId(nodeId) - .epoch(epoch) - .direct(directIO) - .build() - .start(); + .nodeId(nodeId) + .epoch(epoch) + .direct(directIO) + .build() + .start(); recoverAndReset(wal1); wal1.append(TestUtils.random(4097)).future().join(); // open in recovery mode WriteAheadLog wal2 = BlockWALService.recoveryBuilder(path) - .direct(directIO) - .build() - .start(); + .direct(directIO) + .build() + .start(); assertEquals(nodeId, wal2.metadata().nodeId()); assertEquals(epoch, wal2.metadata().epoch()); // we can recover and reset the WAL @@ -1158,16 +1156,16 @@ private void testCapacityMismatchFileSize0(boolean directIO) throws IOException // init a WAL with capacity1 WriteAheadLog wal1 = BlockWALService.builder(path, capacity1) - .direct(directIO) - .build() - .start(); + .direct(directIO) + .build() + .start(); recoverAndReset(wal1); wal1.shutdownGracefully(); // try to open it with capacity2 WriteAheadLog wal2 = BlockWALService.builder(path, capacity2) - .direct(directIO) - .build(); + .direct(directIO) + .build(); assertThrows(WALCapacityMismatchException.class, wal2::start); } @@ -1194,25 +1192,25 @@ private void testCapacityMismatchInHeader0(boolean directIO) throws IOException // init a WAL with capacity1 WriteAheadLog wal1 = BlockWALService.builder(path, capacity1) - .direct(directIO) - .build() - .start(); + .direct(directIO) + .build() + .start(); recoverAndReset(wal1); wal1.shutdownGracefully(); // overwrite the capacity in the header with capacity2 WALChannel walChannel = WALChannel.builder(path) - .capacity(capacity1) - .direct(directIO) - .build(); + .capacity(capacity1) + .direct(directIO) + .build(); walChannel.open(); walChannel.writeAndFlush(new WALHeader(capacity2, 42).marshal(), 0); walChannel.close(); // try to open it with capacity1 WriteAheadLog wal2 = BlockWALService.builder(path, capacity1) - .direct(directIO) - .build(); + .direct(directIO) + .build(); assertThrows(WALCapacityMismatchException.class, wal2::start); } @@ -1231,8 +1229,8 @@ private void testRecoveryModeWALFileNotExist0(boolean directIO) throws IOExcepti final String path = TestUtils.tempFilePath(); WriteAheadLog wal = BlockWALService.recoveryBuilder(path) - .direct(directIO) - .build(); + .direct(directIO) + .build(); assertThrows(WALNotInitializedException.class, wal::start); } @@ -1258,50 +1256,57 @@ private void testRecoveryModeNoHeader0(boolean directIO) throws IOException { // init a WAL WriteAheadLog wal1 = BlockWALService.builder(path, capacity) - .direct(directIO) - .build() - .start(); + .direct(directIO) + .build() + .start(); recoverAndReset(wal1); wal1.shutdownGracefully(); // clear the WAL header WALChannel walChannel = WALChannel.builder(path) - .capacity(capacity) - .direct(directIO) - .build(); + .capacity(capacity) + .direct(directIO) + .build(); walChannel.open(); walChannel.writeAndFlush(Unpooled.buffer(WAL_HEADER_TOTAL_CAPACITY).writeZero(WAL_HEADER_TOTAL_CAPACITY), 0); walChannel.close(); // try to open it in recovery mode WriteAheadLog wal2 = BlockWALService.recoveryBuilder(path) - .direct(directIO) - .build(); + .direct(directIO) + .build(); assertThrows(WALNotInitializedException.class, wal2::start); } - /** - * Call {@link WriteAheadLog#recover()} {@link WriteAheadLog#reset()} and drop all records. - */ - private static void recoverAndReset(WriteAheadLog wal) { - for (Iterator it = wal.recover(); it.hasNext(); ) { - it.next().record().release(); + private static class RecoverFromDisasterParam { + int recordSize; + long capacity; + // WAL header + long trimOffset; + long maxLength; + // WAL records + List writeOffsets; + List recoveredOffsets; + + public RecoverFromDisasterParam( + int recordSize, + long capacity, + long trimOffset, + long maxLength, + List writeOffsets, + List recoveredOffsets, + int unit + ) { + this.recordSize = recordSize; + this.capacity = capacity * unit + WAL_HEADER_TOTAL_CAPACITY; + this.trimOffset = trimOffset * unit; + this.maxLength = maxLength * unit; + this.writeOffsets = writeOffsets.stream().map(offset -> offset * unit).collect(Collectors.toList()); + this.recoveredOffsets = recoveredOffsets.stream().map(offset -> offset * unit).collect(Collectors.toList()); } - wal.reset().join(); - } - /** - * Write "0"s to the block device to reset it. - */ - private static void resetBlockDevice(String path, long capacity) throws IOException { - WALChannel channel = WALChannel.builder(path) - .capacity(capacity) - .direct(true) - .build(); - channel.open(); - ByteBuf buf = Unpooled.buffer((int) capacity); - buf.writeZero((int) capacity); - channel.write(buf, 0); - channel.close(); + public Arguments toArguments(String name) { + return Arguments.of(name, recordSize, capacity, trimOffset, maxLength, writeOffsets, recoveredOffsets); + } } } diff --git a/s3stream/src/test/java/com/automq/stream/s3/wal/util/WALBlockDeviceChannelTest.java b/s3stream/src/test/java/com/automq/stream/s3/wal/util/WALBlockDeviceChannelTest.java index c50a304ab..750cc65ed 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/wal/util/WALBlockDeviceChannelTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/wal/util/WALBlockDeviceChannelTest.java @@ -23,16 +23,15 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.CompositeByteBuf; import io.netty.buffer.Unpooled; -import org.junit.jupiter.api.Tag; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.condition.EnabledOnOs; -import org.junit.jupiter.api.condition.OS; - import java.io.IOException; import java.util.Optional; import java.util.concurrent.ExecutorService; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.condition.EnabledOnOs; +import org.junit.jupiter.api.condition.OS; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -100,7 +99,7 @@ public void testMultiThreadWrite() throws IOException, InterruptedException { channel.open(); ExecutorService executor = Threads.newFixedThreadPool(threads, - ThreadUtils.createThreadFactory("test-block-device-channel-write-%d", false), null); + ThreadUtils.createThreadFactory("test-block-device-channel-write-%d", false), null); for (int i = 0; i < count; i++) { final int index = i; executor.submit(() -> { diff --git a/s3stream/src/test/java/com/automq/stream/s3/wal/util/WALChannelTest.java b/s3stream/src/test/java/com/automq/stream/s3/wal/util/WALChannelTest.java index 3b510c00f..f4d32d86b 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/wal/util/WALChannelTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/wal/util/WALChannelTest.java @@ -20,14 +20,13 @@ import com.automq.stream.s3.TestUtils; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; +import java.io.IOException; +import java.nio.ByteBuffer; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; -import java.io.IOException; -import java.nio.ByteBuffer; - @Tag("S3Unit") class WALChannelTest { WALChannel walChannel; diff --git a/s3stream/src/test/java/com/automq/stream/utils/FutureTickerTest.java b/s3stream/src/test/java/com/automq/stream/utils/FutureTickerTest.java index 3daccc152..a3dffc777 100644 --- a/s3stream/src/test/java/com/automq/stream/utils/FutureTickerTest.java +++ b/s3stream/src/test/java/com/automq/stream/utils/FutureTickerTest.java @@ -17,12 +17,11 @@ package com.automq.stream.utils; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; diff --git a/s3stream/src/test/resources/log4j.properties b/s3stream/src/test/resources/log4j.properties index a2bfd2006..655a111b8 100644 --- a/s3stream/src/test/resources/log4j.properties +++ b/s3stream/src/test/resources/log4j.properties @@ -13,9 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. log4j.rootLogger=OFF, stdout - log4j.appender.stdout=org.apache.log4j.ConsoleAppender log4j.appender.stdout.layout=org.apache.log4j.PatternLayout log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n - log4j.logger.com.automq=WARN