From 8dd229132951891b409b458f4b5196d6bb3d8a70 Mon Sep 17 00:00:00 2001 From: Shichao Nie Date: Thu, 7 Sep 2023 22:19:08 +0800 Subject: [PATCH 1/7] feat: Implements major compact Signed-off-by: Shichao Nie --- .../kafka/log/s3/ControllerKVClient.java | 102 ++--- .../scala/kafka/log/s3/DefaultS3Client.java | 4 + .../kafka/log/s3/StreamObjectCopyer.java | 7 +- .../kafka/log/s3/compact/CompactResult.java | 25 ++ .../log/s3/compact/CompactionAnalyzer.java | 243 ++++++++++++ .../log/s3/compact/CompactionManager.java | 157 ++++++++ .../kafka/log/s3/compact/CompactionPlan.java | 43 +++ .../log/s3/compact/CompactionUploader.java | 163 ++++++++ .../log/s3/compact/TokenBucketThrottle.java | 70 ++++ .../s3/compact/objects/CompactedObject.java | 44 +++ .../objects/CompactedObjectBuilder.java | 133 +++++++ .../s3/compact/objects/CompactionType.java | 23 ++ .../s3/compact/objects/StreamDataBlock.java | 130 +++++++ .../s3/compact/operator/DataBlockReader.java | 179 +++++++++ .../s3/compact/operator/DataBlockWriter.java | 163 ++++++++ .../exceptions/IndexBlockParseException.java | 30 ++ .../log/s3/memory/MemoryMetadataManager.java | 49 +-- .../s3/metadata/InRangeObjectsFetcher.java | 9 +- .../s3/metadata/StreamMetadataManager.java | 74 ++-- .../s3/network/ControllerRequestSender.java | 7 +- .../s3/objects/ControllerObjectManager.java | 19 +- .../kafka/log/s3/objects/ObjectManager.java | 9 +- .../log/s3/objects/ObjectStreamRange.java | 10 +- .../kafka/log/s3/objects/StreamObject.java | 13 +- .../log/s3/operator/DefaultS3Operator.java | 3 +- .../log/s3/operator/MemoryS3Operator.java | 5 +- .../scala/kafka/log/s3/operator/Writer.java | 3 +- .../kafka/log/s3/streams/StreamManager.java | 11 +- .../scala/kafka/log/s3/wal/WriteAheadLog.java | 3 + .../main/scala/kafka/server/KafkaConfig.scala | 33 ++ .../log/s3/StreamMetadataManagerTest.java | 33 +- .../s3/compact/CompactionAnalyzerTest.java | 361 ++++++++++++++++++ .../log/s3/compact/CompactionManagerTest.java | 46 +++ .../kafka/image/S3StreamsMetadataImage.java | 9 + .../metadata/stream/S3WALObjectMetadata.java | 36 ++ 35 files changed, 2100 insertions(+), 149 deletions(-) create mode 100644 core/src/main/scala/kafka/log/s3/compact/CompactResult.java create mode 100644 core/src/main/scala/kafka/log/s3/compact/CompactionAnalyzer.java create mode 100644 core/src/main/scala/kafka/log/s3/compact/CompactionManager.java create mode 100644 core/src/main/scala/kafka/log/s3/compact/CompactionPlan.java create mode 100644 core/src/main/scala/kafka/log/s3/compact/CompactionUploader.java create mode 100644 core/src/main/scala/kafka/log/s3/compact/TokenBucketThrottle.java create mode 100644 core/src/main/scala/kafka/log/s3/compact/objects/CompactedObject.java create mode 100644 core/src/main/scala/kafka/log/s3/compact/objects/CompactedObjectBuilder.java create mode 100644 core/src/main/scala/kafka/log/s3/compact/objects/CompactionType.java create mode 100644 core/src/main/scala/kafka/log/s3/compact/objects/StreamDataBlock.java create mode 100644 core/src/main/scala/kafka/log/s3/compact/operator/DataBlockReader.java create mode 100644 core/src/main/scala/kafka/log/s3/compact/operator/DataBlockWriter.java create mode 100644 core/src/main/scala/kafka/log/s3/exceptions/IndexBlockParseException.java create mode 100644 core/src/test/java/kafka/log/s3/compact/CompactionAnalyzerTest.java create mode 100644 core/src/test/java/kafka/log/s3/compact/CompactionManagerTest.java create mode 100644 metadata/src/main/java/org/apache/kafka/metadata/stream/S3WALObjectMetadata.java diff --git a/core/src/main/scala/kafka/log/s3/ControllerKVClient.java b/core/src/main/scala/kafka/log/s3/ControllerKVClient.java index 7270ac0f8b..c3b59b7a28 100644 --- a/core/src/main/scala/kafka/log/s3/ControllerKVClient.java +++ b/core/src/main/scala/kafka/log/s3/ControllerKVClient.java @@ -19,11 +19,6 @@ import com.automq.elasticstream.client.api.KVClient; import com.automq.elasticstream.client.api.KeyValue; - -import java.nio.ByteBuffer; -import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.stream.Collectors; import kafka.log.s3.network.ControllerRequestSender; import org.apache.kafka.common.message.DeleteKVRequestData; import org.apache.kafka.common.message.GetKVRequestData; @@ -38,6 +33,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; + public class ControllerKVClient implements KVClient { private static final Logger LOGGER = LoggerFactory.getLogger(ControllerKVClient.class); @@ -51,69 +51,69 @@ public ControllerKVClient(ControllerRequestSender requestSender) { public CompletableFuture putKV(List list) { LOGGER.trace("[ControllerKVClient]: Put KV: {}", list); PutKVRequest.Builder requestBuilder = new Builder( - new PutKVRequestData() - .setKeyValues(list.stream().map(kv -> new PutKVRequestData.KeyValue() - .setKey(kv.key()) - .setValue(kv.value().array()) - ).collect(Collectors.toList())) + new PutKVRequestData() + .setKeyValues(list.stream().map(kv -> new PutKVRequestData.KeyValue() + .setKey(kv.key()) + .setValue(kv.value().array()) + ).collect(Collectors.toList())) ); return this.requestSender.send(requestBuilder, PutKVResponseData.class) - .thenApply(resp -> { - Errors code = Errors.forCode(resp.errorCode()); - switch (code) { - case NONE: - LOGGER.trace("[ControllerKVClient]: Put KV: {}, result: {}", list, resp); - return null; - default: - LOGGER.error("[ControllerKVClient]: Failed to Put KV: {}, code: {}", list, code); - throw code.exception(); - } - }); + .thenApply(resp -> { + Errors code = Errors.forCode(resp.errorCode()); + switch (code) { + case NONE: + LOGGER.trace("[ControllerKVClient]: Put KV: {}, result: {}", list, resp); + return null; + default: + LOGGER.error("[ControllerKVClient]: Failed to Put KV: {}, code: {}", list, code); + throw code.exception(); + } + }); } @Override public CompletableFuture> getKV(List list) { LOGGER.trace("[ControllerKVClient]: Get KV: {}", list); GetKVRequest.Builder requestBuilder = new GetKVRequest.Builder( - new GetKVRequestData() - .setKeys(list) + new GetKVRequestData() + .setKeys(list) ); return this.requestSender.send(requestBuilder, GetKVResponseData.class) - .thenApply(resp -> { - Errors code = Errors.forCode(resp.errorCode()); - switch (code) { - case NONE: - List keyValues = resp.keyValues() - .stream() - .map(kv -> KeyValue.of(kv.key(), kv.value() != null ? ByteBuffer.wrap(kv.value()) : null)) - .collect(Collectors.toList()); - LOGGER.trace("[ControllerKVClient]: Get KV: {}, result: {}", list, keyValues); - return keyValues; - default: - LOGGER.error("[ControllerKVClient]: Failed to Get KV: {}, code: {}", String.join(",", list), code); - throw code.exception(); - } - }); + .thenApply(resp -> { + Errors code = Errors.forCode(resp.errorCode()); + switch (code) { + case NONE: + List keyValues = resp.keyValues() + .stream() + .map(kv -> KeyValue.of(kv.key(), kv.value() != null ? ByteBuffer.wrap(kv.value()) : null)) + .collect(Collectors.toList()); + LOGGER.trace("[ControllerKVClient]: Get KV: {}, result: {}", list, keyValues); + return keyValues; + default: + LOGGER.error("[ControllerKVClient]: Failed to Get KV: {}, code: {}", String.join(",", list), code); + throw code.exception(); + } + }); } @Override public CompletableFuture delKV(List list) { LOGGER.trace("[ControllerKVClient]: Delete KV: {}", String.join(",", list)); DeleteKVRequest.Builder requestBuilder = new DeleteKVRequest.Builder( - new DeleteKVRequestData() - .setKeys(list) + new DeleteKVRequestData() + .setKeys(list) ); return this.requestSender.send(requestBuilder, PutKVResponseData.class) - .thenApply(resp -> { - Errors code = Errors.forCode(resp.errorCode()); - switch (code) { - case NONE: - LOGGER.trace("[ControllerKVClient]: Delete KV: {}, result: {}", list, resp); - return null; - default: - LOGGER.error("[ControllerKVClient]: Failed to Delete KV: {}, code: {}", String.join(",", list), code); - throw code.exception(); - } - }); + .thenApply(resp -> { + Errors code = Errors.forCode(resp.errorCode()); + switch (code) { + case NONE: + LOGGER.trace("[ControllerKVClient]: Delete KV: {}, result: {}", list, resp); + return null; + default: + LOGGER.error("[ControllerKVClient]: Failed to Delete KV: {}, code: {}", String.join(",", list), code); + throw code.exception(); + } + }); } } diff --git a/core/src/main/scala/kafka/log/s3/DefaultS3Client.java b/core/src/main/scala/kafka/log/s3/DefaultS3Client.java index 97bcc6c3fe..f3b691e3d8 100644 --- a/core/src/main/scala/kafka/log/s3/DefaultS3Client.java +++ b/core/src/main/scala/kafka/log/s3/DefaultS3Client.java @@ -22,6 +22,7 @@ import com.automq.elasticstream.client.api.StreamClient; import kafka.log.s3.cache.DefaultS3BlockCache; import kafka.log.s3.cache.S3BlockCache; +import kafka.log.s3.compact.CompactionManager; import kafka.log.s3.metadata.StreamMetadataManager; import kafka.log.s3.network.ControllerRequestSender; import kafka.log.s3.objects.ControllerObjectManager; @@ -50,6 +51,8 @@ public class DefaultS3Client implements Client { private final StreamManager streamManager; + private final CompactionManager compactionManager; + private final StreamClient streamClient; private final KVClient kvClient; @@ -62,6 +65,7 @@ public DefaultS3Client(BrokerServer brokerServer, KafkaConfig config, S3Operator this.streamManager = new ControllerStreamManager(this.requestSender, config); this.objectManager = new ControllerObjectManager(this.requestSender, this.metadataManager, this.config); this.blockCache = new DefaultS3BlockCache(config.s3CacheSize(), objectManager, operator); + this.compactionManager = new CompactionManager(this.config, this.objectManager, this.metadataManager, this.operator); this.storage = new S3Storage(config, new MemoryWriteAheadLog(), objectManager, blockCache, operator); this.streamClient = new S3StreamClient(this.streamManager, this.storage); this.kvClient = new ControllerKVClient(this.requestSender); diff --git a/core/src/main/scala/kafka/log/s3/StreamObjectCopyer.java b/core/src/main/scala/kafka/log/s3/StreamObjectCopyer.java index e4a423ecc4..262274c961 100644 --- a/core/src/main/scala/kafka/log/s3/StreamObjectCopyer.java +++ b/core/src/main/scala/kafka/log/s3/StreamObjectCopyer.java @@ -20,15 +20,16 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.CompositeByteBuf; import io.netty.buffer.Unpooled; -import java.util.LinkedList; -import java.util.List; -import java.util.concurrent.CompletableFuture; import kafka.log.s3.operator.S3Operator; import kafka.log.s3.operator.Writer; import org.apache.kafka.metadata.stream.ObjectUtils; import org.apache.kafka.metadata.stream.S3ObjectMetadata; import org.apache.kafka.metadata.stream.S3ObjectType; +import java.util.LinkedList; +import java.util.List; +import java.util.concurrent.CompletableFuture; + public class StreamObjectCopyer { private final List completedObjects; private final S3Operator s3Operator; diff --git a/core/src/main/scala/kafka/log/s3/compact/CompactResult.java b/core/src/main/scala/kafka/log/s3/compact/CompactResult.java new file mode 100644 index 0000000000..72d58de9e3 --- /dev/null +++ b/core/src/main/scala/kafka/log/s3/compact/CompactResult.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.s3.compact; + +public enum CompactResult { + SUCCESS, + SKIPPED, + FAILED + +} diff --git a/core/src/main/scala/kafka/log/s3/compact/CompactionAnalyzer.java b/core/src/main/scala/kafka/log/s3/compact/CompactionAnalyzer.java new file mode 100644 index 0000000000..4fe2439fb3 --- /dev/null +++ b/core/src/main/scala/kafka/log/s3/compact/CompactionAnalyzer.java @@ -0,0 +1,243 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.s3.compact; + +import kafka.log.s3.compact.objects.CompactedObject; +import kafka.log.s3.compact.objects.CompactedObjectBuilder; +import kafka.log.s3.compact.objects.CompactionType; +import kafka.log.s3.compact.objects.StreamDataBlock; +import kafka.log.s3.compact.operator.DataBlockReader; +import kafka.log.s3.operator.S3Operator; +import org.apache.kafka.metadata.stream.S3WALObject; +import org.apache.kafka.metadata.stream.S3WALObjectMetadata; +import org.slf4j.Logger; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; + +public class CompactionAnalyzer { + private final static Logger LOGGER = org.slf4j.LoggerFactory.getLogger(CompactionAnalyzer.class); + private final long compactionCacheSize; + private final double executionScoreThreshold; + private final long streamSplitSize; + private final S3Operator s3Operator; + + public CompactionAnalyzer(long compactionCacheSize, double executionScoreThreshold, long streamSplitSize, S3Operator s3Operator) { + this.compactionCacheSize = compactionCacheSize; + this.executionScoreThreshold = executionScoreThreshold; + this.streamSplitSize = streamSplitSize; + this.s3Operator = s3Operator; + } + + public List analyze(List objectMetadataList) { + List compactionPlans = new ArrayList<>(); + try { + List compactedObjectBuilders = buildCompactedObjects(objectMetadataList); + List compactedObjects = new ArrayList<>(); + CompactedObjectBuilder compactedWALObjectBuilder = null; + long totalSize = 0L; + for (int i = 0; i < compactedObjectBuilders.size(); ) { + CompactedObjectBuilder compactedObjectBuilder = compactedObjectBuilders.get(i); + if (totalSize + compactedObjectBuilder.totalBlockSize() > compactionCacheSize) { + if (shouldSplitObject(compactedObjectBuilder)) { + // split object to fit into cache + int endOffset = 0; + for (int j = 0; j < compactedObjectBuilder.streamDataBlocks().size(); j++) { + if (totalSize + compactedObjectBuilder.streamDataBlocks().get(j).getBlockSize() > compactionCacheSize) { + endOffset = j; + break; + } + } + if (endOffset != 0) { + CompactedObjectBuilder builder = compactedObjectBuilder.split(0, endOffset); + compactedWALObjectBuilder = addOrMergeCompactedObject(builder, compactedObjects, compactedWALObjectBuilder); + } + } + compactionPlans.add(generateCompactionPlan(compactedObjects, compactedWALObjectBuilder)); + compactedObjects.clear(); + compactedWALObjectBuilder = null; + totalSize = 0; + } else { + // object fits into cache size + compactedWALObjectBuilder = addOrMergeCompactedObject(compactedObjectBuilder, compactedObjects, compactedWALObjectBuilder); + totalSize += compactedObjectBuilder.totalBlockSize(); + i++; + } + + } + if (!compactedObjects.isEmpty()) { + compactionPlans.add(generateCompactionPlan(compactedObjects, compactedWALObjectBuilder)); + } + return compactionPlans; + } catch (Exception e) { + LOGGER.error("Error while analyzing compaction plan", e); + } + return compactionPlans; + } + + private CompactedObjectBuilder addOrMergeCompactedObject(CompactedObjectBuilder compactedObjectBuilder, + List compactedObjects, + CompactedObjectBuilder compactedWALObjectBuilder) { + if (compactedObjectBuilder.type() == CompactionType.SPLIT) { + compactedObjects.add(compactedObjectBuilder.build()); + } else { + if (compactedWALObjectBuilder == null) { + compactedWALObjectBuilder = new CompactedObjectBuilder(); + } + compactedWALObjectBuilder.merge(compactedObjectBuilder); + } + return compactedWALObjectBuilder; + } + + private boolean shouldSplitObject(CompactedObjectBuilder compactedObjectBuilder) { + //TODO: split object depends on available cache size and current object size + //TODO: use multipart upload to upload split stream object + return true; + } + + private CompactionPlan generateCompactionPlan(List compactedObjects, CompactedObjectBuilder compactedWALObject) { + if (compactedWALObject != null) { + compactedObjects.add(compactedWALObject.build()); + } + Map> streamDataBlockMap = new HashMap<>(); + for (CompactedObject compactedObject : compactedObjects) { + for (StreamDataBlock streamDataBlock : compactedObject.streamDataBlocks()) { + streamDataBlockMap.computeIfAbsent(streamDataBlock.getObjectId(), k -> new ArrayList<>()).add(streamDataBlock); + } + } + for (List dataBlocks : streamDataBlockMap.values()) { + dataBlocks.sort(StreamDataBlock.BLOCK_POSITION_COMPARATOR); + } + + return new CompactionPlan(new ArrayList<>(compactedObjects), streamDataBlockMap); + } + + public List buildCompactedObjects(List objects) { + List streamDataBlocks = blockWaitObjectIndices(objects); + if (!shouldCompact(streamDataBlocks)) { + return new ArrayList<>(); + } + return compactObjects(sortStreamRangePositions(streamDataBlocks)); + } + + List blockWaitObjectIndices(List objectMetadataList) { + Map s3WALObjectMap = objectMetadataList.stream() + .collect(Collectors.toMap(e -> e.getWalObject().objectId(), S3WALObjectMetadata::getWalObject)); + List>> objectStreamRangePositionFutures = new ArrayList<>(); + for (S3WALObjectMetadata walObjectMetadata : objectMetadataList) { + DataBlockReader dataBlockReader = new DataBlockReader(walObjectMetadata.getObjectMetadata(), s3Operator); + dataBlockReader.parseDataBlockIndex(); + objectStreamRangePositionFutures.add(dataBlockReader.getDataBlockIndex()); + } + return objectStreamRangePositionFutures.stream().flatMap(f -> { + try { + List streamDataBlocks = f.join(); + List validStreamDataBlocks = new ArrayList<>(); + S3WALObject s3WALObject = s3WALObjectMap.get(streamDataBlocks.get(0).getObjectId()); + // filter out invalid stream data blocks in case metadata is inconsistent with S3 index block + for (StreamDataBlock streamDataBlock: streamDataBlocks) { + if (s3WALObject.intersect(streamDataBlock.getStreamId(), streamDataBlock.getStartOffset(), streamDataBlock.getEndOffset())) { + validStreamDataBlocks.add(streamDataBlock); + } + } + return validStreamDataBlocks.stream(); + } catch (Exception ex) { + // continue compaction without invalid object + LOGGER.error("Read on invalid object ", ex); + return null; + } + }).collect(Collectors.toList()); + } + + private List compactObjects(List streamDataBlocks) { + List compactedObjectBuilders = new ArrayList<>(); + CompactedObjectBuilder builder = new CompactedObjectBuilder(); + for (StreamDataBlock streamDataBlock : streamDataBlocks) { + if (builder.lastStreamId() == -1L) { + // init state + builder.addStreamDataBlock(streamDataBlock); + } else if (builder.lastStreamId() == streamDataBlock.getStreamId()) { + // data range from same stream + if (streamDataBlock.getStartOffset() > builder.lastOffset()) { + // data range is not continuous, split current object as StreamObject + builder = splitObject(builder, compactedObjectBuilders); + builder.addStreamDataBlock(streamDataBlock); + } else if (streamDataBlock.getStartOffset() == builder.lastOffset()) { + builder.addStreamDataBlock(streamDataBlock); + } else { + // should not go there + LOGGER.error("FATAL ERROR: illegal stream range position, last offset: {}, curr: {}", + builder.lastOffset(), streamDataBlock); + return new ArrayList<>(); + } + } else { + builder = splitAndAddBlock(builder, streamDataBlock, compactedObjectBuilders); + } + } + compactedObjectBuilders.add(builder); + return compactedObjectBuilders; + } + + boolean shouldCompact(List streamDataBlocks) { + // do compact if there is any stream with data placed in more than one WAL objects + Map streamIdToDistinctObjectMap = streamDataBlocks.stream() + .collect(Collectors.groupingBy(StreamDataBlock::getStreamId, Collectors.mapping(StreamDataBlock::getObjectId, Collectors.toSet()))) + .entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().size())); + return streamIdToDistinctObjectMap.values().stream().filter(v -> v > 1).count() > 1; + } + + private CompactedObjectBuilder splitAndAddBlock(CompactedObjectBuilder builder, + StreamDataBlock streamDataBlock, + List compactedObjectBuilders) { + if (builder.currStreamBlockSize() > streamSplitSize) { + builder = splitObject(builder, compactedObjectBuilders); + } + builder.addStreamDataBlock(streamDataBlock); + return builder; + } + + private CompactedObjectBuilder splitObject(CompactedObjectBuilder builder, + List compactedObjectBuilders) { + CompactedObjectBuilder splitBuilder = builder.splitCurrentStream(); + splitBuilder.setType(CompactionType.SPLIT); + if (builder.totalBlockSize() != 0) { + compactedObjectBuilders.add(builder); + } + compactedObjectBuilders.add(splitBuilder); + builder = new CompactedObjectBuilder(); + return builder; + } + + List sortStreamRangePositions(List streamDataBlocks) { + //TODO: use merge sort + Map> sortedStreamObjectMap = new TreeMap<>(); + for (StreamDataBlock streamDataBlock : streamDataBlocks) { + sortedStreamObjectMap.computeIfAbsent(streamDataBlock.getStreamId(), k -> new ArrayList<>()).add(streamDataBlock); + } + return sortedStreamObjectMap.values().stream().flatMap(list -> { + list.sort(StreamDataBlock.STREAM_OFFSET_COMPARATOR); + return list.stream(); + }).collect(Collectors.toList()); + } + +} diff --git a/core/src/main/scala/kafka/log/s3/compact/CompactionManager.java b/core/src/main/scala/kafka/log/s3/compact/CompactionManager.java new file mode 100644 index 0000000000..6f5144608e --- /dev/null +++ b/core/src/main/scala/kafka/log/s3/compact/CompactionManager.java @@ -0,0 +1,157 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.log.s3.compact; + +import io.netty.util.concurrent.DefaultThreadFactory; +import kafka.log.s3.compact.objects.CompactedObject; +import kafka.log.s3.compact.objects.CompactionType; +import kafka.log.s3.compact.objects.StreamDataBlock; +import kafka.log.s3.compact.operator.DataBlockReader; +import kafka.log.s3.metadata.StreamMetadataManager; +import kafka.log.s3.objects.CommitWALObjectRequest; +import kafka.log.s3.objects.ObjectManager; +import kafka.log.s3.objects.ObjectStreamRange; +import kafka.log.s3.objects.StreamObject; +import kafka.log.s3.operator.S3Operator; +import kafka.server.KafkaConfig; +import org.apache.kafka.metadata.stream.S3ObjectMetadata; +import org.apache.kafka.metadata.stream.S3WALObjectMetadata; +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.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +public class CompactionManager { + private final static Logger LOGGER = LoggerFactory.getLogger(CompactionManager.class); + private final ObjectManager objectManager; + private final StreamMetadataManager streamMetadataManager; + private final S3Operator s3Operator; + private final CompactionAnalyzer compactionAnalyzer; + private final ScheduledExecutorService executorService; + private final CompactionUploader uploader; + private final long compactionCacheSize; + private final double executionScoreThreshold; + private final long streamSplitSize; + private final TokenBucketThrottle networkInThrottle; + + public CompactionManager(KafkaConfig config, ObjectManager objectManager, StreamMetadataManager streamMetadataManager, S3Operator s3Operator) { + this.objectManager = objectManager; + this.streamMetadataManager = streamMetadataManager; + this.s3Operator = s3Operator; + this.compactionCacheSize = config.s3ObjectCompactionCacheSize(); + this.executionScoreThreshold = config.s3ObjectCompactionExecutionScoreThreshold(); + this.streamSplitSize = config.s3ObjectCompactionStreamSplitSize(); + this.networkInThrottle = new TokenBucketThrottle(config.s3ObjectCompactionNWInBandwidth()); + this.uploader = new CompactionUploader(objectManager, s3Operator, config); + this.compactionAnalyzer = new CompactionAnalyzer(compactionCacheSize, executionScoreThreshold, streamSplitSize, s3Operator); + this.executorService = Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("object-compaction-manager")); + this.executorService.scheduleWithFixedDelay(this::compact, 0, 600, TimeUnit.SECONDS); + } + + public void shutdown() { + this.executorService.shutdown(); + this.networkInThrottle.stop(); + this.uploader.stop(); + } + + public CompletableFuture compact() { + List s3ObjectMetadata = this.streamMetadataManager.getWALObjects(); + try { + Map s3ObjectMetadataMap = s3ObjectMetadata.stream() + .collect(Collectors.toMap(e -> e.getWalObject().objectId(), e -> e)); + List compactionPlans = this.compactionAnalyzer.analyze(s3ObjectMetadata); + if (compactionPlans.isEmpty()) { + return CompletableFuture.completedFuture(CompactResult.SKIPPED); + } + CommitWALObjectRequest request = new CommitWALObjectRequest(); + for (CompactionPlan compactionPlan : compactionPlans) { + List> streamObjectFutureList = new ArrayList<>(); + List>> walObjStreamRangeFutureList = new ArrayList<>(); + // iterate over each compaction plan + for (Map.Entry> streamDataBlocEntry : compactionPlan.streamDataBlocksMap().entrySet()) { + S3ObjectMetadata metadata = s3ObjectMetadataMap.get(streamDataBlocEntry.getKey()).getObjectMetadata(); + List streamDataBlocks = streamDataBlocEntry.getValue(); + List blockIndices = buildBlockIndicesFromStreamDataBlock(streamDataBlocks); + networkInThrottle.throttle(streamDataBlocks.stream().mapToLong(StreamDataBlock::getBlockSize).sum()); + DataBlockReader reader = new DataBlockReader(metadata, s3Operator); + reader.readBlocks(blockIndices).thenAccept(dataBlocks -> { + for (int i = 0; i < blockIndices.size(); i++) { + StreamDataBlock streamDataBlock = streamDataBlocks.get(i); + streamDataBlock.getDataCf().complete(dataBlocks.get(i).buffer()); + } + }).exceptionally(ex -> { + LOGGER.error("read on invalid object {}, ex ", metadata.key(), ex); + for (int i = 0; i < blockIndices.size(); i++) { + StreamDataBlock streamDataBlock = streamDataBlocks.get(i); + streamDataBlock.getDataCf().completeExceptionally(ex); + } + return null; + }); + } + for (CompactedObject compactedObject : compactionPlan.compactedObjects()) { + if (compactedObject.type() == CompactionType.COMPACT) { + walObjStreamRangeFutureList.add(uploader.writeWALObject(compactedObject)); + } else { + streamObjectFutureList.add(uploader.writeStreamObject(compactedObject)); + } + } + // wait for all stream objects and wal object parts to be uploaded + try { + walObjStreamRangeFutureList.stream().map(CompletableFuture::join).forEach(e -> e.forEach(request::addStreamRange)); + streamObjectFutureList.stream().map(CompletableFuture::join).forEach(request::addStreamObject); + } catch (Exception ex) { + LOGGER.error("Error while uploading compaction objects", ex); + uploader.reset(); + return CompletableFuture.failedFuture(ex); + } + compactionPlan.streamDataBlocksMap().values().forEach(e -> e.forEach(StreamDataBlock::free)); + } + request.setObjectId(uploader.getWALObjectId()); + // set wal object id to be the first object id of compacted objects + request.setOrderId(s3ObjectMetadata.get(0).getObjectMetadata().getObjectId()); + request.setCompactedObjectIds(s3ObjectMetadata.stream().map(s -> s.getObjectMetadata().getObjectId()).collect(Collectors.toList())); + uploader.getWalObjectWriter().close().thenAccept(nil -> request.setObjectSize(uploader.getWalObjectWriter().size())).join(); + uploader.reset(); + return objectManager.commitWALObject(request).thenApply(nil -> { + LOGGER.info("Compaction success, WAL object id: {}, size: {}, stream object num: {}", + request.getObjectId(), request.getObjectSize(), request.getStreamObjects().size()); + return CompactResult.SUCCESS; + }); + } catch (Exception e) { + LOGGER.error("Error while analyzing compaction objects", e); + return CompletableFuture.failedFuture(e); + } + + } + + private List buildBlockIndicesFromStreamDataBlock(List streamDataBlocks) { + List blockIndices = new ArrayList<>(); + for (StreamDataBlock streamDataBlock : streamDataBlocks) { + blockIndices.add(new DataBlockReader.DataBlockIndex(streamDataBlock.getBlockId(), streamDataBlock.getBlockPosition(), + streamDataBlock.getBlockSize(), streamDataBlock.getRecordCount())); + } + return blockIndices; + } + +} diff --git a/core/src/main/scala/kafka/log/s3/compact/CompactionPlan.java b/core/src/main/scala/kafka/log/s3/compact/CompactionPlan.java new file mode 100644 index 0000000000..38ad0763a5 --- /dev/null +++ b/core/src/main/scala/kafka/log/s3/compact/CompactionPlan.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.s3.compact; + +import kafka.log.s3.compact.objects.CompactedObject; +import kafka.log.s3.compact.objects.StreamDataBlock; + +import java.util.List; +import java.util.Map; + +public class CompactionPlan { + private final List compactedObjects; + private final Map> streamDataBlocksMap; + + public CompactionPlan(List compactedObjects, + Map> streamDataBlocksMap) { + this.compactedObjects = compactedObjects; + this.streamDataBlocksMap = streamDataBlocksMap; + } + + public List compactedObjects() { + return compactedObjects; + } + + public Map> streamDataBlocksMap() { + return streamDataBlocksMap; + } +} diff --git a/core/src/main/scala/kafka/log/s3/compact/CompactionUploader.java b/core/src/main/scala/kafka/log/s3/compact/CompactionUploader.java new file mode 100644 index 0000000000..fde25f0fad --- /dev/null +++ b/core/src/main/scala/kafka/log/s3/compact/CompactionUploader.java @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.s3.compact; + +import kafka.log.s3.compact.objects.CompactedObject; +import kafka.log.s3.compact.objects.StreamDataBlock; +import kafka.log.s3.compact.operator.DataBlockWriter; +import kafka.log.s3.objects.ObjectManager; +import kafka.log.s3.objects.ObjectStreamRange; +import kafka.log.s3.objects.StreamObject; +import kafka.log.s3.operator.S3Operator; +import kafka.server.KafkaConfig; +import org.apache.kafka.common.utils.ThreadUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class CompactionUploader { + private final static Logger LOGGER = LoggerFactory.getLogger(CompactionUploader.class); + private final ObjectManager objectManager; + private final TokenBucketThrottle throttle; + private final ScheduledExecutorService executorService; + private final S3Operator s3Operator; + private final KafkaConfig kafkaConfig; + private CompletableFuture walObjectIdCf = null; + private DataBlockWriter walObjectWriter = null; + + // TODO: add network outbound throttle + public CompactionUploader(ObjectManager objectManager, S3Operator s3Operator, KafkaConfig kafkaConfig) { + this.objectManager = objectManager; + this.s3Operator = s3Operator; + this.kafkaConfig = kafkaConfig; + this.throttle = new TokenBucketThrottle(kafkaConfig.s3ObjectCompactionNWOutBandwidth()); + this.executorService = Executors.newScheduledThreadPool(kafkaConfig.s3ObjectCompactionUploadConcurrency(), + ThreadUtils.createThreadFactory("compaction-uploader", true)); + } + + public void stop() { + this.executorService.shutdown(); + this.throttle.stop(); + } + + public CompletableFuture> writeWALObject(CompactedObject compactedObject) { + CompletableFuture> cf = new CompletableFuture<>(); + CompletableFuture.allOf(compactedObject.streamDataBlocks() + .stream() + .map(StreamDataBlock::getDataCf) + .toArray(CompletableFuture[]::new)) + .thenAcceptAsync(v -> { + prepareObjectAndWrite(compactedObject, cf); + }, executorService) + .exceptionally(ex -> { + LOGGER.error("wal object write failed", ex); + cf.completeExceptionally(ex); + return null; + }); + return cf; + } + + private void prepareObjectAndWrite(CompactedObject compactedObject, CompletableFuture> cf) { + // no race condition, only one thread at a time will request for wal object id + if (walObjectIdCf == null) { + walObjectIdCf = this.objectManager.prepareObject(1, TimeUnit.MINUTES.toMillis(30)); + } + walObjectIdCf.thenAcceptAsync(objectId -> { + if (walObjectWriter == null) { + walObjectWriter = new DataBlockWriter(objectId, s3Operator, kafkaConfig.s3ObjectPartSize()); + } + ObjectStreamRange currObjectStreamRange = null; + List> writeFutureList = new ArrayList<>(); + List objectStreamRanges = new ArrayList<>(); + for (StreamDataBlock streamDataBlock : compactedObject.streamDataBlocks()) { + if (currObjectStreamRange == null) { + currObjectStreamRange = new ObjectStreamRange(streamDataBlock.getStreamId(), -1L, + streamDataBlock.getStartOffset(), streamDataBlock.getEndOffset()); + } else { + if (currObjectStreamRange.getStreamId() == streamDataBlock.getStreamId()) { + currObjectStreamRange.setEndOffset(streamDataBlock.getEndOffset()); + } else { + objectStreamRanges.add(currObjectStreamRange); + currObjectStreamRange = new ObjectStreamRange(streamDataBlock.getStreamId(), -1L, + streamDataBlock.getStartOffset(), streamDataBlock.getEndOffset()); + } + } + writeFutureList.add(walObjectWriter.write(streamDataBlock)); + } + objectStreamRanges.add(currObjectStreamRange); + CompletableFuture.allOf(writeFutureList.toArray(new CompletableFuture[0])).thenAccept(v -> cf.complete(objectStreamRanges)); + }, executorService).exceptionally(ex -> { + LOGGER.error("prepare wal object failed", ex); + prepareObjectAndWrite(compactedObject, cf); + return null; + }); + } + + public CompletableFuture writeStreamObject(CompactedObject compactedObject) { + return CompletableFuture.allOf(compactedObject.streamDataBlocks() + .stream() + .map(StreamDataBlock::getDataCf) + .toArray(CompletableFuture[]::new)) + .thenComposeAsync(v -> objectManager.prepareObject(1, TimeUnit.MINUTES.toMillis(30)) + .thenComposeAsync(objectId -> { + DataBlockWriter dataBlockWriter = new DataBlockWriter(objectId, s3Operator, kafkaConfig.s3ObjectPartSize()); + for (StreamDataBlock streamDataBlock : compactedObject.streamDataBlocks()) { + dataBlockWriter.write(streamDataBlock); + } + long streamId = compactedObject.streamDataBlocks().get(0).getStreamId(); + long startOffset = compactedObject.streamDataBlocks().get(0).getStartOffset(); + long endOffset = compactedObject.streamDataBlocks().get(compactedObject.streamDataBlocks().size() - 1).getEndOffset(); + StreamObject streamObject = new StreamObject(); + streamObject.setObjectId(objectId); + streamObject.setStreamId(streamId); + streamObject.setStartOffset(startOffset); + streamObject.setEndOffset(endOffset); + return dataBlockWriter.close().thenApply(nil -> { + streamObject.setObjectSize(dataBlockWriter.size()); + return streamObject; + }); + }, executorService), + executorService) + .exceptionally(ex -> { + LOGGER.error("stream object write failed", ex); + return null; + }); + } + + public DataBlockWriter getWalObjectWriter() { + return walObjectWriter; + } + + public void reset() { + walObjectIdCf = null; + walObjectWriter = null; + } + + public long getWALObjectId() { + if (walObjectIdCf == null) { + return -1; + } + return walObjectIdCf.getNow(-1L); + } +} diff --git a/core/src/main/scala/kafka/log/s3/compact/TokenBucketThrottle.java b/core/src/main/scala/kafka/log/s3/compact/TokenBucketThrottle.java new file mode 100644 index 0000000000..c445b91b28 --- /dev/null +++ b/core/src/main/scala/kafka/log/s3/compact/TokenBucketThrottle.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.s3.compact; + +import io.netty.util.concurrent.DefaultThreadFactory; + +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +public class TokenBucketThrottle { + private final Lock lock = new ReentrantLock(); + private final Condition condition = lock.newCondition(); + private final long tokenSize; + private long availableTokens; + + private final ScheduledExecutorService executorService; + + public TokenBucketThrottle(long tokenSize) { + this.tokenSize = tokenSize; + this.executorService = Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("token-bucket-throttle")); + this.executorService.scheduleAtFixedRate(() -> { + try { + lock.lock(); + availableTokens = tokenSize; + condition.signalAll(); + } finally { + lock.unlock(); + } + }, 0, 1, java.util.concurrent.TimeUnit.SECONDS); + } + + public void stop() { + this.executorService.shutdown(); + } + + public long getTokenSize() { + return tokenSize; + } + + public void throttle(long size) { + try { + lock.lock(); + while (availableTokens < size) { + condition.await(); + } + availableTokens -= size; + } catch (InterruptedException ignored) { + } finally { + lock.unlock(); + } + } +} diff --git a/core/src/main/scala/kafka/log/s3/compact/objects/CompactedObject.java b/core/src/main/scala/kafka/log/s3/compact/objects/CompactedObject.java new file mode 100644 index 0000000000..990ea5c3d6 --- /dev/null +++ b/core/src/main/scala/kafka/log/s3/compact/objects/CompactedObject.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.s3.compact.objects; + +import java.util.List; + +public class CompactedObject { + private final CompactionType type; + private final List streamDataBlocks; + private final long size; + + public CompactedObject(CompactionType type, List streamDataBlocks, long size) { + this.type = type; + this.streamDataBlocks = streamDataBlocks; + this.size = size; + } + + public CompactionType type() { + return type; + } + + public List streamDataBlocks() { + return this.streamDataBlocks; + } + + public long size() { + return size; + } +} diff --git a/core/src/main/scala/kafka/log/s3/compact/objects/CompactedObjectBuilder.java b/core/src/main/scala/kafka/log/s3/compact/objects/CompactedObjectBuilder.java new file mode 100644 index 0000000000..15588fce1e --- /dev/null +++ b/core/src/main/scala/kafka/log/s3/compact/objects/CompactedObjectBuilder.java @@ -0,0 +1,133 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.s3.compact.objects; + +import java.util.ArrayList; +import java.util.List; + +public class CompactedObjectBuilder { + private CompactionType type; + private final List streamDataBlocks; + private int currStreamIndexHead; + private int currStreamIndexTail; + + public CompactedObjectBuilder() { + this.type = CompactionType.COMPACT; + this.streamDataBlocks = new ArrayList<>(); + this.currStreamIndexHead = -1; + this.currStreamIndexTail = -1; + } + + public CompactedObjectBuilder splitCurrentStream() { + return split(currStreamIndexHead, currStreamIndexTail); + } + + public CompactedObjectBuilder split(int start, int end) { + if (start < 0 || end > currStreamIndexTail) { + // split out of range + return new CompactedObjectBuilder(); + } + CompactedObjectBuilder builder = new CompactedObjectBuilder(); + List streamRangePositionsSubList = streamDataBlocks.subList(start, end); + for (StreamDataBlock streamRangePosition : streamRangePositionsSubList) { + builder.addStreamDataBlock(streamRangePosition); + } + builder.setType(type); + streamRangePositionsSubList.clear(); + resetCurrStreamPosition(); + return builder; + } + + private void resetCurrStreamPosition() { + currStreamIndexHead = -1; + currStreamIndexTail = -1; + long currStreamId = -1; + for (int i = 0; i < streamDataBlocks.size(); i++) { + StreamDataBlock streamDataBlock = streamDataBlocks.get(i); + if (currStreamId != streamDataBlock.getStreamId()) { + currStreamId = streamDataBlock.getStreamId(); + currStreamIndexHead = i; + } + currStreamIndexTail = i + 1; + } + } + + private List getCurrentStreamRangePositions() { + if (currStreamIndexHead == -1 || currStreamIndexTail == -1) { + return new ArrayList<>(); + } + return streamDataBlocks.subList(currStreamIndexHead, currStreamIndexTail); + } + + public CompactedObjectBuilder setType(CompactionType type) { + this.type = type; + return this; + } + + public CompactionType type() { + return this.type; + } + + public long lastStreamId() { + if (streamDataBlocks.isEmpty()) { + return -1; + } + return streamDataBlocks.get(streamDataBlocks.size() - 1).getStreamId(); + } + + public long lastOffset() { + if (streamDataBlocks.isEmpty()) { + return -1; + } + return streamDataBlocks.get(streamDataBlocks.size() - 1).getEndOffset(); + } + + public CompactedObjectBuilder addStreamDataBlock(StreamDataBlock streamDataBlock) { + if (streamDataBlock.getStreamId() != lastStreamId()) { + this.currStreamIndexHead = this.streamDataBlocks.size(); + } + this.streamDataBlocks.add(streamDataBlock); + this.currStreamIndexTail = this.streamDataBlocks.size(); + return this; + } + + public List streamDataBlocks() { + return this.streamDataBlocks; + } + + public long currStreamBlockSize() { + return getCurrentStreamRangePositions().stream().mapToLong(StreamDataBlock::getBlockSize).sum(); + } + + public long totalBlockSize() { + return streamDataBlocks.stream().mapToLong(StreamDataBlock::getBlockSize).sum(); + } + + public void merge(CompactedObjectBuilder other) { + if (other.type == CompactionType.SPLIT) { + // cannot merge compacted object of split type as split strategy is determined when constructing compacted objects + return; + } + this.streamDataBlocks.addAll(other.streamDataBlocks); + resetCurrStreamPosition(); + } + + public CompactedObject build() { + return new CompactedObject(type, streamDataBlocks, totalBlockSize()); + } +} diff --git a/core/src/main/scala/kafka/log/s3/compact/objects/CompactionType.java b/core/src/main/scala/kafka/log/s3/compact/objects/CompactionType.java new file mode 100644 index 0000000000..a6312990cd --- /dev/null +++ b/core/src/main/scala/kafka/log/s3/compact/objects/CompactionType.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.s3.compact.objects; + +public enum CompactionType { + COMPACT, + SPLIT, +} diff --git a/core/src/main/scala/kafka/log/s3/compact/objects/StreamDataBlock.java b/core/src/main/scala/kafka/log/s3/compact/objects/StreamDataBlock.java new file mode 100644 index 0000000000..575b6ae7f4 --- /dev/null +++ b/core/src/main/scala/kafka/log/s3/compact/objects/StreamDataBlock.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.s3.compact.objects; + +import io.netty.buffer.ByteBuf; + +import java.util.Objects; +import java.util.Comparator; +import java.util.concurrent.CompletableFuture; + +public class StreamDataBlock { + public static final Comparator STREAM_OFFSET_COMPARATOR = Comparator.comparingLong(StreamDataBlock::getStartOffset); + public static final Comparator BLOCK_POSITION_COMPARATOR = Comparator.comparingLong(StreamDataBlock::getBlockPosition); + + // Stream attributes + private final long streamId; + private final long startOffset; + private final long endOffset; + private final int blockId; + + // Object attributes + private final long objectId; + private final long blockPosition; + private final int blockSize; + private final int recordCount; + private final CompletableFuture dataCf = new CompletableFuture<>(); + + public StreamDataBlock(long streamId, long startOffset, long endOffset, int blockId, + long objectId, long blockPosition, int blockSize, int recordCount) { + this.streamId = streamId; + this.startOffset = startOffset; + this.endOffset = endOffset; + this.blockId = blockId; + this.objectId = objectId; + this.blockPosition = blockPosition; + this.blockSize = blockSize; + this.recordCount = recordCount; + } + + public long getStreamId() { + return streamId; + } + + public long getStartOffset() { + return startOffset; + } + + public long getEndOffset() { + return endOffset; + } + + public long getStreamRangeSize() { + return endOffset - startOffset; + } + + public int getBlockId() { + return blockId; + } + + public long getObjectId() { + return objectId; + } + + public long getBlockPosition() { + return blockPosition; + } + + public int getBlockSize() { + return blockSize; + } + + public int getRecordCount() { + return recordCount; + } + + public CompletableFuture getDataCf() { + return this.dataCf; + } + + public void free() { + this.dataCf.thenAccept(buf -> { + if (buf != null) { + buf.release(); + } + }); + } + + @Override + public String toString() { + return "ObjectStreamRangePosition{" + + "streamId=" + streamId + + ", startOffset=" + startOffset + + ", endOffset=" + endOffset + + ", objectId=" + objectId + + ", blockPosition=" + blockPosition + + ", blockSize=" + blockSize + + '}'; + } + + @Override + public boolean equals(Object o) { + 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 + && blockId == that.blockId && objectId == that.objectId && blockPosition == that.blockPosition + && blockSize == that.blockSize && recordCount == that.recordCount; + } + + @Override + public int hashCode() { + return Objects.hash(streamId, startOffset, endOffset, blockId, objectId, blockPosition, blockSize, recordCount); + } + +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/log/s3/compact/operator/DataBlockReader.java b/core/src/main/scala/kafka/log/s3/compact/operator/DataBlockReader.java new file mode 100644 index 0000000000..c0aeb66c6e --- /dev/null +++ b/core/src/main/scala/kafka/log/s3/compact/operator/DataBlockReader.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.s3.compact.operator; + +import io.netty.buffer.ByteBuf; +import kafka.log.s3.ObjectReader; +import kafka.log.s3.compact.objects.StreamDataBlock; +import kafka.log.s3.operator.S3Operator; +import org.apache.kafka.metadata.stream.S3ObjectMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +//TODO: refactor to reduce duplicate code with ObjectWriter +public class DataBlockReader { + private static final Logger LOGGER = LoggerFactory.getLogger(ObjectReader.class); + private final S3ObjectMetadata metadata; + private final String objectKey; + private final S3Operator s3Operator; + private final CompletableFuture> indexBlockCf = new CompletableFuture<>(); + + public DataBlockReader(S3ObjectMetadata metadata, S3Operator s3Operator) { + this.metadata = metadata; + this.objectKey = metadata.key(); + this.s3Operator = s3Operator; + } + + public CompletableFuture> getDataBlockIndex() { + return indexBlockCf; + } + + public void parseDataBlockIndex() { + parseDataBlockIndex(Math.max(0, metadata.getObjectSize() - 1024 * 1024)); + } + + public void parseDataBlockIndex(long startPosition) { + s3Operator.rangeRead(objectKey, startPosition, metadata.getObjectSize()) + .thenAccept(buf -> { + try { + indexBlockCf.complete(IndexBlock.parse(buf, metadata.getObjectSize(), metadata.getObjectId())); + } catch (IndexBlockParseException ex) { + parseDataBlockIndex(ex.indexBlockPosition); + } + }).exceptionally(ex -> { + // unrecoverable error, possibly read on a deleted object + LOGGER.warn("s3 range read from {} [{}, {}) failed, ex", objectKey, startPosition, metadata.getObjectSize(), ex); + indexBlockCf.completeExceptionally(ex); + return null; + }); + } + + public CompletableFuture> readBlocks(List blockIndices) { + CompletableFuture rangeReadCf = s3Operator.rangeRead(objectKey, blockIndices.get(0).startPosition(), + blockIndices.get(blockIndices.size() - 1).endPosition()); + return rangeReadCf.thenApply(buf -> { + List dataBlocks = new ArrayList<>(); + parseDataBlocks(buf, blockIndices, dataBlocks); + return dataBlocks; + }); + } + + private void parseDataBlocks(ByteBuf buf, List blockIndices, List dataBlocks) { + for (DataBlockIndex blockIndexEntry : blockIndices) { + int blockSize = blockIndexEntry.size; + ByteBuf blockBuf = buf.slice(buf.readerIndex(), blockSize); + buf.skipBytes(blockSize); + dataBlocks.add(new DataBlock(blockBuf, blockIndexEntry.recordCount)); + } + } + + static class IndexBlock { + static List parse(ByteBuf objectTailBuf, long objectSize, long objectId) throws IndexBlockParseException { + long indexBlockPosition = objectTailBuf.getLong(objectTailBuf.readableBytes() - 48); + int indexBlockSize = objectTailBuf.getInt(objectTailBuf.readableBytes() - 40); + if (indexBlockPosition + objectTailBuf.readableBytes() < objectSize) { + throw new IndexBlockParseException(indexBlockPosition); + } else { + int indexRelativePosition = objectTailBuf.readableBytes() - (int) (objectSize - indexBlockPosition); + ByteBuf indexBlockBuf = objectTailBuf.slice(objectTailBuf.readerIndex() + indexRelativePosition, indexBlockSize); + int blockCount = indexBlockBuf.readInt(); + ByteBuf blocks = indexBlockBuf.slice(indexBlockBuf.readerIndex(), blockCount * 16); + List dataBlockIndices = new ArrayList<>(); + for (int i = 0; i < blockCount; i++) { + long blockPosition = blocks.readLong(); + int blockSize = blocks.readInt(); + int recordCount = blocks.readInt(); + dataBlockIndices.add(new DataBlockIndex(i, blockPosition, blockSize, recordCount)); + } + indexBlockBuf.skipBytes(blockCount * 16); + ByteBuf streamRanges = indexBlockBuf.slice(indexBlockBuf.readerIndex(), indexBlockBuf.readableBytes()); + List streamDataBlocks = new ArrayList<>(); + for (int i = 0; i < blockCount; i++) { + long streamId = streamRanges.readLong(); + long startOffset = streamRanges.readLong(); + int rangeSize = streamRanges.readInt(); + int blockIndex = streamRanges.readInt(); + streamDataBlocks.add(new StreamDataBlock(streamId, startOffset, startOffset + rangeSize, blockIndex, + objectId, dataBlockIndices.get(i).startPosition, dataBlockIndices.get(i).size, dataBlockIndices.get(i).recordCount)); + } + return streamDataBlocks; + } + } + + } + + static class IndexBlockParseException extends Exception { + long indexBlockPosition; + + public IndexBlockParseException(long indexBlockPosition) { + this.indexBlockPosition = indexBlockPosition; + } + + } + + + public static class DataBlockIndex { + public static final int BLOCK_INDEX_SIZE = 8 + 4 + 4; + private final int blockId; + private final long startPosition; + private final int size; + private final int recordCount; + + public DataBlockIndex(int blockId, long startPosition, int size, int recordCount) { + this.blockId = blockId; + this.startPosition = startPosition; + this.size = size; + this.recordCount = recordCount; + } + + public int blockId() { + return blockId; + } + + public long startPosition() { + return startPosition; + } + + public long endPosition() { + return startPosition + size; + } + + public int recordCount() { + return recordCount; + } + } + + public static class DataBlock { + private final ByteBuf buf; + private final int recordCount; + + public DataBlock(ByteBuf buf, int recordCount) { + this.buf = buf; + this.recordCount = recordCount; + } + + public ByteBuf buffer() { + return buf; + } + } + +} diff --git a/core/src/main/scala/kafka/log/s3/compact/operator/DataBlockWriter.java b/core/src/main/scala/kafka/log/s3/compact/operator/DataBlockWriter.java new file mode 100644 index 0000000000..40d26bdfb3 --- /dev/null +++ b/core/src/main/scala/kafka/log/s3/compact/operator/DataBlockWriter.java @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.s3.compact.operator; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.CompositeByteBuf; +import io.netty.buffer.Unpooled; +import kafka.log.s3.compact.objects.StreamDataBlock; +import kafka.log.s3.operator.S3Operator; +import kafka.log.s3.operator.Writer; +import org.apache.kafka.metadata.stream.ObjectUtils; + +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +//TODO: refactor to reduce duplicate code with ObjectWriter +public class DataBlockWriter { + private final int partSizeThreshold; + private final List waitingUploadBlocks; + private final Map> waitingUploadBlockCfs; + private final List completedBlocks; + private IndexBlock indexBlock; + private final Writer writer; + private final long objectId; + private long nextDataBlockPosition; + private long size; + + public DataBlockWriter(long objectId, S3Operator s3Operator, int partSizeThreshold) { + this.objectId = objectId; + String objectKey = ObjectUtils.genKey(0, "todocluster", objectId); + this.partSizeThreshold = partSizeThreshold; + waitingUploadBlocks = new LinkedList<>(); + waitingUploadBlockCfs = new HashMap<>(); + completedBlocks = new LinkedList<>(); + writer = s3Operator.writer(objectKey); + } + + public CompletableFuture write(StreamDataBlock dataBlock) { + CompletableFuture writeCf = new CompletableFuture<>(); + waitingUploadBlockCfs.put(dataBlock, writeCf); + waitingUploadBlocks.add(dataBlock); + long waitingUploadSize = waitingUploadBlocks.stream().mapToLong(StreamDataBlock::getBlockSize).sum(); + if (waitingUploadSize >= partSizeThreshold) { + CompositeByteBuf partBuf = Unpooled.compositeBuffer(); + for (StreamDataBlock block : waitingUploadBlocks) { + partBuf.addComponent(true, dataBlock.getDataCf().join()); + completedBlocks.add(block); + nextDataBlockPosition += block.getBlockSize(); + } + List blocks = new LinkedList<>(waitingUploadBlocks); + writer.write(partBuf).thenAccept(v -> { + for (StreamDataBlock block : blocks) { + waitingUploadBlockCfs.get(block).complete(null); + } + }); + waitingUploadBlocks.clear(); + } + return writeCf; + } + + public CompletableFuture close() { + CompositeByteBuf buf = Unpooled.compositeBuffer(); + for (StreamDataBlock block : waitingUploadBlocks) { + buf.addComponent(true, block.getDataCf().join()); + completedBlocks.add(block); + nextDataBlockPosition += block.getBlockSize(); + } + waitingUploadBlocks.clear(); + indexBlock = new IndexBlock(); + buf.addComponent(true, indexBlock.buffer()); + Footer footer = new Footer(); + buf.addComponent(true, footer.buffer()); + writer.write(buf.duplicate()); + size = indexBlock.position() + indexBlock.size() + footer.size(); + return writer.close(); + } + + public long objectId() { + return objectId; + } + + public long size() { + return size; + } + + class IndexBlock { + private final ByteBuf buf; + private final long position; + + public IndexBlock() { + position = nextDataBlockPosition; + buf = Unpooled.buffer(1024 * 1024); + buf.writeInt(completedBlocks.size()); // block count + // block index + for (StreamDataBlock block : completedBlocks) { + buf.writeLong(block.getBlockPosition()); + buf.writeInt(block.getBlockSize()); + buf.writeInt(block.getRecordCount()); + } + // object stream range + for (int blockIndex = 0; blockIndex < completedBlocks.size(); blockIndex++) { + StreamDataBlock block = completedBlocks.get(blockIndex); + buf.writeLong(block.getStreamId()); + buf.writeLong(block.getStartOffset()); + buf.writeInt((int) (block.getEndOffset() - block.getStartOffset())); + buf.writeInt(blockIndex); + } + } + + public ByteBuf buffer() { + return buf.duplicate(); + } + + public long position() { + return position; + } + + public int size() { + return buf.readableBytes(); + } + } + + class Footer { + private static final int FOOTER_SIZE = 48; + private static final long MAGIC = 0x88e241b785f4cff7L; + private final ByteBuf buf; + + public Footer() { + buf = Unpooled.buffer(FOOTER_SIZE); + buf.writeLong(indexBlock.position()); + buf.writeInt(indexBlock.size()); + buf.writeZero(40 - 8 - 4); + buf.writeLong(MAGIC); + } + + public ByteBuf buffer() { + return buf.duplicate(); + } + + public int size() { + return FOOTER_SIZE; + } + + } +} diff --git a/core/src/main/scala/kafka/log/s3/exceptions/IndexBlockParseException.java b/core/src/main/scala/kafka/log/s3/exceptions/IndexBlockParseException.java new file mode 100644 index 0000000000..2a257ac8ac --- /dev/null +++ b/core/src/main/scala/kafka/log/s3/exceptions/IndexBlockParseException.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.s3.exceptions; + +public class IndexBlockParseException extends Exception { + private final long indexBlockPosition; + + public IndexBlockParseException(long indexBlockPosition) { + this.indexBlockPosition = indexBlockPosition; + } + + public long getIndexBlockPosition() { + return indexBlockPosition; + } +} diff --git a/core/src/main/scala/kafka/log/s3/memory/MemoryMetadataManager.java b/core/src/main/scala/kafka/log/s3/memory/MemoryMetadataManager.java index 8faeca87d7..41fad49511 100644 --- a/core/src/main/scala/kafka/log/s3/memory/MemoryMetadataManager.java +++ b/core/src/main/scala/kafka/log/s3/memory/MemoryMetadataManager.java @@ -17,43 +17,44 @@ package kafka.log.s3.memory; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.TimeUnit; -import java.util.function.Supplier; -import java.util.stream.Collectors; - import kafka.log.s3.objects.CommitStreamObjectRequest; import kafka.log.s3.objects.CommitWALObjectRequest; import kafka.log.s3.objects.CommitWALObjectResponse; import kafka.log.s3.objects.ObjectManager; import kafka.log.s3.objects.ObjectStreamRange; import kafka.log.s3.objects.OpenStreamMetadata; -import org.apache.kafka.common.errors.s3.StreamNotClosedException; -import org.apache.kafka.metadata.stream.S3StreamConstant; -import org.apache.kafka.metadata.stream.S3ObjectMetadata; import kafka.log.s3.streams.StreamManager; -import org.apache.kafka.metadata.stream.ObjectUtils; import org.apache.kafka.common.errors.s3.StreamFencedException; +import org.apache.kafka.common.errors.s3.StreamNotClosedException; import org.apache.kafka.common.errors.s3.StreamNotExistException; +import org.apache.kafka.metadata.stream.ObjectUtils; import org.apache.kafka.metadata.stream.S3Object; +import org.apache.kafka.metadata.stream.S3ObjectMetadata; import org.apache.kafka.metadata.stream.S3ObjectState; -import org.apache.kafka.metadata.stream.StreamOffsetRange; +import org.apache.kafka.metadata.stream.S3StreamConstant; import org.apache.kafka.metadata.stream.S3StreamObject; import org.apache.kafka.metadata.stream.S3WALObject; +import org.apache.kafka.metadata.stream.StreamOffsetRange; import org.apache.kafka.metadata.stream.StreamState; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Supplier; +import java.util.stream.Collectors; + public class MemoryMetadataManager implements StreamManager, ObjectManager { private static final int MOCK_BROKER_ID = 0; @@ -61,14 +62,14 @@ public class MemoryMetadataManager implements StreamManager, ObjectManager { private final EventDriver eventDriver; private volatile long nextAssignedObjectId = 0; private final Map objectsMetadata; - private volatile long nextAssignedStreamId = 0; + private final AtomicLong nextAssignedStreamId = new AtomicLong(0L); private final Map streamsMetadata; private final Map brokerWALMetadata; private static class MemoryStreamMetadata { private final long streamId; - private StreamState state = StreamState.CLOSED; + private StreamState state; private long epoch = S3StreamConstant.INIT_EPOCH; private long startOffset = S3StreamConstant.INIT_START_OFFSET; private long endOffset = S3StreamConstant.INIT_END_OFFSET; @@ -115,7 +116,7 @@ public MemoryMetadataManager() { public CompletableFuture submitEvent(Supplier eventHandler) { CompletableFuture cb = new CompletableFuture<>(); - MemoryMetadataEvent event = new MemoryMetadataEvent(cb, eventHandler); + MemoryMetadataEvent event = new MemoryMetadataEvent<>(cb, eventHandler); if (!eventDriver.submit(event)) { throw new RuntimeException("Offer event failed"); } @@ -269,7 +270,7 @@ public List getObjects(long streamId, long startOffset, long e @Override public CompletableFuture createStream() { return this.submitEvent(() -> { - long streamId = this.nextAssignedStreamId++; + long streamId = this.nextAssignedStreamId.getAndIncrement(); this.streamsMetadata.put(streamId, new MemoryStreamMetadata(streamId)); return streamId; diff --git a/core/src/main/scala/kafka/log/s3/metadata/InRangeObjectsFetcher.java b/core/src/main/scala/kafka/log/s3/metadata/InRangeObjectsFetcher.java index 5c883a2b22..ff16a95734 100644 --- a/core/src/main/scala/kafka/log/s3/metadata/InRangeObjectsFetcher.java +++ b/core/src/main/scala/kafka/log/s3/metadata/InRangeObjectsFetcher.java @@ -17,18 +17,19 @@ package kafka.log.s3.metadata; -import java.util.concurrent.CompletableFuture; import org.apache.kafka.metadata.stream.InRangeObjects; +import java.util.concurrent.CompletableFuture; + public interface InRangeObjectsFetcher { /** * fetch stream interval related objects * - * @param streamId stream id + * @param streamId stream id * @param startOffset start offset, inclusive, if not exist, return INVALID - * @param endOffset end offset, exclusive, if not exist, wait for it - * @param limit max object count + * @param endOffset end offset, exclusive, if not exist, wait for it + * @param limit max object count * @return {@link InRangeObjects} */ CompletableFuture fetch(long streamId, long startOffset, long endOffset, int limit); diff --git a/core/src/main/scala/kafka/log/s3/metadata/StreamMetadataManager.java b/core/src/main/scala/kafka/log/s3/metadata/StreamMetadataManager.java index cda835464d..bd4777a276 100644 --- a/core/src/main/scala/kafka/log/s3/metadata/StreamMetadataManager.java +++ b/core/src/main/scala/kafka/log/s3/metadata/StreamMetadataManager.java @@ -18,19 +18,6 @@ package kafka.log.s3.metadata; import io.netty.util.concurrent.DefaultThreadFactory; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; -import java.util.TreeMap; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.stream.Collectors; import kafka.log.es.FutureUtil; import kafka.server.BrokerServer; import kafka.server.KafkaConfig; @@ -42,11 +29,26 @@ import org.apache.kafka.metadata.stream.InRangeObjects; import org.apache.kafka.metadata.stream.S3Object; import org.apache.kafka.metadata.stream.S3ObjectMetadata; +import org.apache.kafka.metadata.stream.S3WALObjectMetadata; import org.apache.kafka.metadata.stream.StreamOffsetRange; import org.apache.kafka.raft.OffsetAndEpoch; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.stream.Collectors; + public class StreamMetadataManager implements InRangeObjectsFetcher { // TODO: optimize by more suitable concurrent protection @@ -92,6 +94,18 @@ private void onImageChanged(MetadataDelta delta, MetadataImage newImage) { } } + public List getWALObjects() { + synchronized (this) { + return this.streamsImage.getWALObjects(config.brokerId()).stream() + .map(walObject -> { + S3Object s3Object = this.objectsImage.getObjectMetadata(walObject.objectId()); + S3ObjectMetadata metadata = new S3ObjectMetadata(walObject.objectId(), s3Object.getObjectSize(), walObject.objectType()); + return new S3WALObjectMetadata(walObject, metadata); + }) + .collect(Collectors.toList()); + } + } + // must access thread safe private List removePendingTasks() { if (this.pendingGetObjectsTasks == null || this.pendingGetObjectsTasks.isEmpty()) { @@ -99,10 +113,10 @@ private List removePendingTasks() { } Set pendingStreams = pendingGetObjectsTasks.keySet(); List pendingStreamsOffsetRange = pendingStreams - .stream() - .map(streamsImage::offsetRange) - .filter(offset -> offset != StreamOffsetRange.INVALID) - .collect(Collectors.toList()); + .stream() + .map(streamsImage::offsetRange) + .filter(offset -> offset != StreamOffsetRange.INVALID) + .collect(Collectors.toList()); if (pendingStreamsOffsetRange.isEmpty()) { return Collections.emptyList(); } @@ -115,7 +129,7 @@ private List removePendingTasks() { return; } Iterator>> iterator = - tasks.entrySet().iterator(); + tasks.entrySet().iterator(); while (iterator.hasNext()) { Entry> entry = iterator.next(); long pendingEndOffset = entry.getKey(); @@ -139,8 +153,8 @@ public CompletableFuture fetch(long streamId, long startOffset, S3StreamMetadataImage streamImage = streamsImage.streamsMetadata().get(streamId); if (streamImage == null) { LOGGER.warn( - "[FetchObjects]: stream: {}, startOffset: {}, endOffset: {}, limit: {}, and streamImage is null", - streamId, startOffset, endOffset, limit); + "[FetchObjects]: stream: {}, startOffset: {}, endOffset: {}, limit: {}, and streamImage is null", + streamId, startOffset, endOffset, limit); return CompletableFuture.completedFuture(InRangeObjects.INVALID); } StreamOffsetRange offsetRange = streamImage.offsetRange(); @@ -151,8 +165,8 @@ public CompletableFuture fetch(long streamId, long startOffset, long streamEndOffset = offsetRange.getEndOffset(); if (startOffset < streamStartOffset) { LOGGER.warn( - "[FetchObjects]: stream: {}, startOffset: {}, endOffset: {}, limit: {}, and startOffset < streamStartOffset: {}", - streamId, startOffset, endOffset, limit, streamStartOffset); + "[FetchObjects]: stream: {}, startOffset: {}, endOffset: {}, limit: {}, and startOffset < streamStartOffset: {}", + streamId, startOffset, endOffset, limit, streamStartOffset); return CompletableFuture.completedFuture(InRangeObjects.INVALID); } if (endOffset > streamEndOffset) { @@ -167,11 +181,11 @@ public CompletableFuture fetch(long streamId, long startOffset, private CompletableFuture pendingFetch(long streamId, long startOffset, long endOffset, int limit) { GetObjectsTask task = GetObjectsTask.of(streamId, startOffset, endOffset, limit); Map> tasks = StreamMetadataManager.this.pendingGetObjectsTasks.computeIfAbsent(task.streamId, - k -> new TreeMap<>()); + k -> new TreeMap<>()); List getObjectsTasks = tasks.computeIfAbsent(task.endOffset, k -> new ArrayList<>()); getObjectsTasks.add(task); LOGGER.warn("[PendingFetch]: stream: {}, startOffset: {}, endOffset: {}, limit: {}, and pending fetch", streamId, startOffset, endOffset, - limit); + limit); return task.cf; } @@ -180,8 +194,8 @@ private CompletableFuture fetch0(long streamId, long startOffset InRangeObjects cachedInRangeObjects = streamsImage.getObjects(streamId, startOffset, endOffset, limit); if (cachedInRangeObjects == null || cachedInRangeObjects == InRangeObjects.INVALID) { LOGGER.warn( - "[FetchObjects]: stream: {}, startOffset: {}, endOffset: {}, limit: {}, and search in metadataCache failed with empty result", - streamId, startOffset, endOffset, limit); + "[FetchObjects]: stream: {}, startOffset: {}, endOffset: {}, limit: {}, and search in metadataCache failed with empty result", + streamId, startOffset, endOffset, limit); return CompletableFuture.completedFuture(InRangeObjects.INVALID); } // fill the objects' size @@ -190,15 +204,15 @@ private CompletableFuture fetch0(long streamId, long startOffset if (objectMetadata == null) { // should not happen LOGGER.error( - "[FetchObjects]: stream: {}, startOffset: {}, endOffset: {}, limit: {}, and search in metadataCache failed with empty result", - streamId, startOffset, endOffset, limit); + "[FetchObjects]: stream: {}, startOffset: {}, endOffset: {}, limit: {}, and search in metadataCache failed with empty result", + streamId, startOffset, endOffset, limit); return CompletableFuture.completedFuture(InRangeObjects.INVALID); } object.setObjectSize(objectMetadata.getObjectSize()); } LOGGER.trace( - "[FetchObjects]: stream: {}, startOffset: {}, endOffset: {}, limit: {}, and search in metadataCache success with result: {}", - streamId, startOffset, endOffset, limit, cachedInRangeObjects); + "[FetchObjects]: stream: {}, startOffset: {}, endOffset: {}, limit: {}, and search in metadataCache success with result: {}", + streamId, startOffset, endOffset, limit, cachedInRangeObjects); return CompletableFuture.completedFuture(cachedInRangeObjects); } diff --git a/core/src/main/scala/kafka/log/s3/network/ControllerRequestSender.java b/core/src/main/scala/kafka/log/s3/network/ControllerRequestSender.java index 6e6f45cecd..643f685a90 100644 --- a/core/src/main/scala/kafka/log/s3/network/ControllerRequestSender.java +++ b/core/src/main/scala/kafka/log/s3/network/ControllerRequestSender.java @@ -17,7 +17,6 @@ package kafka.log.s3.network; -import java.util.concurrent.CompletableFuture; import kafka.server.BrokerServer; import kafka.server.BrokerToControllerChannelManager; import kafka.server.ControllerRequestCompletionHandler; @@ -28,6 +27,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.concurrent.CompletableFuture; + public class ControllerRequestSender { private static final Logger LOGGER = LoggerFactory.getLogger(ControllerRequestSender.class); @@ -40,7 +41,7 @@ public ControllerRequestSender(BrokerServer brokerServer) { } public CompletableFuture send(AbstractRequest.Builder requestBuilder, - Class responseDataType) { + Class responseDataType) { CompletableFuture cf = new CompletableFuture<>(); LOGGER.debug("Sending request {}", requestBuilder); channelManager.sendRequest(requestBuilder, new ControllerRequestCompletionHandler() { @@ -65,7 +66,7 @@ public void onComplete(ClientResponse response) { } if (!responseDataType.isInstance(response.responseBody().data())) { LOGGER.error("Unexpected response type: {} while sending request: {}", - response.responseBody().data().getClass().getSimpleName(), requestBuilder); + response.responseBody().data().getClass().getSimpleName(), requestBuilder); cf.completeExceptionally(new RuntimeException("Unexpected response type while sending request")); } cf.complete((R) response.responseBody().data()); diff --git a/core/src/main/scala/kafka/log/s3/objects/ControllerObjectManager.java b/core/src/main/scala/kafka/log/s3/objects/ControllerObjectManager.java index 0c61b39aee..70d5f14cde 100644 --- a/core/src/main/scala/kafka/log/s3/objects/ControllerObjectManager.java +++ b/core/src/main/scala/kafka/log/s3/objects/ControllerObjectManager.java @@ -18,10 +18,6 @@ package kafka.log.s3.objects; -import java.util.Collections; -import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.stream.Collectors; import kafka.log.s3.metadata.StreamMetadataManager; import kafka.log.s3.network.ControllerRequestSender; import kafka.server.KafkaConfig; @@ -37,6 +33,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; + public class ControllerObjectManager implements ObjectManager { private final static Logger LOGGER = LoggerFactory.getLogger(ControllerObjectManager.class); @@ -54,10 +55,10 @@ public ControllerObjectManager(ControllerRequestSender requestSender, StreamMeta @Override public CompletableFuture prepareObject(int count, long ttl) { PrepareS3ObjectRequest.Builder request = new Builder( - new PrepareS3ObjectRequestData() - .setBrokerId(config.brokerId()) - .setPreparedCount(count) - .setTimeToLiveInMs(ttl) + new PrepareS3ObjectRequestData() + .setBrokerId(config.brokerId()) + .setPreparedCount(count) + .setTimeToLiveInMs(ttl) ); return requestSender.send(request, PrepareS3ObjectResponseData.class).thenApply(resp -> { Errors code = Errors.forCode(resp.errorCode()); @@ -115,7 +116,7 @@ public List getObjects(long streamId, long startOffset, long e }).get(); } catch (Exception e) { LOGGER.error("Error while get objects, streamId: {}, startOffset: {}, endOffset: {}, limit: {}", streamId, startOffset, endOffset, limit, - e); + e); return Collections.emptyList(); } } diff --git a/core/src/main/scala/kafka/log/s3/objects/ObjectManager.java b/core/src/main/scala/kafka/log/s3/objects/ObjectManager.java index 4ff1a8ce7a..fc01549ac1 100644 --- a/core/src/main/scala/kafka/log/s3/objects/ObjectManager.java +++ b/core/src/main/scala/kafka/log/s3/objects/ObjectManager.java @@ -17,9 +17,10 @@ package kafka.log.s3.objects; +import org.apache.kafka.metadata.stream.S3ObjectMetadata; + import java.util.List; import java.util.concurrent.CompletableFuture; -import org.apache.kafka.metadata.stream.S3ObjectMetadata; /** * Object metadata registry. @@ -55,10 +56,10 @@ public interface ObjectManager { * When obj1 contains stream0 [0, 100) [200, 300) and obj2 contains stream1 [100, 200), * expect getObjects(streamId, 0, 300) return [obj1, obj2, obj1] * - * @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. Why use limit instead of maxBytes? Because we cannot get stream size from object metadata. + * @param endOffset get range end offset. + * @param limit max object count. Why use limit instead of maxBytes? Because we cannot get stream size from object metadata. * @return {@link S3ObjectMetadata} */ List getObjects(long streamId, long startOffset, long endOffset, int limit); diff --git a/core/src/main/scala/kafka/log/s3/objects/ObjectStreamRange.java b/core/src/main/scala/kafka/log/s3/objects/ObjectStreamRange.java index 79f01813b1..43777ab71f 100644 --- a/core/src/main/scala/kafka/log/s3/objects/ObjectStreamRange.java +++ b/core/src/main/scala/kafka/log/s3/objects/ObjectStreamRange.java @@ -25,7 +25,8 @@ public class ObjectStreamRange { private long startOffset; private long endOffset; - public ObjectStreamRange() {} + public ObjectStreamRange() { + } public ObjectStreamRange(long streamId, long epoch, long startOffset, long endOffset) { this.streamId = streamId; @@ -41,6 +42,7 @@ public long getStreamId() { public long getEpoch() { return epoch; } + public long getStartOffset() { return startOffset; } @@ -67,9 +69,9 @@ public void setEndOffset(long endOffset) { public CommitWALObjectRequestData.ObjectStreamRange toObjectStreamRangeInRequest() { return new CommitWALObjectRequestData.ObjectStreamRange() - .setStreamId(streamId) - .setStartOffset(startOffset) - .setEndOffset(endOffset); + .setStreamId(streamId) + .setStartOffset(startOffset) + .setEndOffset(endOffset); } @Override diff --git a/core/src/main/scala/kafka/log/s3/objects/StreamObject.java b/core/src/main/scala/kafka/log/s3/objects/StreamObject.java index 79966d12f8..741bc3ada6 100644 --- a/core/src/main/scala/kafka/log/s3/objects/StreamObject.java +++ b/core/src/main/scala/kafka/log/s3/objects/StreamObject.java @@ -17,9 +17,10 @@ package kafka.log.s3.objects; -import java.util.List; import org.apache.kafka.common.message.CommitWALObjectRequestData; +import java.util.List; + public class StreamObject { private long objectId; private long objectSize; @@ -82,11 +83,11 @@ public void setSourceObjectIds(List sourceObjectIds) { public CommitWALObjectRequestData.StreamObject toStreamObjectInRequest() { return new CommitWALObjectRequestData.StreamObject() - .setStreamId(streamId) - .setObjectId(objectId) - .setObjectSize(objectSize) - .setStartOffset(startOffset) - .setEndOffset(endOffset); + .setStreamId(streamId) + .setObjectId(objectId) + .setObjectSize(objectSize) + .setStartOffset(startOffset) + .setEndOffset(endOffset); } @Override diff --git a/core/src/main/scala/kafka/log/s3/operator/DefaultS3Operator.java b/core/src/main/scala/kafka/log/s3/operator/DefaultS3Operator.java index 9bee9b4618..738cc05e81 100644 --- a/core/src/main/scala/kafka/log/s3/operator/DefaultS3Operator.java +++ b/core/src/main/scala/kafka/log/s3/operator/DefaultS3Operator.java @@ -206,7 +206,7 @@ private void init() { } @Override - public void write(ByteBuf part) { + public CompletableFuture write(ByteBuf part) { long start = System.nanoTime(); OBJECT_UPLOAD_SIZE.inc(part.readableBytes()); int partNumber = nextPartNumber.getAndIncrement(); @@ -217,6 +217,7 @@ public void write(ByteBuf part) { PART_UPLOAD_COST.update(System.nanoTime() - start); part.release(); }); + return partCf; } private void write0(String uploadId, int partNumber, ByteBuf part, CompletableFuture partCf) { diff --git a/core/src/main/scala/kafka/log/s3/operator/MemoryS3Operator.java b/core/src/main/scala/kafka/log/s3/operator/MemoryS3Operator.java index 42ed424857..91c66b207f 100644 --- a/core/src/main/scala/kafka/log/s3/operator/MemoryS3Operator.java +++ b/core/src/main/scala/kafka/log/s3/operator/MemoryS3Operator.java @@ -20,7 +20,7 @@ import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.Unpooled; import kafka.log.es.FutureUtil; - +import software.amazon.awssdk.services.s3.model.CompletedPart; import java.util.HashMap; import java.util.Map; @@ -55,8 +55,9 @@ public Writer writer(String path) { storage.put(path, buf); return new Writer() { @Override - public void write(ByteBuf part) { + public CompletableFuture write(ByteBuf part) { buf.writeBytes(part); + return CompletableFuture.completedFuture(null); } @Override diff --git a/core/src/main/scala/kafka/log/s3/operator/Writer.java b/core/src/main/scala/kafka/log/s3/operator/Writer.java index b311f5e77e..988e35dbde 100644 --- a/core/src/main/scala/kafka/log/s3/operator/Writer.java +++ b/core/src/main/scala/kafka/log/s3/operator/Writer.java @@ -18,6 +18,7 @@ package kafka.log.s3.operator; import io.netty.buffer.ByteBuf; +import software.amazon.awssdk.services.s3.model.CompletedPart; import java.util.concurrent.CompletableFuture; @@ -30,7 +31,7 @@ public interface Writer { * * @param part object part. */ - void write(ByteBuf part); + CompletableFuture write(ByteBuf part); /** * Copy a part of the object. diff --git a/core/src/main/scala/kafka/log/s3/streams/StreamManager.java b/core/src/main/scala/kafka/log/s3/streams/StreamManager.java index 51e3c39de8..8387a1dcbc 100644 --- a/core/src/main/scala/kafka/log/s3/streams/StreamManager.java +++ b/core/src/main/scala/kafka/log/s3/streams/StreamManager.java @@ -17,11 +17,12 @@ package kafka.log.s3.streams; -import java.util.List; -import java.util.concurrent.CompletableFuture; import kafka.log.s3.objects.OpenStreamMetadata; import org.apache.kafka.metadata.stream.StreamOffsetRange; +import java.util.List; +import java.util.concurrent.CompletableFuture; + public interface StreamManager { /** @@ -55,8 +56,8 @@ public interface StreamManager { /** * Close stream. Other server can open stream with newer epoch. * - * @param streamId stream id. - * @param epoch stream epoch. + * @param streamId stream id. + * @param epoch stream epoch. */ CompletableFuture closeStream(long streamId, long epoch); @@ -70,7 +71,7 @@ public interface StreamManager { /** * Get streams offset. - * + *

* When server is starting or recovering, wal in EBS need streams offset to determine the recover point. * * @param streamIds stream ids. diff --git a/core/src/main/scala/kafka/log/s3/wal/WriteAheadLog.java b/core/src/main/scala/kafka/log/s3/wal/WriteAheadLog.java index 27e0d4310c..305722739f 100644 --- a/core/src/main/scala/kafka/log/s3/wal/WriteAheadLog.java +++ b/core/src/main/scala/kafka/log/s3/wal/WriteAheadLog.java @@ -27,18 +27,21 @@ public interface WriteAheadLog { /** * Get log start offset. + * * @return start offset. */ long startOffset(); /** * Get log end offset. + * * @return exclusive end offset. */ long endOffset(); /** * Read data from log. + * * @return list of {@link WalRecord}. */ List read(); diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index c76842a1a7..07f878fa29 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -309,6 +309,14 @@ object Defaults { val QuorumLingerMs = RaftConfig.DEFAULT_QUORUM_LINGER_MS val QuorumRequestTimeoutMs = RaftConfig.DEFAULT_QUORUM_REQUEST_TIMEOUT_MS val QuorumRetryBackoffMs = RaftConfig.DEFAULT_QUORUM_RETRY_BACKOFF_MS + + /** ********* Kafka on S3 Configuration *********/ + val S3ObjectCompactionCacheSize: Long = 2 * 1024 * 1024 * 1024 // 2GB + val S3ObjectCompactionNWInBandwidth: Long = 50 * 1024 * 1024 // 50MB/s + val S3ObjectCompactionNWOutBandwidth: Long = 50 * 1024 * 1024 // 50MB/s + val S3ObjectCompactionUploadConcurrency: Int = 8 + val S3ObjectCompactionExecutionScoreThreshold: Double = 0.5 + val S3ObjectCompactionStreamSplitSize: Long = 16 * 1024 * 1024 // 16MB } object KafkaConfig { @@ -684,6 +692,12 @@ object KafkaConfig { val S3ObjectBlockSizeProp = "s3.object.block.size" val S3ObjectPartSizeProp = "s3.object.part.size" val S3CacheSizeProp = "s3.cache.size" + val S3ObjectCompactionCacheSize = "s3.object.compaction.cache.size" + val S3ObjectCompactionNWInBandwidth = "s3.object.compaction.network.in.bandwidth" + val S3ObjectCompactionNWOutBandwidth = "s3.object.compaction.network.out.bandwidth" + val S3ObjectCompactionUploadConcurrency = "s3.object.compaction.upload.concurrency" + val S3ObjectCompactionExecutionScoreThreshold = "s3.object.compaction.execution.score.threshold" + val S3ObjectCompactionStreamSplitSize = "s3.object.compaction.stream.split.size" val S3EndpointDoc = "The S3 endpoint, ex. https://s3.{region}.amazonaws.com." val S3RegionDoc = "The S3 region, ex. us-east-1." @@ -693,6 +707,12 @@ object KafkaConfig { val S3ObjectBlockSizeDoc = "The S3 object compressed block size threshold." val S3ObjectPartSizeDoc = "The S3 object multi-part upload part size threshold." val S3CacheSizeDoc = "The S3 block cache size in MiB." + val S3ObjectCompactionCacheSizeDoc = "The S3 object compaction cache size in Bytes." + val S3ObjectCompactionNWInBandwidthDoc = "The S3 object compaction network in bandwidth in Bytes/s." + val S3ObjectCompactionNWOutBandwidthDoc = "The S3 object compaction network out bandwidth in Bytes/s." + val S3ObjectCompactionUploadConcurrencyDoc = "The S3 object compaction upload concurrency." + val S3ObjectCompactionExecutionScoreThresholdDoc = "The S3 object compaction execution score threshold." + val S3ObjectCompactionStreamSplitSizeDoc = "The S3 object compaction stream split size threshold in Bytes." // Kafka on S3 inject end @@ -1501,6 +1521,13 @@ object KafkaConfig { .define(S3ObjectBlockSizeProp, INT, 8388608, MEDIUM, S3ObjectBlockSizeDoc) .define(S3ObjectPartSizeProp, INT, 16777216, MEDIUM, S3ObjectPartSizeDoc) .define(S3CacheSizeProp, LONG, 1073741824, MEDIUM, S3CacheSizeDoc) + .define(S3ObjectCompactionCacheSize, LONG, Defaults.S3ObjectCompactionCacheSize, MEDIUM, S3ObjectCompactionCacheSizeDoc) + .define(S3ObjectCompactionNWInBandwidth, LONG, Defaults.S3ObjectCompactionNWInBandwidth, MEDIUM, S3ObjectCompactionNWInBandwidthDoc) + .define(S3ObjectCompactionNWOutBandwidth, LONG, Defaults.S3ObjectCompactionNWOutBandwidth, MEDIUM, S3ObjectCompactionNWOutBandwidthDoc) + .define(S3ObjectCompactionUploadConcurrency, INT, Defaults.S3ObjectCompactionUploadConcurrency, MEDIUM, S3ObjectCompactionUploadConcurrencyDoc) + .define(S3ObjectCompactionExecutionScoreThreshold, DOUBLE, Defaults.S3ObjectCompactionExecutionScoreThreshold, MEDIUM, S3ObjectCompactionExecutionScoreThresholdDoc) + .define(S3ObjectCompactionStreamSplitSize, LONG, Defaults.S3ObjectCompactionStreamSplitSize, MEDIUM, S3ObjectCompactionStreamSplitSizeDoc) + .define(S3CacheSizeProp, INT, 1024, MEDIUM, S3CacheSizeDoc) // Kafka on S3 inject end } @@ -2044,6 +2071,12 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val s3ObjectBlockSize = getInt(KafkaConfig.S3ObjectBlockSizeProp) val s3ObjectPartSize = getInt(KafkaConfig.S3ObjectPartSizeProp) val s3CacheSize = getLong(KafkaConfig.S3CacheSizeProp) + val s3ObjectCompactionCacheSize = getLong(KafkaConfig.S3ObjectCompactionCacheSize) + val s3ObjectCompactionNWInBandwidth = getLong(KafkaConfig.S3ObjectCompactionNWInBandwidth) + val s3ObjectCompactionNWOutBandwidth = getLong(KafkaConfig.S3ObjectCompactionNWInBandwidth) + val s3ObjectCompactionUploadConcurrency = getInt(KafkaConfig.S3ObjectCompactionUploadConcurrency) + val s3ObjectCompactionExecutionScoreThreshold = getDouble(KafkaConfig.S3ObjectCompactionExecutionScoreThreshold) + val s3ObjectCompactionStreamSplitSize = getLong(KafkaConfig.S3ObjectCompactionStreamSplitSize) // Kafka on S3 inject end def addReconfigurable(reconfigurable: Reconfigurable): Unit = { diff --git a/core/src/test/java/kafka/log/s3/StreamMetadataManagerTest.java b/core/src/test/java/kafka/log/s3/StreamMetadataManagerTest.java index cff1f79994..85971dcce3 100644 --- a/core/src/test/java/kafka/log/s3/StreamMetadataManagerTest.java +++ b/core/src/test/java/kafka/log/s3/StreamMetadataManagerTest.java @@ -23,6 +23,7 @@ import static org.mockito.ArgumentMatchers.any; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; @@ -30,6 +31,7 @@ import kafka.log.s3.metadata.StreamMetadataManager; import kafka.log.s3.metadata.StreamMetadataManager.StreamMetadataListener; import kafka.server.BrokerServer; +import kafka.server.KafkaConfig; import kafka.server.metadata.BrokerMetadataListener; import kafka.server.metadata.KRaftMetadataCache; import org.apache.kafka.image.BrokerS3WALMetadataImage; @@ -45,6 +47,10 @@ import org.apache.kafka.metadata.stream.S3ObjectState; import org.apache.kafka.metadata.stream.S3ObjectType; import org.apache.kafka.metadata.stream.S3StreamObject; +import org.apache.kafka.metadata.stream.S3WALObject; +import org.apache.kafka.metadata.stream.S3WALObjectMetadata; +import org.apache.kafka.metadata.stream.SortedWALObjectsList; +import org.apache.kafka.metadata.stream.StreamOffsetRange; import org.apache.kafka.metadata.stream.StreamState; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; @@ -60,6 +66,7 @@ public class StreamMetadataManagerTest { private static final int BROKER1 = 1; private static final long STREAM0 = 0; private static final long STREAM1 = 1; + private static final long STREAM2 = 2; private BrokerServer mockBroker; private KRaftMetadataCache mockMetadataCache; @@ -79,7 +86,9 @@ public void setUp() { return null; }).when(this.mockBrokerMetadataListener).registerStreamMetadataListener(any()); Mockito.when(this.mockMetadataCache.currentImage()).thenReturn(MetadataImage.EMPTY); - this.manager = new StreamMetadataManager(this.mockBroker, null); + KafkaConfig config = Mockito.mock(KafkaConfig.class); + Mockito.when(config.brokerId()).thenReturn(BROKER0); + this.manager = new StreamMetadataManager(this.mockBroker, config); } private static MetadataImage image0; @@ -99,8 +108,15 @@ public void setUp() { Map streamObjects = Map.of( 0L, new S3StreamObject(0L, 128, STREAM0, 10L, 100L)); S3StreamMetadataImage streamImage = new S3StreamMetadataImage(STREAM0, 1L, StreamState.OPENED, 0, 10L, ranges, streamObjects); + + BrokerS3WALMetadataImage walMetadataImage0 = new BrokerS3WALMetadataImage(BROKER0, new SortedWALObjectsList(List.of( + new S3WALObject(1L, BROKER0, Map.of( + STREAM1, List.of(new StreamOffsetRange(STREAM1, 0L, 100L))), 1L), + new S3WALObject(2L, BROKER0, Map.of( + STREAM2, List.of(new StreamOffsetRange(STREAM2, 0L, 100L))), 1L)))); + S3StreamsMetadataImage streamsImage = new S3StreamsMetadataImage(STREAM0, Map.of(STREAM0, streamImage), - Map.of(BROKER0, BrokerS3WALMetadataImage.EMPTY)); + Map.of(BROKER0, walMetadataImage0)); image0 = new MetadataImage(new MetadataProvenance(0, 0, 0), null, null, null, null, null, null, null, streamsImage, objectsImage, null); ranges = new HashMap<>(ranges); @@ -197,4 +213,17 @@ public void testFetch() throws Exception { } + @Test + public void testGetWALObjects() { + this.streamMetadataListener.onChange(null, image0); + List objectMetadata = this.manager.getWALObjects(); + List expected = List.of(new S3ObjectMetadata(1L, 128, S3ObjectType.UNKNOWN), + new S3ObjectMetadata(2L, 128, S3ObjectType.UNKNOWN)); + // compare objectMetadata with expected + assertEquals(expected.size(), objectMetadata.size()); + for (int i = 0; i < expected.size(); i++) { + assertEquals(expected.get(i), objectMetadata.get(i).getObjectMetadata()); + } + } + } diff --git a/core/src/test/java/kafka/log/s3/compact/CompactionAnalyzerTest.java b/core/src/test/java/kafka/log/s3/compact/CompactionAnalyzerTest.java new file mode 100644 index 0000000000..d208ac5f6c --- /dev/null +++ b/core/src/test/java/kafka/log/s3/compact/CompactionAnalyzerTest.java @@ -0,0 +1,361 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.s3.compact; + +import kafka.log.s3.ObjectWriter; +import kafka.log.s3.TestUtils; +import kafka.log.s3.compact.objects.CompactedObject; +import kafka.log.s3.compact.objects.CompactedObjectBuilder; +import kafka.log.s3.compact.objects.CompactionType; +import kafka.log.s3.compact.objects.StreamDataBlock; +import kafka.log.s3.model.StreamRecordBatch; +import kafka.log.s3.operator.MemoryS3Operator; +import kafka.log.s3.operator.S3Operator; +import org.apache.kafka.metadata.stream.S3ObjectMetadata; +import org.apache.kafka.metadata.stream.S3ObjectType; +import org.apache.kafka.metadata.stream.S3WALObject; +import org.apache.kafka.metadata.stream.S3WALObjectMetadata; +import org.apache.kafka.metadata.stream.StreamOffsetRange; +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 java.util.ArrayList; +import java.util.List; +import java.util.Map; + +@Tag("S3Unit") +public class CompactionAnalyzerTest { + private static final int BROKER_0 = 0; + private static final long STREAM_0 = 0; + private static final long STREAM_1 = 1; + private static final long STREAM_2 = 2; + private static final long OBJECT_0 = 0; + private static final long OBJECT_1 = 1; + private static final long OBJECT_2 = 2; + private static final long CACHE_SIZE = 1024; + private static final double EXECUTION_SCORE_THRESHOLD = 0.5; + private static final long STREAM_SPLIT_SIZE = 30; + private static final List S3_WAL_OBJECT_METADATA_LIST = new ArrayList<>(); + private CompactionAnalyzer compactionAnalyzer; + private S3Operator s3Operator; + + @BeforeEach + public void setUp() throws Exception { + s3Operator = new MemoryS3Operator(); + // stream data for object 0 + ObjectWriter objectWriter = new ObjectWriter(OBJECT_0, s3Operator, 1024, 1024); + StreamRecordBatch r1 = new StreamRecordBatch(STREAM_0, 0, 0, 20, TestUtils.random(20)); + StreamRecordBatch r2 = new StreamRecordBatch(STREAM_1, 0, 30, 30, TestUtils.random(30)); + StreamRecordBatch r3 = new StreamRecordBatch(STREAM_2, 0, 30, 30, TestUtils.random(30)); + objectWriter.write(STREAM_0, List.of(r1)); + objectWriter.write(STREAM_1, List.of(r2)); + objectWriter.write(STREAM_2, List.of(r3)); + objectWriter.close().get(); + S3ObjectMetadata objectMetadata = new S3ObjectMetadata(OBJECT_0, objectWriter.size(), S3ObjectType.WAL); + Map> streamsIndex = Map.of( + (long) STREAM_0, List.of(new StreamOffsetRange(STREAM_0, 0, 20)), + (long) STREAM_1, List.of(new StreamOffsetRange(STREAM_1, 30, 60)), + (long) STREAM_2, List.of(new StreamOffsetRange(STREAM_2, 30, 60)) + ); + S3WALObject walObject = new S3WALObject(OBJECT_0, BROKER_0, streamsIndex, OBJECT_0); + S3_WAL_OBJECT_METADATA_LIST.add(new S3WALObjectMetadata(walObject, objectMetadata)); + + // stream data for object 1 + objectWriter = new ObjectWriter(OBJECT_1, s3Operator, 1024, 1024); + StreamRecordBatch r4 = new StreamRecordBatch(STREAM_0, 0, 20, 5, TestUtils.random(5)); + StreamRecordBatch r5 = new StreamRecordBatch(STREAM_1, 0, 60, 60, TestUtils.random(60)); + objectWriter.write(STREAM_0, List.of(r4)); + objectWriter.write(STREAM_1, List.of(r5)); + objectWriter.close().get(); + objectMetadata = new S3ObjectMetadata(OBJECT_1, objectWriter.size(), S3ObjectType.WAL); + streamsIndex = Map.of( + (long) STREAM_0, List.of(new StreamOffsetRange(STREAM_0, 20, 25)), + (long) STREAM_1, List.of(new StreamOffsetRange(STREAM_1, 60, 120)) + ); + walObject = new S3WALObject(OBJECT_1, BROKER_0, streamsIndex, OBJECT_1); + S3_WAL_OBJECT_METADATA_LIST.add(new S3WALObjectMetadata(walObject, objectMetadata)); + + // stream data for object 0 + objectWriter = new ObjectWriter(OBJECT_2, s3Operator, 1024, 1024); + // redundant record + StreamRecordBatch r6 = new StreamRecordBatch(STREAM_1, 0, 260, 20, TestUtils.random(20)); + StreamRecordBatch r7 = new StreamRecordBatch(STREAM_1, 0, 400, 100, TestUtils.random(100)); + StreamRecordBatch r8 = new StreamRecordBatch(STREAM_2, 0, 230, 40, TestUtils.random(40)); + objectWriter.write(STREAM_1, List.of(r6)); + objectWriter.write(STREAM_1, List.of(r7)); + objectWriter.write(STREAM_2, List.of(r8)); + objectWriter.close().get(); + objectMetadata = new S3ObjectMetadata(OBJECT_2, objectWriter.size(), S3ObjectType.WAL); + streamsIndex = Map.of( + (long) STREAM_1, List.of(new StreamOffsetRange(STREAM_1, 400, 500)), + (long) STREAM_2, List.of(new StreamOffsetRange(STREAM_2, 230, 270)) + ); + walObject = new S3WALObject(OBJECT_2, BROKER_0, streamsIndex, OBJECT_2); + S3_WAL_OBJECT_METADATA_LIST.add(new S3WALObjectMetadata(walObject, objectMetadata)); + } + + @AfterEach + public void tearDown() { + S3_WAL_OBJECT_METADATA_LIST.clear(); + } + + private boolean compare(StreamDataBlock block1, StreamDataBlock block2) { + return block1.getStreamId() == block2.getStreamId() && + block1.getStartOffset() == block2.getStartOffset() && + block1.getEndOffset() == block2.getEndOffset() && + block1.getRecordCount() == block2.getRecordCount() && + block1.getObjectId() == block2.getObjectId(); + } + + private boolean compare(List streamDataBlocks1, List streamDataBlocks2) { + if (streamDataBlocks1.size() != streamDataBlocks2.size()) { + return false; + } + for (int i = 0; i < streamDataBlocks1.size(); i++) { + if (!compare(streamDataBlocks1.get(i), streamDataBlocks2.get(i))) { + return false; + } + } + return true; + } + + private boolean compare(CompactedObjectBuilder builder1, CompactedObjectBuilder builder2) { + if (builder1.type() != builder2.type()) { + return false; + } + return compare(builder1.streamDataBlocks(), builder2.streamDataBlocks()); + } + + private boolean compare(CompactedObject compactedObject1, CompactedObject compactedObject2) { + if (compactedObject1.type() != compactedObject2.type()) { + return false; + } + return compare(compactedObject1.streamDataBlocks(), compactedObject2.streamDataBlocks()); + } + + @Test + public void testReadObjectIndices() { + this.compactionAnalyzer = new CompactionAnalyzer(CACHE_SIZE, EXECUTION_SCORE_THRESHOLD, STREAM_SPLIT_SIZE, s3Operator); + List streamDataBlocks = this.compactionAnalyzer.blockWaitObjectIndices(S3_WAL_OBJECT_METADATA_LIST); + List expectedBlocks = 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), + new StreamDataBlock(STREAM_2, 30, 60, 2, OBJECT_0, -1, -1, 1), + new StreamDataBlock(STREAM_0, 20, 25, 0, OBJECT_1, -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, 230, 270, 1, OBJECT_2, -1, -1, 1)); + for (int i = 0; i < streamDataBlocks.size(); i++) { + Assertions.assertTrue(compare(streamDataBlocks.get(i), expectedBlocks.get(i))); + } + } + + @Test + public void testShouldCompact() { + this.compactionAnalyzer = new CompactionAnalyzer(CACHE_SIZE, EXECUTION_SCORE_THRESHOLD, STREAM_SPLIT_SIZE, s3Operator); + List streamDataBlocks = this.compactionAnalyzer.blockWaitObjectIndices(S3_WAL_OBJECT_METADATA_LIST); + Assertions.assertTrue(this.compactionAnalyzer.shouldCompact(streamDataBlocks)); + } + + @Test + public void testSortStreamRangePositions() { + this.compactionAnalyzer = new CompactionAnalyzer(CACHE_SIZE, EXECUTION_SCORE_THRESHOLD, STREAM_SPLIT_SIZE, s3Operator); + List streamDataBlocks = this.compactionAnalyzer.blockWaitObjectIndices(S3_WAL_OBJECT_METADATA_LIST); + List sortedStreamDataBlocks = this.compactionAnalyzer.sortStreamRangePositions(streamDataBlocks); + List expectedBlocks = List.of( + new StreamDataBlock(STREAM_0, 0, 20, 0, OBJECT_0, -1, -1, 1), + new StreamDataBlock(STREAM_0, 20, 25, 0, OBJECT_1, -1, -1, 1), + new StreamDataBlock(STREAM_1, 30, 60, 1, 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, 2, 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++) { + Assertions.assertTrue(compare(sortedStreamDataBlocks.get(i), expectedBlocks.get(i))); + } + } + + @Test + public void testBuildCompactedObject1() { + this.compactionAnalyzer = new CompactionAnalyzer(CACHE_SIZE, EXECUTION_SCORE_THRESHOLD, 100, s3Operator); + List compactedObjectBuilders = this.compactionAnalyzer.buildCompactedObjects(S3_WAL_OBJECT_METADATA_LIST); + 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_1, 400, 500, 0, OBJECT_2, -1, -1, 1)), + new CompactedObjectBuilder() + .setType(CompactionType.SPLIT) + .addStreamDataBlock(new StreamDataBlock(STREAM_2, 30, 60, 2, 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++) { + Assertions.assertTrue(compare(compactedObjectBuilders.get(i), expectedCompactedObject.get(i))); + } + } + + @Test + public void testBuildCompactedObject2() { + this.compactionAnalyzer = new CompactionAnalyzer(CACHE_SIZE, EXECUTION_SCORE_THRESHOLD, 30, s3Operator); + List compactedObjectBuilders = this.compactionAnalyzer.buildCompactedObjects(S3_WAL_OBJECT_METADATA_LIST); + List expectedCompactedObject = List.of( + new CompactedObjectBuilder() + .setType(CompactionType.SPLIT) + .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_1, 400, 500, 0, OBJECT_2, -1, -1, 1)), + new CompactedObjectBuilder() + .setType(CompactionType.SPLIT) + .addStreamDataBlock(new StreamDataBlock(STREAM_2, 30, 60, 2, 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++) { + Assertions.assertTrue(compare(compactedObjectBuilders.get(i), expectedCompactedObject.get(i))); + } + } + + @Test + public void testCompactionPlans1() { + this.compactionAnalyzer = new CompactionAnalyzer(CACHE_SIZE, EXECUTION_SCORE_THRESHOLD, 100, s3Operator); + List compactionPlans = this.compactionAnalyzer.analyze(S3_WAL_OBJECT_METADATA_LIST); + Assertions.assertEquals(1, compactionPlans.size()); + List expectCompactedObjects = List.of( + 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)) + .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, 2, OBJECT_0, -1, -1, 1)) + .build(), + 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)) + .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, 20, 0, OBJECT_0, -1, -1, 1), + new StreamDataBlock(STREAM_1, 30, 60, 1, OBJECT_0, -1, -1, 1), + new StreamDataBlock(STREAM_2, 30, 60, 2, 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_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++) { + Assertions.assertTrue(compare(compactionPlan.compactedObjects().get(i), expectCompactedObjects.get(i))); + } + for (Long objectId : compactionPlan.streamDataBlocksMap().keySet()) { + Assertions.assertTrue(compare(compactionPlan.streamDataBlocksMap().get(objectId), expectObjectStreamDataBlocks.get(objectId))); + } + } + + @Test + public void testCompactionPlans2() { + this.compactionAnalyzer = new CompactionAnalyzer(300, EXECUTION_SCORE_THRESHOLD, 100, s3Operator); + List compactionPlans = this.compactionAnalyzer.analyze(S3_WAL_OBJECT_METADATA_LIST); + Assertions.assertEquals(2, compactionPlans.size()); + + // first iteration + List expectCompactedObjects = List.of( + 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)) + .build(), + 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)) + .build()); + Map> expectObjectStreamDataBlocks = 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))); + CompactionPlan compactionPlan = compactionPlans.get(0); + for (int i = 0; i < compactionPlan.compactedObjects().size(); i++) { + Assertions.assertTrue(compare(compactionPlan.compactedObjects().get(i), expectCompactedObjects.get(i))); + } + for (Long objectId : compactionPlan.streamDataBlocksMap().keySet()) { + Assertions.assertTrue(compare(compactionPlan.streamDataBlocksMap().get(objectId), expectObjectStreamDataBlocks.get(objectId))); + } + + // 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, 2, 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, 2, 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++) { + Assertions.assertTrue(compare(compactionPlan.compactedObjects().get(i), expectCompactedObjects.get(i))); + } + for (Long objectId : compactionPlan.streamDataBlocksMap().keySet()) { + Assertions.assertTrue(compare(compactionPlan.streamDataBlocksMap().get(objectId), expectObjectStreamDataBlocks.get(objectId))); + } + } + + @Test + public void testCompactionPlanWithException() { + + } +} diff --git a/core/src/test/java/kafka/log/s3/compact/CompactionManagerTest.java b/core/src/test/java/kafka/log/s3/compact/CompactionManagerTest.java new file mode 100644 index 0000000000..586981dcbb --- /dev/null +++ b/core/src/test/java/kafka/log/s3/compact/CompactionManagerTest.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.s3.compact; + +import kafka.log.s3.memory.MemoryMetadataManager; +import kafka.log.s3.metadata.StreamMetadataManager; +import kafka.log.s3.operator.MemoryS3Operator; +import kafka.server.KafkaConfig; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; + +public class CompactionManagerTest { + private static final int BROKER0 = 0; + private CompactionManager compactionManager; + + @BeforeEach + public void setUp() { + // mock parameters for CompactionManager + KafkaConfig config = Mockito.mock(KafkaConfig.class); + Mockito.when(config.brokerId()).thenReturn(BROKER0); + StreamMetadataManager streamMetadataManager = Mockito.mock(StreamMetadataManager.class); + this.compactionManager = new CompactionManager(config, new MemoryMetadataManager(), streamMetadataManager, new MemoryS3Operator()); + } + + // Test compacting multiple WAL objects + @Test + public void testCompact() { + + } +} diff --git a/metadata/src/main/java/org/apache/kafka/image/S3StreamsMetadataImage.java b/metadata/src/main/java/org/apache/kafka/image/S3StreamsMetadataImage.java index 0d4946ed3f..deea17ea55 100644 --- a/metadata/src/main/java/org/apache/kafka/image/S3StreamsMetadataImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/S3StreamsMetadataImage.java @@ -35,6 +35,7 @@ import org.apache.kafka.metadata.stream.StreamOffsetRange; import org.apache.kafka.metadata.stream.S3ObjectType; import org.apache.kafka.metadata.stream.S3StreamObject; +import org.apache.kafka.metadata.stream.S3WALObject; import org.apache.kafka.server.common.ApiMessageAndVersion; public final class S3StreamsMetadataImage { @@ -98,6 +99,14 @@ public InRangeObjects getObjects(long streamId, long startOffset, long endOffset return new InRangeObjects(streamId, startOffset, realEndOffset, objects); } + public List getWALObjects(int brokerId) { + BrokerS3WALMetadataImage wal = brokerWALMetadata.get(brokerId); + if (wal == null) { + return Collections.emptyList(); + } + return wal.getWalObjects().list(); + } + private List rangeSearchers(long streamId, long startOffset, long endOffset) { S3StreamMetadataImage streamMetadata = streamsMetadata.get(streamId); List rangeSearchers = new ArrayList<>(); diff --git a/metadata/src/main/java/org/apache/kafka/metadata/stream/S3WALObjectMetadata.java b/metadata/src/main/java/org/apache/kafka/metadata/stream/S3WALObjectMetadata.java new file mode 100644 index 0000000000..84ea92afcf --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/metadata/stream/S3WALObjectMetadata.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.metadata.stream; + +public class S3WALObjectMetadata { + private final S3WALObject walObject; + private final S3ObjectMetadata objectMetadata; + + public S3WALObjectMetadata(S3WALObject walObject, S3ObjectMetadata objectMetadata) { + this.walObject = walObject; + this.objectMetadata = objectMetadata; + } + + public S3WALObject getWalObject() { + return walObject; + } + + public S3ObjectMetadata getObjectMetadata() { + return objectMetadata; + } +} From 970ba706fee5274847edcca57abe69ae1d5b7014 Mon Sep 17 00:00:00 2001 From: Shichao Nie Date: Fri, 8 Sep 2023 16:28:05 +0800 Subject: [PATCH 2/7] feat: add unit tests for major compact Signed-off-by: Shichao Nie --- .../scala/kafka/log/s3/DefaultS3Client.java | 1 + .../log/s3/compact/CompactionAnalyzer.java | 2 +- .../log/s3/compact/CompactionManager.java | 127 ++++++------- .../log/s3/compact/CompactionUploader.java | 58 +++--- .../kafka/log/s3/compact/CompactionUtils.java | 58 ++++++ .../s3/compact/objects/StreamDataBlock.java | 2 +- .../s3/compact/operator/DataBlockWriter.java | 39 ++-- .../log/s3/memory/MemoryMetadataManager.java | 13 +- .../s3/compact/CompactionAnalyzerTest.java | 156 ++++------------ .../log/s3/compact/CompactionManagerTest.java | 72 +++++++- .../log/s3/compact/CompactionTestBase.java | 167 ++++++++++++++++++ .../s3/compact/CompactionUploaderTest.java | 138 +++++++++++++++ 12 files changed, 590 insertions(+), 243 deletions(-) create mode 100644 core/src/main/scala/kafka/log/s3/compact/CompactionUtils.java create mode 100644 core/src/test/java/kafka/log/s3/compact/CompactionTestBase.java create mode 100644 core/src/test/java/kafka/log/s3/compact/CompactionUploaderTest.java diff --git a/core/src/main/scala/kafka/log/s3/DefaultS3Client.java b/core/src/main/scala/kafka/log/s3/DefaultS3Client.java index f3b691e3d8..5aa85c1611 100644 --- a/core/src/main/scala/kafka/log/s3/DefaultS3Client.java +++ b/core/src/main/scala/kafka/log/s3/DefaultS3Client.java @@ -66,6 +66,7 @@ public DefaultS3Client(BrokerServer brokerServer, KafkaConfig config, S3Operator this.objectManager = new ControllerObjectManager(this.requestSender, this.metadataManager, this.config); this.blockCache = new DefaultS3BlockCache(config.s3CacheSize(), objectManager, operator); this.compactionManager = new CompactionManager(this.config, this.objectManager, this.metadataManager, this.operator); + this.compactionManager.start(); this.storage = new S3Storage(config, new MemoryWriteAheadLog(), objectManager, blockCache, operator); this.streamClient = new S3StreamClient(this.streamManager, this.storage); this.kvClient = new ControllerKVClient(this.requestSender); diff --git a/core/src/main/scala/kafka/log/s3/compact/CompactionAnalyzer.java b/core/src/main/scala/kafka/log/s3/compact/CompactionAnalyzer.java index 4fe2439fb3..1fc5f4610b 100644 --- a/core/src/main/scala/kafka/log/s3/compact/CompactionAnalyzer.java +++ b/core/src/main/scala/kafka/log/s3/compact/CompactionAnalyzer.java @@ -155,7 +155,7 @@ List blockWaitObjectIndices(List objectMet List validStreamDataBlocks = new ArrayList<>(); S3WALObject s3WALObject = s3WALObjectMap.get(streamDataBlocks.get(0).getObjectId()); // filter out invalid stream data blocks in case metadata is inconsistent with S3 index block - for (StreamDataBlock streamDataBlock: streamDataBlocks) { + for (StreamDataBlock streamDataBlock : streamDataBlocks) { if (s3WALObject.intersect(streamDataBlock.getStreamId(), streamDataBlock.getStartOffset(), streamDataBlock.getEndOffset())) { validStreamDataBlocks.add(streamDataBlock); } diff --git a/core/src/main/scala/kafka/log/s3/compact/CompactionManager.java b/core/src/main/scala/kafka/log/s3/compact/CompactionManager.java index 6f5144608e..92af43ee44 100644 --- a/core/src/main/scala/kafka/log/s3/compact/CompactionManager.java +++ b/core/src/main/scala/kafka/log/s3/compact/CompactionManager.java @@ -66,6 +66,9 @@ public CompactionManager(KafkaConfig config, ObjectManager objectManager, Stream this.uploader = new CompactionUploader(objectManager, s3Operator, config); this.compactionAnalyzer = new CompactionAnalyzer(compactionCacheSize, executionScoreThreshold, streamSplitSize, s3Operator); this.executorService = Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("object-compaction-manager")); + } + + public void start() { this.executorService.scheduleWithFixedDelay(this::compact, 0, 600, TimeUnit.SECONDS); } @@ -78,80 +81,86 @@ public void shutdown() { public CompletableFuture compact() { List s3ObjectMetadata = this.streamMetadataManager.getWALObjects(); try { - Map s3ObjectMetadataMap = s3ObjectMetadata.stream() - .collect(Collectors.toMap(e -> e.getWalObject().objectId(), e -> e)); List compactionPlans = this.compactionAnalyzer.analyze(s3ObjectMetadata); if (compactionPlans.isEmpty()) { return CompletableFuture.completedFuture(CompactResult.SKIPPED); } - CommitWALObjectRequest request = new CommitWALObjectRequest(); - for (CompactionPlan compactionPlan : compactionPlans) { - List> streamObjectFutureList = new ArrayList<>(); - List>> walObjStreamRangeFutureList = new ArrayList<>(); - // iterate over each compaction plan - for (Map.Entry> streamDataBlocEntry : compactionPlan.streamDataBlocksMap().entrySet()) { - S3ObjectMetadata metadata = s3ObjectMetadataMap.get(streamDataBlocEntry.getKey()).getObjectMetadata(); - List streamDataBlocks = streamDataBlocEntry.getValue(); - List blockIndices = buildBlockIndicesFromStreamDataBlock(streamDataBlocks); - networkInThrottle.throttle(streamDataBlocks.stream().mapToLong(StreamDataBlock::getBlockSize).sum()); - DataBlockReader reader = new DataBlockReader(metadata, s3Operator); - reader.readBlocks(blockIndices).thenAccept(dataBlocks -> { - for (int i = 0; i < blockIndices.size(); i++) { - StreamDataBlock streamDataBlock = streamDataBlocks.get(i); - streamDataBlock.getDataCf().complete(dataBlocks.get(i).buffer()); - } - }).exceptionally(ex -> { - LOGGER.error("read on invalid object {}, ex ", metadata.key(), ex); - for (int i = 0; i < blockIndices.size(); i++) { - StreamDataBlock streamDataBlock = streamDataBlocks.get(i); - streamDataBlock.getDataCf().completeExceptionally(ex); - } - return null; - }); - } - for (CompactedObject compactedObject : compactionPlan.compactedObjects()) { - if (compactedObject.type() == CompactionType.COMPACT) { - walObjStreamRangeFutureList.add(uploader.writeWALObject(compactedObject)); - } else { - streamObjectFutureList.add(uploader.writeStreamObject(compactedObject)); - } - } - // wait for all stream objects and wal object parts to be uploaded - try { - walObjStreamRangeFutureList.stream().map(CompletableFuture::join).forEach(e -> e.forEach(request::addStreamRange)); - streamObjectFutureList.stream().map(CompletableFuture::join).forEach(request::addStreamObject); - } catch (Exception ex) { - LOGGER.error("Error while uploading compaction objects", ex); - uploader.reset(); - return CompletableFuture.failedFuture(ex); - } - compactionPlan.streamDataBlocksMap().values().forEach(e -> e.forEach(StreamDataBlock::free)); - } - request.setObjectId(uploader.getWALObjectId()); - // set wal object id to be the first object id of compacted objects - request.setOrderId(s3ObjectMetadata.get(0).getObjectMetadata().getObjectId()); - request.setCompactedObjectIds(s3ObjectMetadata.stream().map(s -> s.getObjectMetadata().getObjectId()).collect(Collectors.toList())); - uploader.getWalObjectWriter().close().thenAccept(nil -> request.setObjectSize(uploader.getWalObjectWriter().size())).join(); - uploader.reset(); + CommitWALObjectRequest request = buildCompactRequest(compactionPlans, s3ObjectMetadata); return objectManager.commitWALObject(request).thenApply(nil -> { LOGGER.info("Compaction success, WAL object id: {}, size: {}, stream object num: {}", request.getObjectId(), request.getObjectSize(), request.getStreamObjects().size()); return CompactResult.SUCCESS; }); } catch (Exception e) { - LOGGER.error("Error while analyzing compaction objects", e); + LOGGER.error("Error while compaction objects", e); return CompletableFuture.failedFuture(e); } } - private List buildBlockIndicesFromStreamDataBlock(List streamDataBlocks) { - List blockIndices = new ArrayList<>(); - for (StreamDataBlock streamDataBlock : streamDataBlocks) { - blockIndices.add(new DataBlockReader.DataBlockIndex(streamDataBlock.getBlockId(), streamDataBlock.getBlockPosition(), - streamDataBlock.getBlockSize(), streamDataBlock.getRecordCount())); + CommitWALObjectRequest buildCompactRequest(List compactionPlans, List s3ObjectMetadata) + throws IllegalArgumentException { + CommitWALObjectRequest request = new CommitWALObjectRequest(); + Map s3ObjectMetadataMap = s3ObjectMetadata.stream() + .collect(Collectors.toMap(e -> e.getWalObject().objectId(), e -> e)); + for (CompactionPlan compactionPlan : compactionPlans) { + // iterate over each compaction plan + for (Map.Entry> streamDataBlocEntry : compactionPlan.streamDataBlocksMap().entrySet()) { + S3ObjectMetadata metadata = s3ObjectMetadataMap.get(streamDataBlocEntry.getKey()).getObjectMetadata(); + List streamDataBlocks = streamDataBlocEntry.getValue(); + List blockIndices = CompactionUtils.buildBlockIndicesFromStreamDataBlock(streamDataBlocks); + networkInThrottle.throttle(streamDataBlocks.stream().mapToLong(StreamDataBlock::getBlockSize).sum()); + DataBlockReader reader = new DataBlockReader(metadata, s3Operator); + reader.readBlocks(blockIndices).thenAccept(dataBlocks -> { + for (int i = 0; i < blockIndices.size(); i++) { + StreamDataBlock streamDataBlock = streamDataBlocks.get(i); + streamDataBlock.getDataCf().complete(dataBlocks.get(i).buffer()); + } + }).exceptionally(ex -> { + LOGGER.error("read on invalid object {}, ex ", metadata.key(), ex); + for (int i = 0; i < blockIndices.size(); i++) { + StreamDataBlock streamDataBlock = streamDataBlocks.get(i); + streamDataBlock.getDataCf().completeExceptionally(ex); + } + return null; + }); + } + List> streamObjectCFList = new ArrayList<>(); + CompletableFuture> walObjectCF = null; + List objectStreamRanges = new ArrayList<>(); + for (CompactedObject compactedObject : compactionPlan.compactedObjects()) { + if (compactedObject.type() == CompactionType.COMPACT) { + objectStreamRanges = CompactionUtils.buildObjectStreamRange(compactedObject); + walObjectCF = uploader.writeWALObject(compactedObject); + } else { + streamObjectCFList.add(uploader.writeStreamObject(compactedObject)); + } + } + // wait for all stream objects and wal object part to be uploaded + try { + if (walObjectCF != null) { + // wait for all blocks to be uploaded or added to waiting list + CompletableFuture writeObjectCF = walObjectCF.join(); + // force upload all blocks still in waiting list + uploader.forceUploadWAL(); + // wait for all blocks to be uploaded + writeObjectCF.join(); + objectStreamRanges.forEach(request::addStreamRange); + } + streamObjectCFList.stream().map(CompletableFuture::join).forEach(request::addStreamObject); + } catch (Exception ex) { + LOGGER.error("Error while uploading compaction objects", ex); + uploader.reset(); + throw new IllegalArgumentException("Error while uploading compaction objects", ex); + } +// compactionPlan.streamDataBlocksMap().values().forEach(e -> e.forEach(StreamDataBlock::free)); } - return blockIndices; + request.setObjectId(uploader.getWALObjectId()); + // set wal object id to be the first object id of compacted objects + request.setOrderId(s3ObjectMetadata.get(0).getObjectMetadata().getObjectId()); + request.setCompactedObjectIds(s3ObjectMetadata.stream().map(s -> s.getObjectMetadata().getObjectId()).collect(Collectors.toList())); + request.setObjectSize(uploader.completeWAL()); + uploader.reset(); + return request; } - } diff --git a/core/src/main/scala/kafka/log/s3/compact/CompactionUploader.java b/core/src/main/scala/kafka/log/s3/compact/CompactionUploader.java index fde25f0fad..f0a6593e9f 100644 --- a/core/src/main/scala/kafka/log/s3/compact/CompactionUploader.java +++ b/core/src/main/scala/kafka/log/s3/compact/CompactionUploader.java @@ -18,10 +18,10 @@ package kafka.log.s3.compact; import kafka.log.s3.compact.objects.CompactedObject; +import kafka.log.s3.compact.objects.CompactionType; import kafka.log.s3.compact.objects.StreamDataBlock; import kafka.log.s3.compact.operator.DataBlockWriter; import kafka.log.s3.objects.ObjectManager; -import kafka.log.s3.objects.ObjectStreamRange; import kafka.log.s3.objects.StreamObject; import kafka.log.s3.operator.S3Operator; import kafka.server.KafkaConfig; @@ -61,60 +61,45 @@ public void stop() { this.throttle.stop(); } - public CompletableFuture> writeWALObject(CompactedObject compactedObject) { - CompletableFuture> cf = new CompletableFuture<>(); - CompletableFuture.allOf(compactedObject.streamDataBlocks() + public CompletableFuture> writeWALObject(CompactedObject compactedObject) { + if (compactedObject.type() != CompactionType.COMPACT) { + return CompletableFuture.failedFuture(new IllegalArgumentException("wrong compacted object type, expected COMPACT")); + } + return CompletableFuture.allOf(compactedObject.streamDataBlocks() .stream() .map(StreamDataBlock::getDataCf) .toArray(CompletableFuture[]::new)) - .thenAcceptAsync(v -> { - prepareObjectAndWrite(compactedObject, cf); - }, executorService) + .thenComposeAsync(v -> prepareObjectAndWrite(compactedObject), executorService) .exceptionally(ex -> { LOGGER.error("wal object write failed", ex); - cf.completeExceptionally(ex); return null; }); - return cf; } - private void prepareObjectAndWrite(CompactedObject compactedObject, CompletableFuture> cf) { + private CompletableFuture> prepareObjectAndWrite(CompactedObject compactedObject) { // no race condition, only one thread at a time will request for wal object id if (walObjectIdCf == null) { walObjectIdCf = this.objectManager.prepareObject(1, TimeUnit.MINUTES.toMillis(30)); } - walObjectIdCf.thenAcceptAsync(objectId -> { + return walObjectIdCf.thenApplyAsync(objectId -> { if (walObjectWriter == null) { walObjectWriter = new DataBlockWriter(objectId, s3Operator, kafkaConfig.s3ObjectPartSize()); } - ObjectStreamRange currObjectStreamRange = null; List> writeFutureList = new ArrayList<>(); - List objectStreamRanges = new ArrayList<>(); for (StreamDataBlock streamDataBlock : compactedObject.streamDataBlocks()) { - if (currObjectStreamRange == null) { - currObjectStreamRange = new ObjectStreamRange(streamDataBlock.getStreamId(), -1L, - streamDataBlock.getStartOffset(), streamDataBlock.getEndOffset()); - } else { - if (currObjectStreamRange.getStreamId() == streamDataBlock.getStreamId()) { - currObjectStreamRange.setEndOffset(streamDataBlock.getEndOffset()); - } else { - objectStreamRanges.add(currObjectStreamRange); - currObjectStreamRange = new ObjectStreamRange(streamDataBlock.getStreamId(), -1L, - streamDataBlock.getStartOffset(), streamDataBlock.getEndOffset()); - } - } writeFutureList.add(walObjectWriter.write(streamDataBlock)); } - objectStreamRanges.add(currObjectStreamRange); - CompletableFuture.allOf(writeFutureList.toArray(new CompletableFuture[0])).thenAccept(v -> cf.complete(objectStreamRanges)); + return CompletableFuture.allOf(writeFutureList.toArray(new CompletableFuture[0])); }, executorService).exceptionally(ex -> { - LOGGER.error("prepare wal object failed", ex); - prepareObjectAndWrite(compactedObject, cf); + LOGGER.error("prepare and write wal object failed", ex); return null; }); } public CompletableFuture writeStreamObject(CompactedObject compactedObject) { + if (compactedObject.type() != CompactionType.SPLIT) { + return CompletableFuture.failedFuture(new IllegalArgumentException("wrong compacted object type, expected SPLIT")); + } return CompletableFuture.allOf(compactedObject.streamDataBlocks() .stream() .map(StreamDataBlock::getDataCf) @@ -145,8 +130,19 @@ public CompletableFuture writeStreamObject(CompactedObject compact }); } - public DataBlockWriter getWalObjectWriter() { - return walObjectWriter; + public void forceUploadWAL() { + if (walObjectWriter == null) { + return; + } + walObjectWriter.uploadWaitingList(); + } + + public long completeWAL() { + if (walObjectWriter == null) { + return 0L; + } + walObjectWriter.close().join(); + return walObjectWriter.size(); } public void reset() { diff --git a/core/src/main/scala/kafka/log/s3/compact/CompactionUtils.java b/core/src/main/scala/kafka/log/s3/compact/CompactionUtils.java new file mode 100644 index 0000000000..0b4e5a1e63 --- /dev/null +++ b/core/src/main/scala/kafka/log/s3/compact/CompactionUtils.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.s3.compact; + +import kafka.log.s3.compact.objects.CompactedObject; +import kafka.log.s3.compact.objects.StreamDataBlock; +import kafka.log.s3.compact.operator.DataBlockReader; +import kafka.log.s3.objects.ObjectStreamRange; + +import java.util.ArrayList; +import java.util.List; + +public class CompactionUtils { + public static List buildObjectStreamRange(CompactedObject compactedObject) { + List objectStreamRanges = new ArrayList<>(); + ObjectStreamRange currObjectStreamRange = null; + for (StreamDataBlock streamDataBlock : compactedObject.streamDataBlocks()) { + if (currObjectStreamRange == null) { + currObjectStreamRange = new ObjectStreamRange(streamDataBlock.getStreamId(), -1L, + streamDataBlock.getStartOffset(), streamDataBlock.getEndOffset()); + } else { + if (currObjectStreamRange.getStreamId() == streamDataBlock.getStreamId()) { + currObjectStreamRange.setEndOffset(streamDataBlock.getEndOffset()); + } else { + objectStreamRanges.add(currObjectStreamRange); + currObjectStreamRange = new ObjectStreamRange(streamDataBlock.getStreamId(), -1L, + streamDataBlock.getStartOffset(), streamDataBlock.getEndOffset()); + } + } + } + objectStreamRanges.add(currObjectStreamRange); + return objectStreamRanges; + } + + public static List buildBlockIndicesFromStreamDataBlock(List streamDataBlocks) { + List blockIndices = new ArrayList<>(); + for (StreamDataBlock streamDataBlock : streamDataBlocks) { + blockIndices.add(new DataBlockReader.DataBlockIndex(streamDataBlock.getBlockId(), streamDataBlock.getBlockPosition(), + streamDataBlock.getBlockSize(), streamDataBlock.getRecordCount())); + } + return blockIndices; + } +} diff --git a/core/src/main/scala/kafka/log/s3/compact/objects/StreamDataBlock.java b/core/src/main/scala/kafka/log/s3/compact/objects/StreamDataBlock.java index 575b6ae7f4..076bb42d40 100644 --- a/core/src/main/scala/kafka/log/s3/compact/objects/StreamDataBlock.java +++ b/core/src/main/scala/kafka/log/s3/compact/objects/StreamDataBlock.java @@ -19,8 +19,8 @@ import io.netty.buffer.ByteBuf; -import java.util.Objects; import java.util.Comparator; +import java.util.Objects; import java.util.concurrent.CompletableFuture; public class StreamDataBlock { diff --git a/core/src/main/scala/kafka/log/s3/compact/operator/DataBlockWriter.java b/core/src/main/scala/kafka/log/s3/compact/operator/DataBlockWriter.java index 40d26bdfb3..82c8cb9abd 100644 --- a/core/src/main/scala/kafka/log/s3/compact/operator/DataBlockWriter.java +++ b/core/src/main/scala/kafka/log/s3/compact/operator/DataBlockWriter.java @@ -25,11 +25,11 @@ import kafka.log.s3.operator.Writer; import org.apache.kafka.metadata.stream.ObjectUtils; -import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; //TODO: refactor to reduce duplicate code with ObjectWriter public class DataBlockWriter { @@ -48,34 +48,43 @@ public DataBlockWriter(long objectId, S3Operator s3Operator, int partSizeThresho String objectKey = ObjectUtils.genKey(0, "todocluster", objectId); this.partSizeThreshold = partSizeThreshold; waitingUploadBlocks = new LinkedList<>(); - waitingUploadBlockCfs = new HashMap<>(); + waitingUploadBlockCfs = new ConcurrentHashMap<>(); completedBlocks = new LinkedList<>(); writer = s3Operator.writer(objectKey); } + public long getObjectId() { + return objectId; + } + public CompletableFuture write(StreamDataBlock dataBlock) { CompletableFuture writeCf = new CompletableFuture<>(); waitingUploadBlockCfs.put(dataBlock, writeCf); waitingUploadBlocks.add(dataBlock); long waitingUploadSize = waitingUploadBlocks.stream().mapToLong(StreamDataBlock::getBlockSize).sum(); if (waitingUploadSize >= partSizeThreshold) { - CompositeByteBuf partBuf = Unpooled.compositeBuffer(); - for (StreamDataBlock block : waitingUploadBlocks) { - partBuf.addComponent(true, dataBlock.getDataCf().join()); - completedBlocks.add(block); - nextDataBlockPosition += block.getBlockSize(); - } - List blocks = new LinkedList<>(waitingUploadBlocks); - writer.write(partBuf).thenAccept(v -> { - for (StreamDataBlock block : blocks) { - waitingUploadBlockCfs.get(block).complete(null); - } - }); - waitingUploadBlocks.clear(); + uploadWaitingList(); } return writeCf; } + public void uploadWaitingList() { + CompositeByteBuf partBuf = Unpooled.compositeBuffer(); + for (StreamDataBlock block : waitingUploadBlocks) { + partBuf.addComponent(true, block.getDataCf().join()); + completedBlocks.add(block); + nextDataBlockPosition += block.getBlockSize(); + } + List blocks = new LinkedList<>(waitingUploadBlocks); + writer.write(partBuf).thenAccept(v -> { + for (StreamDataBlock block : blocks) { + waitingUploadBlockCfs.get(block).complete(null); + waitingUploadBlockCfs.remove(block); + } + }); + waitingUploadBlocks.clear(); + } + public CompletableFuture close() { CompositeByteBuf buf = Unpooled.compositeBuffer(); for (StreamDataBlock block : waitingUploadBlocks) { diff --git a/core/src/main/scala/kafka/log/s3/memory/MemoryMetadataManager.java b/core/src/main/scala/kafka/log/s3/memory/MemoryMetadataManager.java index 41fad49511..7bfa73daea 100644 --- a/core/src/main/scala/kafka/log/s3/memory/MemoryMetadataManager.java +++ b/core/src/main/scala/kafka/log/s3/memory/MemoryMetadataManager.java @@ -60,7 +60,7 @@ public class MemoryMetadataManager implements StreamManager, ObjectManager { private static final int MOCK_BROKER_ID = 0; private static final Logger LOGGER = LoggerFactory.getLogger(MemoryMetadataManager.class); private final EventDriver eventDriver; - private volatile long nextAssignedObjectId = 0; + private final AtomicLong nextAssignedObjectId = new AtomicLong(0L); private final Map objectsMetadata; private final AtomicLong nextAssignedStreamId = new AtomicLong(0L); private final Map streamsMetadata; @@ -134,13 +134,14 @@ public void start() { @Override public CompletableFuture prepareObject(int count, long ttl) { return this.submitEvent(() -> { - long objectRangeStart = this.nextAssignedObjectId; + List objectIds = new ArrayList<>(); for (int i = 0; i < count; i++) { - long objectId = this.nextAssignedObjectId++; + long objectId = this.nextAssignedObjectId.getAndIncrement(); + objectIds.add(objectId); S3Object object = prepareObject(objectId, ttl); this.objectsMetadata.put(objectId, object); } - return objectRangeStart; + return objectIds.get(0); }); } @@ -168,7 +169,7 @@ public CompletableFuture commitWALObject(CommitWALObjec MemoryBrokerWALMetadata walMetadata = this.brokerWALMetadata.computeIfAbsent(MOCK_BROKER_ID, k -> new MemoryBrokerWALMetadata(k)); Map> index = new HashMap<>(); - streamRanges.stream().forEach(range -> { + streamRanges.forEach(range -> { long streamId = range.getStreamId(); long startOffset = range.getStartOffset(); long endOffset = range.getEndOffset(); @@ -381,7 +382,7 @@ public EventDriver() { } public void start() { - this.service.submit(this::run); + this.service.submit(this); } public void stop() { diff --git a/core/src/test/java/kafka/log/s3/compact/CompactionAnalyzerTest.java b/core/src/test/java/kafka/log/s3/compact/CompactionAnalyzerTest.java index d208ac5f6c..288c829cd9 100644 --- a/core/src/test/java/kafka/log/s3/compact/CompactionAnalyzerTest.java +++ b/core/src/test/java/kafka/log/s3/compact/CompactionAnalyzerTest.java @@ -17,15 +17,10 @@ package kafka.log.s3.compact; -import kafka.log.s3.ObjectWriter; -import kafka.log.s3.TestUtils; import kafka.log.s3.compact.objects.CompactedObject; import kafka.log.s3.compact.objects.CompactedObjectBuilder; import kafka.log.s3.compact.objects.CompactionType; import kafka.log.s3.compact.objects.StreamDataBlock; -import kafka.log.s3.model.StreamRecordBatch; -import kafka.log.s3.operator.MemoryS3Operator; -import kafka.log.s3.operator.S3Operator; import org.apache.kafka.metadata.stream.S3ObjectMetadata; import org.apache.kafka.metadata.stream.S3ObjectType; import org.apache.kafka.metadata.stream.S3WALObject; @@ -42,119 +37,22 @@ import java.util.Map; @Tag("S3Unit") -public class CompactionAnalyzerTest { - private static final int BROKER_0 = 0; - private static final long STREAM_0 = 0; - private static final long STREAM_1 = 1; - private static final long STREAM_2 = 2; - private static final long OBJECT_0 = 0; - private static final long OBJECT_1 = 1; - private static final long OBJECT_2 = 2; - private static final long CACHE_SIZE = 1024; - private static final double EXECUTION_SCORE_THRESHOLD = 0.5; - private static final long STREAM_SPLIT_SIZE = 30; - private static final List S3_WAL_OBJECT_METADATA_LIST = new ArrayList<>(); - private CompactionAnalyzer compactionAnalyzer; - private S3Operator s3Operator; +public class CompactionAnalyzerTest extends CompactionTestBase { @BeforeEach public void setUp() throws Exception { - s3Operator = new MemoryS3Operator(); - // stream data for object 0 - ObjectWriter objectWriter = new ObjectWriter(OBJECT_0, s3Operator, 1024, 1024); - StreamRecordBatch r1 = new StreamRecordBatch(STREAM_0, 0, 0, 20, TestUtils.random(20)); - StreamRecordBatch r2 = new StreamRecordBatch(STREAM_1, 0, 30, 30, TestUtils.random(30)); - StreamRecordBatch r3 = new StreamRecordBatch(STREAM_2, 0, 30, 30, TestUtils.random(30)); - objectWriter.write(STREAM_0, List.of(r1)); - objectWriter.write(STREAM_1, List.of(r2)); - objectWriter.write(STREAM_2, List.of(r3)); - objectWriter.close().get(); - S3ObjectMetadata objectMetadata = new S3ObjectMetadata(OBJECT_0, objectWriter.size(), S3ObjectType.WAL); - Map> streamsIndex = Map.of( - (long) STREAM_0, List.of(new StreamOffsetRange(STREAM_0, 0, 20)), - (long) STREAM_1, List.of(new StreamOffsetRange(STREAM_1, 30, 60)), - (long) STREAM_2, List.of(new StreamOffsetRange(STREAM_2, 30, 60)) - ); - S3WALObject walObject = new S3WALObject(OBJECT_0, BROKER_0, streamsIndex, OBJECT_0); - S3_WAL_OBJECT_METADATA_LIST.add(new S3WALObjectMetadata(walObject, objectMetadata)); - - // stream data for object 1 - objectWriter = new ObjectWriter(OBJECT_1, s3Operator, 1024, 1024); - StreamRecordBatch r4 = new StreamRecordBatch(STREAM_0, 0, 20, 5, TestUtils.random(5)); - StreamRecordBatch r5 = new StreamRecordBatch(STREAM_1, 0, 60, 60, TestUtils.random(60)); - objectWriter.write(STREAM_0, List.of(r4)); - objectWriter.write(STREAM_1, List.of(r5)); - objectWriter.close().get(); - objectMetadata = new S3ObjectMetadata(OBJECT_1, objectWriter.size(), S3ObjectType.WAL); - streamsIndex = Map.of( - (long) STREAM_0, List.of(new StreamOffsetRange(STREAM_0, 20, 25)), - (long) STREAM_1, List.of(new StreamOffsetRange(STREAM_1, 60, 120)) - ); - walObject = new S3WALObject(OBJECT_1, BROKER_0, streamsIndex, OBJECT_1); - S3_WAL_OBJECT_METADATA_LIST.add(new S3WALObjectMetadata(walObject, objectMetadata)); - - // stream data for object 0 - objectWriter = new ObjectWriter(OBJECT_2, s3Operator, 1024, 1024); - // redundant record - StreamRecordBatch r6 = new StreamRecordBatch(STREAM_1, 0, 260, 20, TestUtils.random(20)); - StreamRecordBatch r7 = new StreamRecordBatch(STREAM_1, 0, 400, 100, TestUtils.random(100)); - StreamRecordBatch r8 = new StreamRecordBatch(STREAM_2, 0, 230, 40, TestUtils.random(40)); - objectWriter.write(STREAM_1, List.of(r6)); - objectWriter.write(STREAM_1, List.of(r7)); - objectWriter.write(STREAM_2, List.of(r8)); - objectWriter.close().get(); - objectMetadata = new S3ObjectMetadata(OBJECT_2, objectWriter.size(), S3ObjectType.WAL); - streamsIndex = Map.of( - (long) STREAM_1, List.of(new StreamOffsetRange(STREAM_1, 400, 500)), - (long) STREAM_2, List.of(new StreamOffsetRange(STREAM_2, 230, 270)) - ); - walObject = new S3WALObject(OBJECT_2, BROKER_0, streamsIndex, OBJECT_2); - S3_WAL_OBJECT_METADATA_LIST.add(new S3WALObjectMetadata(walObject, objectMetadata)); + super.setUp(); } @AfterEach public void tearDown() { - S3_WAL_OBJECT_METADATA_LIST.clear(); - } - - private boolean compare(StreamDataBlock block1, StreamDataBlock block2) { - return block1.getStreamId() == block2.getStreamId() && - block1.getStartOffset() == block2.getStartOffset() && - block1.getEndOffset() == block2.getEndOffset() && - block1.getRecordCount() == block2.getRecordCount() && - block1.getObjectId() == block2.getObjectId(); - } - - private boolean compare(List streamDataBlocks1, List streamDataBlocks2) { - if (streamDataBlocks1.size() != streamDataBlocks2.size()) { - return false; - } - for (int i = 0; i < streamDataBlocks1.size(); i++) { - if (!compare(streamDataBlocks1.get(i), streamDataBlocks2.get(i))) { - return false; - } - } - return true; - } - - private boolean compare(CompactedObjectBuilder builder1, CompactedObjectBuilder builder2) { - if (builder1.type() != builder2.type()) { - return false; - } - return compare(builder1.streamDataBlocks(), builder2.streamDataBlocks()); - } - - private boolean compare(CompactedObject compactedObject1, CompactedObject compactedObject2) { - if (compactedObject1.type() != compactedObject2.type()) { - return false; - } - return compare(compactedObject1.streamDataBlocks(), compactedObject2.streamDataBlocks()); + super.tearDown(); } @Test public void testReadObjectIndices() { - this.compactionAnalyzer = new CompactionAnalyzer(CACHE_SIZE, EXECUTION_SCORE_THRESHOLD, STREAM_SPLIT_SIZE, s3Operator); - List streamDataBlocks = this.compactionAnalyzer.blockWaitObjectIndices(S3_WAL_OBJECT_METADATA_LIST); + CompactionAnalyzer compactionAnalyzer = new CompactionAnalyzer(CACHE_SIZE, EXECUTION_SCORE_THRESHOLD, STREAM_SPLIT_SIZE, s3Operator); + List streamDataBlocks = compactionAnalyzer.blockWaitObjectIndices(S3_WAL_OBJECT_METADATA_LIST); List expectedBlocks = 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), @@ -170,16 +68,16 @@ public void testReadObjectIndices() { @Test public void testShouldCompact() { - this.compactionAnalyzer = new CompactionAnalyzer(CACHE_SIZE, EXECUTION_SCORE_THRESHOLD, STREAM_SPLIT_SIZE, s3Operator); - List streamDataBlocks = this.compactionAnalyzer.blockWaitObjectIndices(S3_WAL_OBJECT_METADATA_LIST); - Assertions.assertTrue(this.compactionAnalyzer.shouldCompact(streamDataBlocks)); + CompactionAnalyzer compactionAnalyzer = new CompactionAnalyzer(CACHE_SIZE, EXECUTION_SCORE_THRESHOLD, STREAM_SPLIT_SIZE, s3Operator); + List streamDataBlocks = compactionAnalyzer.blockWaitObjectIndices(S3_WAL_OBJECT_METADATA_LIST); + Assertions.assertTrue(compactionAnalyzer.shouldCompact(streamDataBlocks)); } @Test public void testSortStreamRangePositions() { - this.compactionAnalyzer = new CompactionAnalyzer(CACHE_SIZE, EXECUTION_SCORE_THRESHOLD, STREAM_SPLIT_SIZE, s3Operator); - List streamDataBlocks = this.compactionAnalyzer.blockWaitObjectIndices(S3_WAL_OBJECT_METADATA_LIST); - List sortedStreamDataBlocks = this.compactionAnalyzer.sortStreamRangePositions(streamDataBlocks); + CompactionAnalyzer compactionAnalyzer = new CompactionAnalyzer(CACHE_SIZE, EXECUTION_SCORE_THRESHOLD, STREAM_SPLIT_SIZE, s3Operator); + List streamDataBlocks = compactionAnalyzer.blockWaitObjectIndices(S3_WAL_OBJECT_METADATA_LIST); + List sortedStreamDataBlocks = compactionAnalyzer.sortStreamRangePositions(streamDataBlocks); List expectedBlocks = List.of( new StreamDataBlock(STREAM_0, 0, 20, 0, OBJECT_0, -1, -1, 1), new StreamDataBlock(STREAM_0, 20, 25, 0, OBJECT_1, -1, -1, 1), @@ -195,8 +93,8 @@ public void testSortStreamRangePositions() { @Test public void testBuildCompactedObject1() { - this.compactionAnalyzer = new CompactionAnalyzer(CACHE_SIZE, EXECUTION_SCORE_THRESHOLD, 100, s3Operator); - List compactedObjectBuilders = this.compactionAnalyzer.buildCompactedObjects(S3_WAL_OBJECT_METADATA_LIST); + CompactionAnalyzer compactionAnalyzer = new CompactionAnalyzer(CACHE_SIZE, EXECUTION_SCORE_THRESHOLD, 100, s3Operator); + List compactedObjectBuilders = compactionAnalyzer.buildCompactedObjects(S3_WAL_OBJECT_METADATA_LIST); List expectedCompactedObject = List.of( new CompactedObjectBuilder() .setType(CompactionType.COMPACT) @@ -222,8 +120,8 @@ public void testBuildCompactedObject1() { @Test public void testBuildCompactedObject2() { - this.compactionAnalyzer = new CompactionAnalyzer(CACHE_SIZE, EXECUTION_SCORE_THRESHOLD, 30, s3Operator); - List compactedObjectBuilders = this.compactionAnalyzer.buildCompactedObjects(S3_WAL_OBJECT_METADATA_LIST); + CompactionAnalyzer compactionAnalyzer = new CompactionAnalyzer(CACHE_SIZE, EXECUTION_SCORE_THRESHOLD, 30, s3Operator); + List compactedObjectBuilders = compactionAnalyzer.buildCompactedObjects(S3_WAL_OBJECT_METADATA_LIST); List expectedCompactedObject = List.of( new CompactedObjectBuilder() .setType(CompactionType.SPLIT) @@ -249,8 +147,8 @@ public void testBuildCompactedObject2() { @Test public void testCompactionPlans1() { - this.compactionAnalyzer = new CompactionAnalyzer(CACHE_SIZE, EXECUTION_SCORE_THRESHOLD, 100, s3Operator); - List compactionPlans = this.compactionAnalyzer.analyze(S3_WAL_OBJECT_METADATA_LIST); + CompactionAnalyzer compactionAnalyzer = new CompactionAnalyzer(CACHE_SIZE, EXECUTION_SCORE_THRESHOLD, 100, s3Operator); + List compactionPlans = compactionAnalyzer.analyze(S3_WAL_OBJECT_METADATA_LIST); Assertions.assertEquals(1, compactionPlans.size()); List expectCompactedObjects = List.of( new CompactedObjectBuilder() @@ -292,10 +190,7 @@ public void testCompactionPlans1() { } } - @Test - public void testCompactionPlans2() { - this.compactionAnalyzer = new CompactionAnalyzer(300, EXECUTION_SCORE_THRESHOLD, 100, s3Operator); - List compactionPlans = this.compactionAnalyzer.analyze(S3_WAL_OBJECT_METADATA_LIST); + private void checkCompactionPlan2(List compactionPlans) { Assertions.assertEquals(2, compactionPlans.size()); // first iteration @@ -355,7 +250,20 @@ public void testCompactionPlans2() { } @Test - public void testCompactionPlanWithException() { + public void testCompactionPlans2() { + CompactionAnalyzer compactionAnalyzer = new CompactionAnalyzer(300, EXECUTION_SCORE_THRESHOLD, 100, s3Operator); + List compactionPlans = compactionAnalyzer.analyze(S3_WAL_OBJECT_METADATA_LIST); + checkCompactionPlan2(compactionPlans); + } + @Test + public void testCompactionPlansWithInvalidObject() { + CompactionAnalyzer compactionAnalyzer = new CompactionAnalyzer(300, EXECUTION_SCORE_THRESHOLD, 100, s3Operator); + List s3ObjectMetadata = new ArrayList<>(S3_WAL_OBJECT_METADATA_LIST); + s3ObjectMetadata.add(new S3WALObjectMetadata(new S3WALObject(100, 0, Map.of( + STREAM_2, List.of(new StreamOffsetRange(STREAM_2, 1000, 1200)) + ), 0), new S3ObjectMetadata(100, 0, S3ObjectType.WAL))); + List compactionPlans = compactionAnalyzer.analyze(s3ObjectMetadata); + checkCompactionPlan2(compactionPlans); } } diff --git a/core/src/test/java/kafka/log/s3/compact/CompactionManagerTest.java b/core/src/test/java/kafka/log/s3/compact/CompactionManagerTest.java index 586981dcbb..8b23e3f4c9 100644 --- a/core/src/test/java/kafka/log/s3/compact/CompactionManagerTest.java +++ b/core/src/test/java/kafka/log/s3/compact/CompactionManagerTest.java @@ -17,30 +17,90 @@ package kafka.log.s3.compact; +import kafka.log.s3.compact.objects.CompactionType; +import kafka.log.s3.compact.objects.StreamDataBlock; import kafka.log.s3.memory.MemoryMetadataManager; import kafka.log.s3.metadata.StreamMetadataManager; +import kafka.log.s3.objects.CommitWALObjectRequest; +import kafka.log.s3.objects.StreamObject; import kafka.log.s3.operator.MemoryS3Operator; import kafka.server.KafkaConfig; +import org.apache.kafka.metadata.stream.S3ObjectMetadata; +import org.apache.kafka.metadata.stream.S3ObjectType; +import org.apache.kafka.metadata.stream.S3WALObject; +import org.apache.kafka.metadata.stream.S3WALObjectMetadata; +import org.apache.kafka.metadata.stream.StreamOffsetRange; +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.Mockito; -public class CompactionManagerTest { +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; + +public class CompactionManagerTest extends CompactionTestBase { private static final int BROKER0 = 0; + private StreamMetadataManager streamMetadataManager; + private CompactionAnalyzer compactionAnalyzer; private CompactionManager compactionManager; @BeforeEach - public void setUp() { - // mock parameters for CompactionManager + public void setUp() throws Exception { + super.setUp(); KafkaConfig config = Mockito.mock(KafkaConfig.class); Mockito.when(config.brokerId()).thenReturn(BROKER0); - StreamMetadataManager streamMetadataManager = Mockito.mock(StreamMetadataManager.class); - this.compactionManager = new CompactionManager(config, new MemoryMetadataManager(), streamMetadataManager, new MemoryS3Operator()); + Mockito.when(config.s3ObjectCompactionNWInBandwidth()).thenReturn(500L); + Mockito.when(config.s3ObjectCompactionNWOutBandwidth()).thenReturn(500L); + Mockito.when(config.s3ObjectCompactionUploadConcurrency()).thenReturn(3); + Mockito.when(config.s3ObjectPartSize()).thenReturn(100); + Mockito.when(config.s3ObjectCompactionCacheSize()).thenReturn(300L); + Mockito.when(config.s3ObjectCompactionExecutionScoreThreshold()).thenReturn(0.5); + Mockito.when(config.s3ObjectCompactionStreamSplitSize()).thenReturn(100L); + streamMetadataManager = Mockito.mock(StreamMetadataManager.class); + Mockito.when(streamMetadataManager.getWALObjects()).thenReturn(S3_WAL_OBJECT_METADATA_LIST); + compactionAnalyzer = new CompactionAnalyzer(config.s3ObjectCompactionCacheSize(), + config.s3ObjectCompactionExecutionScoreThreshold(), config.s3ObjectCompactionStreamSplitSize(), s3Operator); + compactionManager = new CompactionManager(config, objectManager, streamMetadataManager, s3Operator); + } + + @AfterEach + public void tearDown() { + super.tearDown(); } - // Test compacting multiple WAL objects @Test public void testCompact() { + List s3ObjectMetadata = this.streamMetadataManager.getWALObjects(); + List compactionPlans = this.compactionAnalyzer.analyze(s3ObjectMetadata); + Assertions.assertEquals(2, compactionPlans.size()); + CommitWALObjectRequest request = compactionManager.buildCompactRequest(compactionPlans, s3ObjectMetadata); + + Assertions.assertEquals(List.of(OBJECT_0, OBJECT_1, OBJECT_2), request.getCompactedObjectIds()); + Assertions.assertEquals(OBJECT_0, request.getOrderId()); + Assertions.assertTrue(request.getObjectId() > OBJECT_2); + request.getStreamObjects().forEach(s -> Assertions.assertTrue(s.getObjectId() > OBJECT_2)); + Assertions.assertEquals(3, request.getStreamObjects().size()); + Assertions.assertEquals(2, request.getStreamRanges().size()); + + List walStreamDataBlocks = compactionPlans.stream() + .map(p -> p.compactedObjects().stream() + .filter(c -> c.type() == CompactionType.COMPACT) + .flatMap(c -> c.streamDataBlocks().stream()).collect(Collectors.toList())) + .flatMap(List::stream).collect(Collectors.toList()); + long expectedWALSize = calculateObjectSize(walStreamDataBlocks); + Assertions.assertEquals(expectedWALSize, request.getObjectSize()); + } + + @Test + public void testCompactNoneExistObjects() { + List s3ObjectMetadata = this.streamMetadataManager.getWALObjects(); + List compactionPlans = this.compactionAnalyzer.analyze(s3ObjectMetadata); + s3Operator.delete(s3ObjectMetadata.get(0).getObjectMetadata().key()).join(); + Assertions.assertThrowsExactly(IllegalArgumentException.class, () -> compactionManager.buildCompactRequest(compactionPlans, s3ObjectMetadata)); } } diff --git a/core/src/test/java/kafka/log/s3/compact/CompactionTestBase.java b/core/src/test/java/kafka/log/s3/compact/CompactionTestBase.java new file mode 100644 index 0000000000..084df2a4a2 --- /dev/null +++ b/core/src/test/java/kafka/log/s3/compact/CompactionTestBase.java @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.s3.compact; + +import com.automq.elasticstream.client.flatc.header.ObjectMetadata; +import kafka.log.s3.ObjectWriter; +import kafka.log.s3.TestUtils; +import kafka.log.s3.compact.objects.CompactedObject; +import kafka.log.s3.compact.objects.CompactedObjectBuilder; +import kafka.log.s3.compact.objects.StreamDataBlock; +import kafka.log.s3.memory.MemoryMetadataManager; +import kafka.log.s3.model.StreamRecordBatch; +import kafka.log.s3.operator.MemoryS3Operator; +import kafka.log.s3.operator.S3Operator; +import org.apache.kafka.metadata.stream.S3ObjectMetadata; +import org.apache.kafka.metadata.stream.S3ObjectType; +import org.apache.kafka.metadata.stream.S3WALObject; +import org.apache.kafka.metadata.stream.S3WALObjectMetadata; +import org.apache.kafka.metadata.stream.StreamOffsetRange; +import org.junit.jupiter.api.Assertions; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +public class CompactionTestBase { + protected static final int BROKER_0 = 0; + protected static final long STREAM_0 = 0; + protected static final long STREAM_1 = 1; + protected static final long STREAM_2 = 2; + protected static final long OBJECT_0 = 0; + protected static final long OBJECT_1 = 1; + protected static final long OBJECT_2 = 2; + protected static final long CACHE_SIZE = 1024; + protected static final double EXECUTION_SCORE_THRESHOLD = 0.5; + protected static final long STREAM_SPLIT_SIZE = 30; + protected static final List S3_WAL_OBJECT_METADATA_LIST = new ArrayList<>(); + protected MemoryMetadataManager objectManager; + protected S3Operator s3Operator; + + public void setUp() throws Exception { + objectManager = new MemoryMetadataManager(); + objectManager.start(); + s3Operator = new MemoryS3Operator(); + // stream data for object 0 + objectManager.prepareObject(1, TimeUnit.MINUTES.toMillis(30)).thenAccept(objectId -> { + Assertions.assertEquals(OBJECT_0, objectId); + ObjectWriter objectWriter = new ObjectWriter(objectId, s3Operator, 1024, 1024); + StreamRecordBatch r1 = new StreamRecordBatch(STREAM_0, 0, 0, 20, TestUtils.random(20)); + StreamRecordBatch r2 = new StreamRecordBatch(STREAM_1, 0, 30, 30, TestUtils.random(30)); + StreamRecordBatch r3 = new StreamRecordBatch(STREAM_2, 0, 30, 30, TestUtils.random(30)); + objectWriter.write(STREAM_0, List.of(r1)); + objectWriter.write(STREAM_1, List.of(r2)); + objectWriter.write(STREAM_2, List.of(r3)); + objectWriter.close().join(); + S3ObjectMetadata objectMetadata = new S3ObjectMetadata(OBJECT_0, objectWriter.size(), S3ObjectType.WAL); + Map> streamsIndex = Map.of( + STREAM_0, List.of(new StreamOffsetRange(STREAM_0, 0, 20)), + STREAM_1, List.of(new StreamOffsetRange(STREAM_1, 30, 60)), + STREAM_2, List.of(new StreamOffsetRange(STREAM_2, 30, 60)) + ); + S3WALObject walObject = new S3WALObject(OBJECT_0, BROKER_0, streamsIndex, OBJECT_0); + S3_WAL_OBJECT_METADATA_LIST.add(new S3WALObjectMetadata(walObject, objectMetadata)); + }).join(); + + // stream data for object 1 + objectManager.prepareObject(1, TimeUnit.MINUTES.toMillis(30)).thenAccept(objectId -> { + Assertions.assertEquals(OBJECT_1, objectId); + ObjectWriter objectWriter = new ObjectWriter(OBJECT_1, s3Operator, 1024, 1024); + StreamRecordBatch r4 = new StreamRecordBatch(STREAM_0, 0, 20, 5, TestUtils.random(5)); + StreamRecordBatch r5 = new StreamRecordBatch(STREAM_1, 0, 60, 60, TestUtils.random(60)); + objectWriter.write(STREAM_0, List.of(r4)); + objectWriter.write(STREAM_1, List.of(r5)); + objectWriter.close().join(); + S3ObjectMetadata objectMetadata = new S3ObjectMetadata(OBJECT_1, objectWriter.size(), S3ObjectType.WAL); + Map> streamsIndex = Map.of( + STREAM_0, List.of(new StreamOffsetRange(STREAM_0, 20, 25)), + STREAM_1, List.of(new StreamOffsetRange(STREAM_1, 60, 120)) + ); + S3WALObject walObject = new S3WALObject(OBJECT_1, BROKER_0, streamsIndex, OBJECT_1); + S3_WAL_OBJECT_METADATA_LIST.add(new S3WALObjectMetadata(walObject, objectMetadata)); + }).join(); + + // stream data for object 2 + objectManager.prepareObject(1, TimeUnit.MINUTES.toMillis(30)).thenAccept(objectId -> { + Assertions.assertEquals(OBJECT_2, objectId); + ObjectWriter objectWriter = new ObjectWriter(OBJECT_2, s3Operator, 1024, 1024); + // redundant record + StreamRecordBatch r6 = new StreamRecordBatch(STREAM_1, 0, 260, 20, TestUtils.random(20)); + StreamRecordBatch r7 = new StreamRecordBatch(STREAM_1, 0, 400, 100, TestUtils.random(100)); + StreamRecordBatch r8 = new StreamRecordBatch(STREAM_2, 0, 230, 40, TestUtils.random(40)); + objectWriter.write(STREAM_1, List.of(r6)); + objectWriter.write(STREAM_1, List.of(r7)); + objectWriter.write(STREAM_2, List.of(r8)); + objectWriter.close().join(); + S3ObjectMetadata objectMetadata = new S3ObjectMetadata(OBJECT_2, objectWriter.size(), S3ObjectType.WAL); + Map> streamsIndex = Map.of( + STREAM_1, List.of(new StreamOffsetRange(STREAM_1, 400, 500)), + STREAM_2, List.of(new StreamOffsetRange(STREAM_2, 230, 270)) + ); + S3WALObject walObject = new S3WALObject(OBJECT_2, BROKER_0, streamsIndex, OBJECT_2); + S3_WAL_OBJECT_METADATA_LIST.add(new S3WALObjectMetadata(walObject, objectMetadata)); + }).join(); + } + + public void tearDown() { + S3_WAL_OBJECT_METADATA_LIST.clear(); + objectManager.shutdown(); + } + + protected boolean compare(StreamDataBlock block1, StreamDataBlock block2) { + return block1.getStreamId() == block2.getStreamId() && + block1.getStartOffset() == block2.getStartOffset() && + block1.getEndOffset() == block2.getEndOffset() && + block1.getRecordCount() == block2.getRecordCount() && + block1.getObjectId() == block2.getObjectId(); + } + + protected boolean compare(List streamDataBlocks1, List streamDataBlocks2) { + if (streamDataBlocks1.size() != streamDataBlocks2.size()) { + return false; + } + for (int i = 0; i < streamDataBlocks1.size(); i++) { + if (!compare(streamDataBlocks1.get(i), streamDataBlocks2.get(i))) { + return false; + } + } + return true; + } + + protected boolean compare(CompactedObjectBuilder builder1, CompactedObjectBuilder builder2) { + if (builder1.type() != builder2.type()) { + return false; + } + return compare(builder1.streamDataBlocks(), builder2.streamDataBlocks()); + } + + protected boolean compare(CompactedObject compactedObject1, CompactedObject compactedObject2) { + if (compactedObject1.type() != compactedObject2.type()) { + return false; + } + return compare(compactedObject1.streamDataBlocks(), compactedObject2.streamDataBlocks()); + } + + protected long calculateObjectSize(List streamDataBlocks) { + long bodySize = streamDataBlocks.stream().mapToLong(StreamDataBlock::getBlockSize).sum(); + long indexBlockSize = 4 + 40L * streamDataBlocks.size(); + long tailSize = 48; + return bodySize + indexBlockSize + tailSize; + } +} diff --git a/core/src/test/java/kafka/log/s3/compact/CompactionUploaderTest.java b/core/src/test/java/kafka/log/s3/compact/CompactionUploaderTest.java new file mode 100644 index 0000000000..99af740b9d --- /dev/null +++ b/core/src/test/java/kafka/log/s3/compact/CompactionUploaderTest.java @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.s3.compact; + +import kafka.log.s3.TestUtils; +import kafka.log.s3.compact.objects.CompactedObject; +import kafka.log.s3.compact.objects.CompactionType; +import kafka.log.s3.compact.objects.StreamDataBlock; +import kafka.log.s3.compact.operator.DataBlockReader; +import kafka.log.s3.memory.MemoryMetadataManager; +import kafka.log.s3.objects.ObjectStreamRange; +import kafka.log.s3.objects.StreamObject; +import kafka.log.s3.operator.MemoryS3Operator; +import kafka.server.KafkaConfig; +import org.apache.kafka.metadata.stream.S3ObjectMetadata; +import org.apache.kafka.metadata.stream.S3ObjectType; +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 java.util.List; +import java.util.concurrent.CompletableFuture; + +@Timeout(30) +@Tag("S3Unit") +public class CompactionUploaderTest extends CompactionTestBase { + + private MemoryMetadataManager objectManager; + private KafkaConfig config; + + @BeforeEach + public void setUp() throws Exception { + s3Operator = new MemoryS3Operator(); + objectManager = new MemoryMetadataManager(); + objectManager.start(); + config = Mockito.mock(KafkaConfig.class); + Mockito.when(config.s3ObjectCompactionNWOutBandwidth()).thenReturn(500L); + Mockito.when(config.s3ObjectCompactionUploadConcurrency()).thenReturn(3); + Mockito.when(config.s3ObjectPartSize()).thenReturn(100); + } + + @AfterEach + public void tearDown() { + objectManager.shutdown(); + } + + @Test + public void testWriteWALObject() { + List streamDataBlocks = List.of( + new StreamDataBlock(STREAM_0, 0, 20, -1, -1, 0, 20, 1), + new StreamDataBlock(STREAM_0, 20, 25, -1, -1, 20, 5, 1), + new StreamDataBlock(STREAM_2, 40, 120, -1, -1, 25, 80, 1), + new StreamDataBlock(STREAM_2, 120, 150, -1, -1, 105, 30, 1)); + CompactedObject compactedObject = new CompactedObject(CompactionType.COMPACT, streamDataBlocks, 100); + List result = CompactionUtils.buildObjectStreamRange(compactedObject); + Assertions.assertEquals(2, result.size()); + Assertions.assertEquals(STREAM_0, result.get(0).getStreamId()); + Assertions.assertEquals(0, result.get(0).getStartOffset()); + Assertions.assertEquals(25, result.get(0).getEndOffset()); + Assertions.assertEquals(STREAM_2, result.get(1).getStreamId()); + Assertions.assertEquals(40, result.get(1).getStartOffset()); + Assertions.assertEquals(150, result.get(1).getEndOffset()); + + CompactionUploader uploader = new CompactionUploader(objectManager, s3Operator, config); + CompletableFuture> cf = uploader.writeWALObject(compactedObject); + for (StreamDataBlock streamDataBlock : streamDataBlocks) { + streamDataBlock.getDataCf().complete(TestUtils.random((int) streamDataBlock.getStreamRangeSize())); + } + CompletableFuture writeCf = cf.join(); + uploader.forceUploadWAL(); + writeCf.join(); + long walObjectSize = uploader.completeWAL(); + System.out.printf("write size: %d%n", walObjectSize); + Assertions.assertEquals(walObjectSize, calculateObjectSize(streamDataBlocks)); + + //check s3 object + DataBlockReader reader = new DataBlockReader(new S3ObjectMetadata(OBJECT_0, walObjectSize, S3ObjectType.WAL), s3Operator); + reader.parseDataBlockIndex(); + List streamDataBlocksFromS3 = reader.getDataBlockIndex().join(); + for (int i = 0; i < streamDataBlocks.size(); i++) { + compare(streamDataBlocksFromS3.get(i), streamDataBlocks.get(i)); + } + List blockIndices = CompactionUtils.buildBlockIndicesFromStreamDataBlock(streamDataBlocksFromS3); + List dataBlocks = reader.readBlocks(blockIndices).join(); + for (int i = 0; i < dataBlocks.size(); i++) { + Assertions.assertEquals(streamDataBlocks.get(i).getDataCf().join(), dataBlocks.get(i).buffer()); + } + } + + @Test + public void testWriteStreamObject() { + List streamDataBlocks = List.of( + new StreamDataBlock(STREAM_0, 0, 60, -1, -1, 0, 60, 1), + new StreamDataBlock(STREAM_0, 60, 120, -1, -1, 60, 60, 1)); + CompactedObject compactedObject = new CompactedObject(CompactionType.SPLIT, streamDataBlocks, 100); + + CompactionUploader uploader = new CompactionUploader(objectManager, s3Operator, config); + CompletableFuture cf = uploader.writeStreamObject(compactedObject); + for (StreamDataBlock streamDataBlock : streamDataBlocks) { + streamDataBlock.getDataCf().complete(TestUtils.random((int) streamDataBlock.getStreamRangeSize())); + } + StreamObject streamObject = cf.join(); + System.out.printf("write size: %d%n", streamObject.getObjectSize()); + Assertions.assertEquals(streamObject.getObjectSize(), calculateObjectSize(streamDataBlocks)); + + //check s3 object + DataBlockReader reader = new DataBlockReader(new S3ObjectMetadata(OBJECT_0, streamObject.getObjectSize(), S3ObjectType.STREAM), s3Operator); + reader.parseDataBlockIndex(); + List streamDataBlocksFromS3 = reader.getDataBlockIndex().join(); + for (int i = 0; i < streamDataBlocks.size(); i++) { + compare(streamDataBlocksFromS3.get(i), streamDataBlocks.get(i)); + } + List blockIndices = CompactionUtils.buildBlockIndicesFromStreamDataBlock(streamDataBlocksFromS3); + List dataBlocks = reader.readBlocks(blockIndices).join(); + for (int i = 0; i < dataBlocks.size(); i++) { + Assertions.assertEquals(streamDataBlocks.get(i).getDataCf().join(), dataBlocks.get(i).buffer()); + } + } +} From dfa40d3d2279ffd424c9c375036b45edd373f38b Mon Sep 17 00:00:00 2001 From: Shichao Nie Date: Fri, 8 Sep 2023 17:29:52 +0800 Subject: [PATCH 3/7] Merge branch 'develop' into major_compact Signed-off-by: Shichao Nie --- build.gradle | 5 +- .../s3/RedundantOperationException.java | 27 ++ .../apache/kafka/common/protocol/Errors.java | 3 +- .../kafka/log/es/AlwaysSuccessClient.java | 95 +++---- .../kafka/log/es/DefaultAppendResult.java | 38 +++ .../log/es/DefaultElasticStreamSlice.java | 14 +- .../kafka/log/es/DefaultRecordBatch.java | 61 +++++ .../log/es/ElasticLeaderEpochCheckpoint.scala | 16 +- .../es/ElasticLeaderEpochCheckpointMeta.java | 13 +- .../main/scala/kafka/log/es/ElasticLog.scala | 57 ++--- .../kafka/log/es/ElasticLogFileRecords.java | 15 +- .../kafka/log/es/ElasticLogManager.scala | 49 ++-- .../scala/kafka/log/es/ElasticLogMeta.java | 9 +- .../kafka/log/es/ElasticLogSegment.scala | 3 +- .../kafka/log/es/ElasticLogStreamManager.java | 5 +- .../kafka/log/es/ElasticPartitionMeta.java | 3 +- .../log/es/ElasticProducerStateManager.scala | 231 +++++++++--------- .../kafka/log/es/ElasticRedisClient.java | 22 +- .../kafka/log/es/ElasticStreamSlice.java | 11 +- .../log/es/ElasticStreamSliceManager.java | 4 +- .../kafka/log/es/ElasticUnifiedLog.scala | 8 +- .../main/scala/kafka/log/es/LazyStream.java | 18 +- .../main/scala/kafka/log/es/MemoryClient.java | 22 +- .../main/scala/kafka/log/es/MetaStream.java | 24 +- .../kafka/log/es/RawPayloadRecordBatch.java | 2 +- .../log/es/RecordBatchWithContextWrapper.java | 13 +- .../log/es/SeparateSlowAndQuickFetchHint.java | 1 + .../main/scala/kafka/log/es/SliceRange.java | 3 +- .../kafka/log/es/StreamSliceSupplier.java | 1 + .../scala/kafka/log/es/api/AppendResult.java | 32 +++ .../main/scala/kafka/log/es/api/Client.java | 37 +++ .../kafka/log/es/api/CreateStreamOptions.java | 57 +++++ .../es/api/ElasticStreamClientException.java | 36 +++ .../scala/kafka/log/es/api/ErrorCode.java | 24 ++ .../scala/kafka/log/es/api/FetchResult.java | 36 +++ .../main/scala/kafka/log/es/api/KVClient.java | 50 ++++ .../main/scala/kafka/log/es/api/KeyValue.java | 57 +++++ .../kafka/log/es/api/OpenStreamOptions.java | 95 +++++++ .../scala/kafka/log/es/api/RecordBatch.java | 55 +++++ .../log/es/api/RecordBatchWithContext.java | 35 +++ .../main/scala/kafka/log/es/api/Stream.java | 81 ++++++ .../scala/kafka/log/es/api/StreamClient.java | 42 ++++ .../log/es/client/ClientFactoryProxy.java | 8 +- .../kafka/log/es/client/es/ClientFactory.java | 41 ---- .../log/es/client/memory/ClientFactory.java | 4 +- .../log/es/client/redis/ClientFactory.java | 2 +- .../kafka/log/es/client/s3/ClientFactory.java | 2 +- .../scala/kafka/log/es/utils/Arguments.java | 34 +++ .../scala/kafka/log/es/utils/Threads.java | 40 +++ .../kafka/log/s3/ControllerKVClient.java | 4 +- .../scala/kafka/log/s3/DefaultS3Client.java | 6 +- .../main/scala/kafka/log/s3/S3Storage.java | 96 +++++--- .../src/main/scala/kafka/log/s3/S3Stream.java | 16 +- .../scala/kafka/log/s3/S3StreamClient.java | 8 +- .../s3/metadata/StreamMetadataManager.java | 20 ++ .../kafka/log/s3/model/StreamRecordBatch.java | 2 +- .../s3/objects/CommitWALObjectRequest.java | 14 +- .../s3/objects/ControllerObjectManager.java | 24 +- .../log/s3/operator/DefaultS3Operator.java | 12 +- .../main/scala/kafka/server/KafkaConfig.scala | 13 +- .../kafka/log/es/AlwaysSuccessClientTest.java | 46 ++-- .../java/kafka/log/s3/DefaultRecordBatch.java | 2 +- .../log/s3/DefaultRecordBatchWithContext.java | 5 +- .../test/java/kafka/log/s3/S3StorageTest.java | 6 +- .../java/kafka/log/s3/S3StreamMemoryTest.java | 49 ++-- .../test/java/kafka/log/s3/S3StreamTest.java | 4 +- .../log/s3/StreamMetadataManagerTest.java | 37 +-- .../kafka/log/s3/StreamObjectCopyerTest.java | 8 +- .../unit/kafka/log/es/ElasticLogTest.scala | 8 - gradle/dependencies.gradle | 4 +- .../stream/S3ObjectControlManager.java | 18 +- .../stream/StreamControlManager.java | 78 +++++- .../kafka/image/S3StreamsMetadataImage.java | 43 +++- .../kafka/metadata/stream/S3StreamObject.java | 18 +- .../stream/S3StreamObjectMetadata.java | 53 ++++ .../metadata/stream/StreamOffsetRange.java | 5 +- .../S3ObjectControlManagerTest.java | 16 +- .../controller/StreamControlManagerTest.java | 138 ++++++++++- 78 files changed, 1720 insertions(+), 574 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/s3/RedundantOperationException.java create mode 100644 core/src/main/scala/kafka/log/es/DefaultAppendResult.java create mode 100644 core/src/main/scala/kafka/log/es/DefaultRecordBatch.java create mode 100644 core/src/main/scala/kafka/log/es/api/AppendResult.java create mode 100644 core/src/main/scala/kafka/log/es/api/Client.java create mode 100644 core/src/main/scala/kafka/log/es/api/CreateStreamOptions.java create mode 100644 core/src/main/scala/kafka/log/es/api/ElasticStreamClientException.java create mode 100644 core/src/main/scala/kafka/log/es/api/ErrorCode.java create mode 100644 core/src/main/scala/kafka/log/es/api/FetchResult.java create mode 100644 core/src/main/scala/kafka/log/es/api/KVClient.java create mode 100644 core/src/main/scala/kafka/log/es/api/KeyValue.java create mode 100644 core/src/main/scala/kafka/log/es/api/OpenStreamOptions.java create mode 100644 core/src/main/scala/kafka/log/es/api/RecordBatch.java create mode 100644 core/src/main/scala/kafka/log/es/api/RecordBatchWithContext.java create mode 100644 core/src/main/scala/kafka/log/es/api/Stream.java create mode 100644 core/src/main/scala/kafka/log/es/api/StreamClient.java delete mode 100644 core/src/main/scala/kafka/log/es/client/es/ClientFactory.java create mode 100644 core/src/main/scala/kafka/log/es/utils/Arguments.java create mode 100644 core/src/main/scala/kafka/log/es/utils/Threads.java create mode 100644 metadata/src/main/java/org/apache/kafka/metadata/stream/S3StreamObjectMetadata.java diff --git a/build.gradle b/build.gradle index ec4c24c59a..77ae9f5731 100644 --- a/build.gradle +++ b/build.gradle @@ -1014,12 +1014,11 @@ project(':core') { implementation libs.commonsCli // implementation libs.elasticstream - implementation(libs.esClient) { - exclude group: 'org.slf4j', module: 'slf4j-api' - } implementation 'redis.clients:jedis:4.3.1' implementation libs.slf4jlog4j implementation libs.s3Client + implementation libs.commonLang + implementation libs.nettyAll implementation libs.zstd diff --git a/clients/src/main/java/org/apache/kafka/common/errors/s3/RedundantOperationException.java b/clients/src/main/java/org/apache/kafka/common/errors/s3/RedundantOperationException.java new file mode 100644 index 0000000000..5f7eb4c3ec --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/s3/RedundantOperationException.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.common.errors.s3; + +import org.apache.kafka.common.errors.ApiException; + +public class RedundantOperationException extends ApiException { + + public RedundantOperationException(String message) { + super(message); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index 6d33e7632e..ee8c9978fe 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -128,6 +128,7 @@ import org.apache.kafka.common.errors.UnsupportedSaslMechanismException; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.errors.s3.ObjectNotExistException; +import org.apache.kafka.common.errors.s3.RedundantOperationException; import org.apache.kafka.common.errors.s3.StreamExistException; import org.apache.kafka.common.errors.s3.StreamFencedException; import org.apache.kafka.common.errors.s3.StreamInnerErrorException; @@ -387,7 +388,7 @@ public enum Errors { OBJECT_NOT_EXIST(504, "The object does not exist.", ObjectNotExistException::new), STREAM_NOT_OPENED(505, "The stream is not opened.", StreamNotOpenedException::new), STREAM_NOT_CLOSED(506, "The stream is not closed.", StreamNotClosedException::new), - + REDUNDANT_OPERATION(507, "The operation is redundant.", RedundantOperationException::new), diff --git a/core/src/main/scala/kafka/log/es/AlwaysSuccessClient.java b/core/src/main/scala/kafka/log/es/AlwaysSuccessClient.java index 7e1111535a..14b36b1ad9 100644 --- a/core/src/main/scala/kafka/log/es/AlwaysSuccessClient.java +++ b/core/src/main/scala/kafka/log/es/AlwaysSuccessClient.java @@ -17,17 +17,22 @@ package kafka.log.es; -import com.automq.elasticstream.client.api.AppendResult; -import com.automq.elasticstream.client.api.Client; -import com.automq.elasticstream.client.api.CreateStreamOptions; -import com.automq.elasticstream.client.api.ElasticStreamClientException; -import com.automq.elasticstream.client.api.FetchResult; -import com.automq.elasticstream.client.api.KVClient; -import com.automq.elasticstream.client.api.OpenStreamOptions; -import com.automq.elasticstream.client.api.RecordBatch; -import com.automq.elasticstream.client.api.Stream; -import com.automq.elasticstream.client.api.StreamClient; -import com.automq.elasticstream.client.flatc.header.ErrorCode; +import kafka.log.es.api.AppendResult; +import kafka.log.es.api.Client; +import kafka.log.es.api.CreateStreamOptions; +import kafka.log.es.api.ElasticStreamClientException; +import kafka.log.es.api.ErrorCode; +import kafka.log.es.api.FetchResult; +import kafka.log.es.api.KVClient; +import kafka.log.es.api.OpenStreamOptions; +import kafka.log.es.api.RecordBatch; +import kafka.log.es.api.Stream; +import kafka.log.es.api.StreamClient; +import org.apache.kafka.common.errors.es.SlowFetchHintException; +import org.apache.kafka.common.utils.ThreadUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.IOException; import java.util.Map; import java.util.Set; @@ -39,34 +44,30 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.function.BiConsumer; -import org.apache.kafka.common.errors.es.SlowFetchHintException; -import org.apache.kafka.common.utils.ThreadUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class AlwaysSuccessClient implements Client { private static final Logger LOGGER = LoggerFactory.getLogger(AlwaysSuccessClient.class); public static final Set HALT_ERROR_CODES = Set.of(ErrorCode.EXPIRED_STREAM_EPOCH, ErrorCode.STREAM_ALREADY_CLOSED); - public static final long SLOW_FETCH_TIMEOUT_MILLIS = 10; + public static final long DEFAULT_SLOW_FETCH_TIMEOUT_MILLIS = 10; private final ScheduledExecutorService streamManagerRetryScheduler = Executors.newScheduledThreadPool(1, - ThreadUtils.createThreadFactory("stream-manager-retry-%d", true)); + ThreadUtils.createThreadFactory("stream-manager-retry-%d", true)); private final ExecutorService streamManagerCallbackExecutors = Executors.newFixedThreadPool(1, - ThreadUtils.createThreadFactory("stream-manager-callback-executor-%d", true)); + ThreadUtils.createThreadFactory("stream-manager-callback-executor-%d", true)); private final ScheduledExecutorService appendRetryScheduler = Executors.newScheduledThreadPool(1, - ThreadUtils.createThreadFactory("append-retry-scheduler-%d", true)); + ThreadUtils.createThreadFactory("append-retry-scheduler-%d", true)); private final ScheduledExecutorService fetchRetryScheduler = Executors.newScheduledThreadPool(1, - ThreadUtils.createThreadFactory("fetch-retry-scheduler-%d", true)); + ThreadUtils.createThreadFactory("fetch-retry-scheduler-%d", true)); private final ScheduledExecutorService generalRetryScheduler = Executors.newScheduledThreadPool(1, - ThreadUtils.createThreadFactory("general-retry-scheduler-%d", true)); + ThreadUtils.createThreadFactory("general-retry-scheduler-%d", true)); private final ExecutorService generalCallbackExecutors = Executors.newFixedThreadPool(4, - ThreadUtils.createThreadFactory("general-callback-scheduler-%d", true)); + ThreadUtils.createThreadFactory("general-callback-scheduler-%d", true)); private final ExecutorService appendCallbackExecutors = Executors.newFixedThreadPool(4, - ThreadUtils.createThreadFactory("append-callback-scheduler-%d", true)); + ThreadUtils.createThreadFactory("append-callback-scheduler-%d", true)); private final ExecutorService fetchCallbackExecutors = Executors.newFixedThreadPool(4, - ThreadUtils.createThreadFactory("fetch-callback-scheduler-%d", true)); + ThreadUtils.createThreadFactory("fetch-callback-scheduler-%d", true)); private final ScheduledExecutorService delayFetchScheduler = Executors.newScheduledThreadPool(1, - ThreadUtils.createThreadFactory("fetch-delayer-%d", true)); + ThreadUtils.createThreadFactory("fetch-delayer-%d", true)); private final StreamClient streamClient; private final KVClient kvClient; private final Delayer delayer; @@ -77,16 +78,22 @@ public class AlwaysSuccessClient implements Client { * due to the delay in updating the committed offset. */ private final boolean appendCallbackAsync; + private final long slowFetchTimeoutMillis; public AlwaysSuccessClient(Client client) { - this(client, true); + this(client, true, DEFAULT_SLOW_FETCH_TIMEOUT_MILLIS); } public AlwaysSuccessClient(Client client, boolean appendCallbackAsync) { + this(client, appendCallbackAsync, DEFAULT_SLOW_FETCH_TIMEOUT_MILLIS); + } + + public AlwaysSuccessClient(Client client, boolean appendCallbackAsync, long slowFetchTimeoutMillis) { this.streamClient = new StreamClientImpl(client.streamClient()); this.kvClient = client.kvClient(); this.delayer = new Delayer(delayFetchScheduler); this.appendCallbackAsync = appendCallbackAsync; + this.slowFetchTimeoutMillis = slowFetchTimeoutMillis; } @Override @@ -113,6 +120,7 @@ public void shutdownNow() { /** * Check if the exception is a ElasticStreamClientException with a halt error code. + * * @param t the exception * @return true if the exception is a ElasticStreamClientException with a halt error code, otherwise false */ @@ -229,7 +237,7 @@ private void append0(RecordBatch recordBatch, CompletableFuture cf stream.append(recordBatch).whenCompleteAsync((rst, ex) -> FutureUtil.suppress(() -> { if (ex != null) { if (!maybeHaltAndCompleteWaitingFuture(ex, cf)) { - LOGGER.error("Appending to stream[{}] failed, retry later", streamId(), ex); + LOGGER.error("Appending to stream[{}] failed, retry later", streamId(), ex); appendRetryScheduler.schedule(() -> append0(recordBatch, cf), 3, TimeUnit.SECONDS); } } else { @@ -241,6 +249,7 @@ private void append0(RecordBatch recordBatch, CompletableFuture cf /** * Append to stream without using async callback threadPools. * Used for tests only. + * * @param recordBatch * @param cf */ @@ -248,7 +257,7 @@ private void append0WithSyncCallback(RecordBatch recordBatch, CompletableFuture< stream.append(recordBatch).whenComplete((rst, ex) -> FutureUtil.suppress(() -> { if (ex != null) { if (!maybeHaltAndCompleteWaitingFuture(ex, cf)) { - LOGGER.error("Appending to stream[{}] failed, retry later", streamId(), ex); + LOGGER.error("Appending to stream[{}] failed, retry later", streamId(), ex); appendRetryScheduler.schedule(() -> append0(recordBatch, cf), 3, TimeUnit.SECONDS); } } else { @@ -268,8 +277,8 @@ private void append0WithSyncCallback(RecordBatch recordBatch, CompletableFuture< * CompletableFuture with a {@link SlowFetchHintException} */ private CompletableFuture timeoutAndStoreFuture(String id, - CompletableFuture rawFuture, long timeout, - TimeUnit unit) { + CompletableFuture rawFuture, long timeout, + TimeUnit unit) { if (unit == null) { throw new NullPointerException(); } @@ -316,19 +325,19 @@ public CompletableFuture fetch(long startOffset, long endOffset, in CompletableFuture firstFetchFuture = new CompletableFuture<>(); fetch0(startOffset, endOffset, maxBytesHint, firstFetchFuture); // Try to have a quick fetch. If the first fetching is timeout, then complete with SlowFetchHintException. - timeoutAndStoreFuture(holdUpKey, firstFetchFuture, SLOW_FETCH_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS) - .whenComplete((rst, ex) -> FutureUtil.suppress(() -> { - if (ex != null) { - if (ex instanceof SlowFetchHintException) { - LOGGER.debug("Fetch stream[{}] [{},{}) timeout for {} ms, retry later with slow fetching", streamId(), startOffset, endOffset, SLOW_FETCH_TIMEOUT_MILLIS); - cf.completeExceptionally(ex); - } else if (!maybeHaltAndCompleteWaitingFuture(ex, cf)) { - cf.completeExceptionally(ex); + timeoutAndStoreFuture(holdUpKey, firstFetchFuture, slowFetchTimeoutMillis, TimeUnit.MILLISECONDS) + .whenComplete((rst, ex) -> FutureUtil.suppress(() -> { + if (ex != null) { + if (ex instanceof SlowFetchHintException) { + LOGGER.debug("Fetch stream[{}] [{},{}) timeout for {} ms, retry later with slow fetching", streamId(), startOffset, endOffset, DEFAULT_SLOW_FETCH_TIMEOUT_MILLIS); + cf.completeExceptionally(ex); + } else if (!maybeHaltAndCompleteWaitingFuture(ex, cf)) { + cf.completeExceptionally(ex); + } + } else { + cf.complete(rst); } - } else { - cf.complete(rst); - } - }, LOGGER)); + }, LOGGER)); } return cf; } @@ -420,7 +429,7 @@ public Delayer(ScheduledExecutorService delayFetchScheduler) { } public ScheduledFuture delay(Runnable command, long delay, - TimeUnit unit) { + TimeUnit unit) { return delayFetchScheduler.schedule(command, delay, unit); } } diff --git a/core/src/main/scala/kafka/log/es/DefaultAppendResult.java b/core/src/main/scala/kafka/log/es/DefaultAppendResult.java new file mode 100644 index 0000000000..6b52e93b0c --- /dev/null +++ b/core/src/main/scala/kafka/log/es/DefaultAppendResult.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.es; + +import kafka.log.es.api.AppendResult; + +public class DefaultAppendResult implements AppendResult { + private final long baseOffset; + + public DefaultAppendResult(long baseOffset) { + this.baseOffset = baseOffset; + } + + @Override + public long baseOffset() { + return baseOffset; + } + + public String toString() { + return "AppendResult(baseOffset=" + baseOffset + ")"; + } +} + diff --git a/core/src/main/scala/kafka/log/es/DefaultElasticStreamSlice.java b/core/src/main/scala/kafka/log/es/DefaultElasticStreamSlice.java index fc68426a03..18c0040749 100644 --- a/core/src/main/scala/kafka/log/es/DefaultElasticStreamSlice.java +++ b/core/src/main/scala/kafka/log/es/DefaultElasticStreamSlice.java @@ -17,11 +17,13 @@ package kafka.log.es; -import com.automq.elasticstream.client.api.AppendResult; -import com.automq.elasticstream.client.api.FetchResult; -import com.automq.elasticstream.client.api.RecordBatch; -import com.automq.elasticstream.client.api.RecordBatchWithContext; -import com.automq.elasticstream.client.api.Stream; +import kafka.log.es.api.AppendResult; +import kafka.log.es.api.FetchResult; +import kafka.log.es.api.RecordBatch; +import kafka.log.es.api.RecordBatchWithContext; +import kafka.log.es.api.Stream; +import org.apache.kafka.common.errors.es.SlowFetchHintException; +import org.apache.kafka.common.utils.Utils; import java.io.IOException; import java.nio.ByteBuffer; @@ -30,8 +32,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; -import org.apache.kafka.common.errors.es.SlowFetchHintException; -import org.apache.kafka.common.utils.Utils; public class DefaultElasticStreamSlice implements ElasticStreamSlice { /** diff --git a/core/src/main/scala/kafka/log/es/DefaultRecordBatch.java b/core/src/main/scala/kafka/log/es/DefaultRecordBatch.java new file mode 100644 index 0000000000..3ef54c4b0d --- /dev/null +++ b/core/src/main/scala/kafka/log/es/DefaultRecordBatch.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.es; + +import kafka.log.es.api.RecordBatch; + +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.Map; + +public class DefaultRecordBatch implements RecordBatch { + private final int count; + private final long baseTimestamp; + private final Map properties; + private final ByteBuffer rawPayload; + + public DefaultRecordBatch(int count, long baseTimestamp, Map properties, ByteBuffer rawPayload) { + this.count = count; + this.baseTimestamp = baseTimestamp; + this.properties = properties; + this.rawPayload = rawPayload; + } + + @Override + public int count() { + return count; + } + + @Override + public long baseTimestamp() { + return baseTimestamp; + } + + @Override + public Map properties() { + if (properties == null) { + return Collections.emptyMap(); + } + return properties; + } + + @Override + public ByteBuffer rawPayload() { + return rawPayload.duplicate(); + } +} diff --git a/core/src/main/scala/kafka/log/es/ElasticLeaderEpochCheckpoint.scala b/core/src/main/scala/kafka/log/es/ElasticLeaderEpochCheckpoint.scala index ae1984eb0c..e412f89c07 100644 --- a/core/src/main/scala/kafka/log/es/ElasticLeaderEpochCheckpoint.scala +++ b/core/src/main/scala/kafka/log/es/ElasticLeaderEpochCheckpoint.scala @@ -23,13 +23,13 @@ import kafka.server.epoch.EpochEntry import scala.jdk.CollectionConverters.{ListHasAsScala, SeqHasAsJava} -class ElasticLeaderEpochCheckpoint(val meta: ElasticLeaderEpochCheckpointMeta, val saveFunc: ElasticLeaderEpochCheckpointMeta => Unit) extends LeaderEpochCheckpoint{ - override def write(epochs: Iterable[EpochEntry]): Unit = this.synchronized { - meta.setEntries(epochs.toList.asJava) - saveFunc(meta) - } +class ElasticLeaderEpochCheckpoint(val meta: ElasticLeaderEpochCheckpointMeta, val saveFunc: ElasticLeaderEpochCheckpointMeta => Unit) extends LeaderEpochCheckpoint { + override def write(epochs: Iterable[EpochEntry]): Unit = this.synchronized { + meta.setEntries(epochs.toList.asJava) + saveFunc(meta) + } - override def read(): collection.Seq[EpochEntry] = this.synchronized { - meta.entries().asScala.toSeq - } + override def read(): collection.Seq[EpochEntry] = this.synchronized { + meta.entries().asScala.toSeq + } } diff --git a/core/src/main/scala/kafka/log/es/ElasticLeaderEpochCheckpointMeta.java b/core/src/main/scala/kafka/log/es/ElasticLeaderEpochCheckpointMeta.java index 7e0b572164..8377c1cb9b 100644 --- a/core/src/main/scala/kafka/log/es/ElasticLeaderEpochCheckpointMeta.java +++ b/core/src/main/scala/kafka/log/es/ElasticLeaderEpochCheckpointMeta.java @@ -17,10 +17,11 @@ package kafka.log.es; +import kafka.server.epoch.EpochEntry; + import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; -import kafka.server.epoch.EpochEntry; public class ElasticLeaderEpochCheckpointMeta { private final int version; @@ -33,11 +34,11 @@ public ElasticLeaderEpochCheckpointMeta(int version, List entries) { public byte[] encode() { int totalLength = 4 // version - + 4 // following entries size - + 12 * entries.size(); // all entries + + 4 // following entries size + + 12 * entries.size(); // all entries ByteBuffer buffer = ByteBuffer.allocate(totalLength) - .putInt(version) - .putInt(entries.size()); + .putInt(version) + .putInt(entries.size()); entries.forEach(entry -> buffer.putInt(entry.epoch()).putLong(entry.startOffset())); buffer.flip(); return buffer.array(); @@ -51,7 +52,7 @@ public static ElasticLeaderEpochCheckpointMeta decode(ByteBuffer buffer) { entryList.add(new EpochEntry(buffer.getInt(), buffer.getLong())); } if (entryList.size() != entryCount) { - throw new RuntimeException("expect entry count:" + entryCount + ", decoded " + entryList.size() + " entries"); + throw new RuntimeException("expect entry count:" + entryCount + ", decoded " + entryList.size() + " entries"); } return new ElasticLeaderEpochCheckpointMeta(version, entryList); } diff --git a/core/src/main/scala/kafka/log/es/ElasticLog.scala b/core/src/main/scala/kafka/log/es/ElasticLog.scala index 18488ea3ea..b4a442a909 100644 --- a/core/src/main/scala/kafka/log/es/ElasticLog.scala +++ b/core/src/main/scala/kafka/log/es/ElasticLog.scala @@ -17,9 +17,9 @@ package kafka.log.es -import com.automq.elasticstream.client.api.{Client, CreateStreamOptions, KeyValue, OpenStreamOptions} import io.netty.buffer.Unpooled import kafka.log._ +import kafka.log.es.api.{Client, CreateStreamOptions, KeyValue, OpenStreamOptions} import kafka.log.es.metrics.Timer import kafka.server.checkpoints.LeaderEpochCheckpointFile import kafka.server.epoch.EpochEntry @@ -291,22 +291,23 @@ class ElasticLog(val metaStream: MetaStream, partitionMeta.setRecoverOffset(recoveryPoint) maybeHandleIOException(s"Error while closing $topicPartition in dir ${dir.getParent}") { - persistPartitionMeta() + CoreUtils.swallow(persistPartitionMeta(), this) if (modified) { // update log size - persistLogMeta() + CoreUtils.swallow(persistLogMeta(), this) } - checkIfMemoryMappedBufferClosed() - segments.close() - closeStreams() + CoreUtils.swallow(checkIfMemoryMappedBufferClosed(), this) + CoreUtils.swallow(segments.close(), this) + CoreUtils.swallow(closeStreams(), this) } + info("log closed"); } /** * Directly close all streams of the log. This method may throw IOException. */ def closeStreams(): Unit = { - try{ + try { CompletableFuture.allOf(streamManager.close(), metaStream.close()).get() } catch { case e: ExecutionException => @@ -367,8 +368,8 @@ object ElasticLog extends Logging { val metaStreamId = Unpooled.wrappedBuffer(keyValue.value()).readLong() // open partition meta stream val stream = client.streamClient().openStream(metaStreamId, OpenStreamOptions.newBuilder().epoch(leaderEpoch).build()) - .thenApply(stream => new MetaStream(stream, META_SCHEDULE_EXECUTOR, logIdent)) - .get() + .thenApply(stream => new MetaStream(stream, META_SCHEDULE_EXECUTOR, logIdent)) + .get() info(s"${logIdent}opened existing meta stream: stream_id=$metaStreamId") stream } @@ -387,7 +388,7 @@ object ElasticLog extends Logging { def loadAllValidSnapshots(): mutable.Map[Long, ElasticPartitionProducerSnapshotMeta] = { metaMap.filter(kv => kv._1.startsWith(MetaStream.PRODUCER_SNAPSHOT_KEY_PREFIX)) - .map(kv => (kv._1.stripPrefix(MetaStream.PRODUCER_SNAPSHOT_KEY_PREFIX).toLong, kv._2.asInstanceOf[ElasticPartitionProducerSnapshotMeta])) + .map(kv => (kv._1.stripPrefix(MetaStream.PRODUCER_SNAPSHOT_KEY_PREFIX).toLong, kv._2.asInstanceOf[ElasticPartitionProducerSnapshotMeta])) } //load producer snapshots for this partition @@ -474,14 +475,14 @@ object ElasticLog extends Logging { // We have to close streams here since this log has not been added to currentLogs yet. It will not be handled // by LogDirFailureChannel. CoreUtils.swallow({ - if (metaStream != null) { - metaStream.close().get - } - if (logStreamManager != null) { - logStreamManager.close().get() - } - client.kvClient().delKV(java.util.Arrays.asList(key)).get() - }, this) + if (metaStream != null) { + metaStream.close().get + } + if (logStreamManager != null) { + logStreamManager.close().get() + } + client.kvClient().delKV(java.util.Arrays.asList(key)).get() + }, this) error(s"${logIdent}failed to open elastic log, trying to close streams and delete key. Error msg: ${e.getMessage}") throw e } @@ -525,8 +526,8 @@ object ElasticLog extends Logging { // Finally, destroy meta stream. metaStream.destroy() } finally { - // remove kv info - client.kvClient().delKV(java.util.Arrays.asList(key)).get() + // remove kv info + client.kvClient().delKV(java.util.Arrays.asList(key)).get() } info(s"$logIdent Destroyed with epoch ${currentEpoch + 1}") @@ -534,19 +535,19 @@ object ElasticLog extends Logging { private def openStreamWithRetry(client: Client, streamId: Long, epoch: Long, logIdent: String): MetaStream = { client.streamClient() - .openStream(streamId, OpenStreamOptions.newBuilder().epoch(epoch).build()) - .exceptionally(_ => client.streamClient() - .openStream(streamId, OpenStreamOptions.newBuilder().build()).join() - ).thenApply(stream => new MetaStream(stream, META_SCHEDULE_EXECUTOR, logIdent)) - .join() + .openStream(streamId, OpenStreamOptions.newBuilder().epoch(epoch).build()) + .exceptionally(_ => client.streamClient() + .openStream(streamId, OpenStreamOptions.newBuilder().build()).join() + ).thenApply(stream => new MetaStream(stream, META_SCHEDULE_EXECUTOR, logIdent)) + .join() } private[es] def createMetaStream(client: Client, key: String, replicaCount: Int, leaderEpoch: Long, logIdent: String): MetaStream = { val metaStream = client.streamClient().createAndOpenStream(CreateStreamOptions.newBuilder() .replicaCount(replicaCount) .epoch(leaderEpoch).build() - ).thenApply(stream => new MetaStream(stream, META_SCHEDULE_EXECUTOR, logIdent)) - .get() + ).thenApply(stream => new MetaStream(stream, META_SCHEDULE_EXECUTOR, logIdent)) + .get() // save partition meta stream id relation to PM val streamId = metaStream.streamId() info(s"${logIdent}created meta stream for $key, streamId: $streamId") @@ -567,7 +568,7 @@ object ElasticLog extends Logging { * For the newly created cleaned segment, the meta should not be saved here. It will be saved iff the replacement happens. */ private def createAndSaveSegment(logSegmentManager: ElasticLogSegmentManager, suffix: String = "", logIdent: String)(baseOffset: Long, dir: File, - config: LogConfig, streamSliceManager: ElasticStreamSliceManager, time: Time): ElasticLogSegment = { + config: LogConfig, streamSliceManager: ElasticStreamSliceManager, time: Time): ElasticLogSegment = { if (!suffix.equals("") && !suffix.equals(LocalLog.CleanedFileSuffix)) { throw new IllegalArgumentException("suffix must be empty or " + LocalLog.CleanedFileSuffix) } diff --git a/core/src/main/scala/kafka/log/es/ElasticLogFileRecords.java b/core/src/main/scala/kafka/log/es/ElasticLogFileRecords.java index 40221455eb..d5df5750ac 100644 --- a/core/src/main/scala/kafka/log/es/ElasticLogFileRecords.java +++ b/core/src/main/scala/kafka/log/es/ElasticLogFileRecords.java @@ -17,10 +17,10 @@ package kafka.log.es; -import com.automq.elasticstream.client.api.FetchResult; -import com.automq.elasticstream.client.api.RecordBatchWithContext; import io.netty.buffer.CompositeByteBuf; import io.netty.buffer.Unpooled; +import kafka.log.es.api.FetchResult; +import kafka.log.es.api.RecordBatchWithContext; import org.apache.kafka.common.errors.es.SlowFetchHintException; import org.apache.kafka.common.network.TransferableChannel; import org.apache.kafka.common.record.AbstractRecords; @@ -37,6 +37,7 @@ import org.apache.kafka.common.record.RecordsUtil; import org.apache.kafka.common.utils.AbstractIterator; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -52,8 +53,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; -import org.apache.kafka.common.utils.Utils; - public class ElasticLogFileRecords { private static final Logger LOGGER = LoggerFactory.getLogger(ElasticLogFileRecords.class); protected final AtomicInteger size; @@ -135,7 +134,8 @@ private Records readAll0(long startOffset, long maxOffset, int maxSize) throws S * Append records to segment. * Note that lastOffset is the expected value of nextOffset after append. lastOffset = (the real last offset of the * records) + 1 - * @param records records to append + * + * @param records records to append * @param lastOffset expected next offset after append * @return the size of the appended records * @throws IOException @@ -150,7 +150,7 @@ public int append(MemoryRecords records, long lastOffset) throws IOException { int appendSize = records.sizeInBytes(); // Note that the calculation of count requires strong consistency between nextOffset and the baseOffset of records. int count = (int) (lastOffset - nextOffset.get()); - com.automq.elasticstream.client.DefaultRecordBatch batch = new com.automq.elasticstream.client.DefaultRecordBatch(count, 0, Collections.emptyMap(), records.buffer()); + kafka.log.es.DefaultRecordBatch batch = new kafka.log.es.DefaultRecordBatch(count, 0, Collections.emptyMap(), records.buffer()); CompletableFuture cf = streamSegment.append(batch); nextOffset.set(lastOffset); size.getAndAdd(appendSize); @@ -223,6 +223,7 @@ private Optional maybeLeaderEpoch(int leaderEpoch) { /** * Return the largest timestamp of the messages after a given offset + * * @param startOffset The starting offset. * @return The largest timestamp of the messages after the given position. */ @@ -240,7 +241,7 @@ public FileRecords.TimestampAndOffset largestTimestampAfter(long startOffset) { } } return new FileRecords.TimestampAndOffset(maxTimestamp, offsetOfMaxTimestamp, - maybeLeaderEpoch(leaderEpochOfMaxTimestamp)); + maybeLeaderEpoch(leaderEpochOfMaxTimestamp)); } public ElasticStreamSlice streamSegment() { diff --git a/core/src/main/scala/kafka/log/es/ElasticLogManager.scala b/core/src/main/scala/kafka/log/es/ElasticLogManager.scala index 6083580545..b567a54dd8 100644 --- a/core/src/main/scala/kafka/log/es/ElasticLogManager.scala +++ b/core/src/main/scala/kafka/log/es/ElasticLogManager.scala @@ -17,11 +17,11 @@ package kafka.log.es -import com.automq.elasticstream.client.api.Client -import kafka.log.{LogConfig, ProducerStateManagerConfig} import kafka.log.es.ElasticLogManager.NAMESPACE +import kafka.log.es.api.Client import kafka.log.es.client.{ClientFactoryProxy, Context} import kafka.log.s3.DefaultS3Client +import kafka.log.{LogConfig, ProducerStateManagerConfig} import kafka.server.{BrokerServer, KafkaConfig, LogDirFailureChannel} import kafka.utils.{Logging, Scheduler} import org.apache.kafka.common.TopicPartition @@ -31,20 +31,20 @@ import java.io.File import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap} import scala.jdk.CollectionConverters.ConcurrentMapHasAsScala -class ElasticLogManager(val client: Client) extends Logging{ +class ElasticLogManager(val client: Client) extends Logging { this.logIdent = s"[ElasticLogManager] " private val elasticLogs = new ConcurrentHashMap[TopicPartition, ElasticLog]() def getOrCreateLog(dir: File, - config: LogConfig, - scheduler: Scheduler, - time: Time, - topicPartition: TopicPartition, - logDirFailureChannel: LogDirFailureChannel, - numRemainingSegments: ConcurrentMap[String, Int] = new ConcurrentHashMap[String, Int], - maxTransactionTimeoutMs: Int, - producerStateManagerConfig: ProducerStateManagerConfig, - leaderEpoch: Long): ElasticLog = { + config: LogConfig, + scheduler: Scheduler, + time: Time, + topicPartition: TopicPartition, + logDirFailureChannel: LogDirFailureChannel, + numRemainingSegments: ConcurrentMap[String, Int] = new ConcurrentHashMap[String, Int], + maxTransactionTimeoutMs: Int, + producerStateManagerConfig: ProducerStateManagerConfig, + leaderEpoch: Long): ElasticLog = { elasticLogs.computeIfAbsent(topicPartition, _ => { var elasticLog: ElasticLog = null ExceptionUtil.maybeRecordThrowableAndRethrow(new Runnable { @@ -59,8 +59,9 @@ class ElasticLogManager(val client: Client) extends Logging{ /** * Delete elastic log by topic partition. Note that this method may not be called by the broker holding the partition. + * * @param topicPartition topic partition - * @param epoch epoch of the partition + * @param epoch epoch of the partition */ def destroyLog(topicPartition: TopicPartition, epoch: Long): Unit = { // Removal may have happened in partition's closure. This is a defensive work. @@ -76,6 +77,7 @@ class ElasticLogManager(val client: Client) extends Logging{ /** * Remove elastic log in the map. + * * @param topicPartition topic partition */ def removeLog(topicPartition: TopicPartition): Unit = { @@ -114,6 +116,7 @@ object ElasticLogManager { var INSTANCE: Option[ElasticLogManager] = None var NAMESPACE = "" var DEFAULT_CLIENT: Option[DefaultS3Client] = None + def init(config: KafkaConfig, clusterId: String, broker: BrokerServer = null, appendWithAsyncCallbacks: Boolean = true): Boolean = { if (!config.elasticStreamEnabled) { return false @@ -136,7 +139,7 @@ object ElasticLogManager { context.config = config context.brokerServer = broker context.appendWithAsyncCallbacks = appendWithAsyncCallbacks - INSTANCE = Some(new ElasticLogManager(ClientFactoryProxy.get(context))) + INSTANCE = Some(new ElasticLogManager(ClientFactoryProxy.get(context))) val namespace = config.elasticStreamNamespace NAMESPACE = if (namespace == null || namespace.isEmpty) { @@ -163,15 +166,15 @@ object ElasticLogManager { // visible for testing def getOrCreateLog(dir: File, - config: LogConfig, - scheduler: Scheduler, - time: Time, - topicPartition: TopicPartition, - logDirFailureChannel: LogDirFailureChannel, - maxTransactionTimeoutMs: Int, - producerStateManagerConfig: ProducerStateManagerConfig, - numRemainingSegments: ConcurrentMap[String, Int] = new ConcurrentHashMap[String, Int], - leaderEpoch: Long): ElasticLog = { + config: LogConfig, + scheduler: Scheduler, + time: Time, + topicPartition: TopicPartition, + logDirFailureChannel: LogDirFailureChannel, + maxTransactionTimeoutMs: Int, + producerStateManagerConfig: ProducerStateManagerConfig, + numRemainingSegments: ConcurrentMap[String, Int] = new ConcurrentHashMap[String, Int], + leaderEpoch: Long): ElasticLog = { INSTANCE.get.getOrCreateLog(dir, config, scheduler, time, topicPartition, logDirFailureChannel, numRemainingSegments, maxTransactionTimeoutMs, producerStateManagerConfig, leaderEpoch) } diff --git a/core/src/main/scala/kafka/log/es/ElasticLogMeta.java b/core/src/main/scala/kafka/log/es/ElasticLogMeta.java index 2cf995e5aa..e0fbf232c6 100644 --- a/core/src/main/scala/kafka/log/es/ElasticLogMeta.java +++ b/core/src/main/scala/kafka/log/es/ElasticLogMeta.java @@ -19,13 +19,14 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; +import org.slf4j.Logger; + import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; -import org.slf4j.Logger; /** * logical meta data for a Kafka topicPartition. @@ -87,8 +88,8 @@ public String toString() { int size = segmentMetas.size(); List lastNthSegmentMetas = segmentMetas.subList(Math.max(0, size - 5), size); return "ElasticLogMeta{" + - "streamMap=" + streamMap + - ", lastNthSegmentMetas=" + lastNthSegmentMetas + - '}'; + "streamMap=" + streamMap + + ", lastNthSegmentMetas=" + lastNthSegmentMetas + + '}'; } } diff --git a/core/src/main/scala/kafka/log/es/ElasticLogSegment.scala b/core/src/main/scala/kafka/log/es/ElasticLogSegment.scala index fbf86af117..e08e474d22 100644 --- a/core/src/main/scala/kafka/log/es/ElasticLogSegment.scala +++ b/core/src/main/scala/kafka/log/es/ElasticLogSegment.scala @@ -33,7 +33,6 @@ import java.util.concurrent.CompletableFuture import scala.jdk.CollectionConverters._ - class ElasticLogSegment(val _meta: ElasticStreamSegmentMeta, val _log: ElasticLogFileRecords, val timeIdx: ElasticTimeIndex, @@ -117,6 +116,7 @@ class ElasticLogSegment(val _meta: ElasticStreamSegmentMeta, def asyncLogFlush(): CompletableFuture[Void] = { _log.asyncFlush() } + def appendFromFile(records: FileRecords, start: Int): Int = { throw new UnsupportedOperationException() } @@ -250,6 +250,7 @@ class ElasticLogSegment(val _meta: ElasticStreamSegmentMeta, /** * get appended offset. It can be used to show whether the segment contains any valid data. + * * @return appended offset */ def appendedOffset: Long = _log.appendedOffset() diff --git a/core/src/main/scala/kafka/log/es/ElasticLogStreamManager.java b/core/src/main/scala/kafka/log/es/ElasticLogStreamManager.java index d187f23b1d..cff30fcddb 100644 --- a/core/src/main/scala/kafka/log/es/ElasticLogStreamManager.java +++ b/core/src/main/scala/kafka/log/es/ElasticLogStreamManager.java @@ -17,8 +17,9 @@ package kafka.log.es; -import com.automq.elasticstream.client.api.Stream; -import com.automq.elasticstream.client.api.StreamClient; +import kafka.log.es.api.Stream; +import kafka.log.es.api.StreamClient; + import java.io.IOException; import java.util.Collections; import java.util.Map; diff --git a/core/src/main/scala/kafka/log/es/ElasticPartitionMeta.java b/core/src/main/scala/kafka/log/es/ElasticPartitionMeta.java index 361917b0d9..a46274f52a 100644 --- a/core/src/main/scala/kafka/log/es/ElasticPartitionMeta.java +++ b/core/src/main/scala/kafka/log/es/ElasticPartitionMeta.java @@ -47,7 +47,8 @@ public class ElasticPartitionMeta { private boolean cleanedShutdown; @SuppressWarnings("unused") // used by jackson - public ElasticPartitionMeta() {} + public ElasticPartitionMeta() { + } public ElasticPartitionMeta(Long startOffset, Long cleanerOffset, Long recoverOffset) { this.startOffset = startOffset; diff --git a/core/src/main/scala/kafka/log/es/ElasticProducerStateManager.scala b/core/src/main/scala/kafka/log/es/ElasticProducerStateManager.scala index 06aca2b7e3..f91d21255e 100644 --- a/core/src/main/scala/kafka/log/es/ElasticProducerStateManager.scala +++ b/core/src/main/scala/kafka/log/es/ElasticProducerStateManager.scala @@ -6,7 +6,7 @@ * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -28,133 +28,134 @@ import scala.collection.mutable /** * ElasticProducerStateManager. Temporarily, we only persist the last snapshot. + * * @param snapshotsMap All valid snapshots. */ class ElasticProducerStateManager( - override val topicPartition: TopicPartition, - var logDir: File, - override val maxTransactionTimeoutMs: Int, - override val producerStateManagerConfig: ProducerStateManagerConfig, - override val time: Time, - val snapshotsMap: mutable.Map[Long, ElasticPartitionProducerSnapshotMeta], - val persistFun: ElasticPartitionProducerSnapshotMeta => Unit -) extends ProducerStateManager(topicPartition, logDir, maxTransactionTimeoutMs, producerStateManagerConfig, time) { - - this.logIdent = s"[ElasticProducerStateManager partition=$topicPartition] " - - override protected def loadSnapshots(): ConcurrentSkipListMap[java.lang.Long, SnapshotFile] = { - val tm = new ConcurrentSkipListMap[java.lang.Long, SnapshotFile]() - snapshotsMap.foreach { case (offset, meta) => - tm.put(offset, SnapshotFile(new File(meta.fileName()))) - } - tm - } - - override protected def loadFromSnapshot(logStartOffset: Long, currentTime: Long): Unit = { - while (true) { - latestSnapshotFile match { - case Some(snapshot) => - try { - info(s"Loading producer state from snapshot file '$snapshot'") - val loadedProducers = readSnapshot(snapshot.file).filter { producerEntry => !isProducerExpired(currentTime, producerEntry) } - loadedProducers.foreach(loadProducerEntry) - lastSnapOffset = snapshot.offset - lastMapOffset = lastSnapOffset - updateOldestTxnTimestamp() - return - } catch { - case e: CorruptSnapshotException => - warn(s"Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}") - removeAndDeleteSnapshot(snapshot.offset) - } - case None => - lastSnapOffset = logStartOffset - lastMapOffset = logStartOffset - return - } - } - } - - override def takeSnapshot(): Unit = { - // If not a new offset, then it is not worth taking another snapshot - if (lastMapOffset > lastSnapOffset) { - val snapshotFile = SnapshotFile(UnifiedLog.producerSnapshotFile(_logDir, lastMapOffset)) - val start = time.hiResClockMs() - writeSnapshot(snapshotFile.offset, producers) - info(s"Wrote producer snapshot at offset $lastMapOffset with ${producers.size} producer ids in ${time.hiResClockMs() - start} ms.") - - snapshots.put(snapshotFile.offset, snapshotFile) - - // Update the last snap offset according to the serialized map - lastSnapOffset = lastMapOffset - } + override val topicPartition: TopicPartition, + var logDir: File, + override val maxTransactionTimeoutMs: Int, + override val producerStateManagerConfig: ProducerStateManagerConfig, + override val time: Time, + val snapshotsMap: mutable.Map[Long, ElasticPartitionProducerSnapshotMeta], + val persistFun: ElasticPartitionProducerSnapshotMeta => Unit + ) extends ProducerStateManager(topicPartition, logDir, maxTransactionTimeoutMs, producerStateManagerConfig, time) { + + this.logIdent = s"[ElasticProducerStateManager partition=$topicPartition] " + + override protected def loadSnapshots(): ConcurrentSkipListMap[java.lang.Long, SnapshotFile] = { + val tm = new ConcurrentSkipListMap[java.lang.Long, SnapshotFile]() + snapshotsMap.foreach { case (offset, meta) => + tm.put(offset, SnapshotFile(new File(meta.fileName()))) } - - private def writeSnapshot(offset: Long, entries: mutable.Map[Long, ProducerStateEntry]): Unit = { - val buffer = ProducerStateManager.writeSnapshotToBuffer(entries) - val rawSnapshot: Array[Byte] = new Array[Byte](buffer.remaining()) - buffer.get(rawSnapshot) - - val meta = new ElasticPartitionProducerSnapshotMeta(offset, rawSnapshot) - snapshotsMap.put(offset, meta) - persistFun(meta) + tm + } + + override protected def loadFromSnapshot(logStartOffset: Long, currentTime: Long): Unit = { + while (true) { + latestSnapshotFile match { + case Some(snapshot) => + try { + info(s"Loading producer state from snapshot file '$snapshot'") + val loadedProducers = readSnapshot(snapshot.file).filter { producerEntry => !isProducerExpired(currentTime, producerEntry) } + loadedProducers.foreach(loadProducerEntry) + lastSnapOffset = snapshot.offset + lastMapOffset = lastSnapOffset + updateOldestTxnTimestamp() + return + } catch { + case e: CorruptSnapshotException => + warn(s"Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}") + removeAndDeleteSnapshot(snapshot.offset) + } + case None => + lastSnapOffset = logStartOffset + lastMapOffset = logStartOffset + return + } } + } - private def readSnapshot(file: File): Iterable[ProducerStateEntry] = { - val offset = LocalLog.offsetFromFile(file) - if (!snapshotsMap.contains(offset)) { - throw new CorruptSnapshotException(s"Snapshot not found") - } - - try { - ProducerStateManager.readSnapshotFromBuffer(snapshotsMap(offset).getRawSnapshotData) - } catch { - case e: SchemaException => - throw new CorruptSnapshotException(s"Snapshot failed schema validation: ${e.getMessage}") - } - } + override def takeSnapshot(): Unit = { + // If not a new offset, then it is not worth taking another snapshot + if (lastMapOffset > lastSnapOffset) { + val snapshotFile = SnapshotFile(UnifiedLog.producerSnapshotFile(_logDir, lastMapOffset)) + val start = time.hiResClockMs() + writeSnapshot(snapshotFile.offset, producers) + info(s"Wrote producer snapshot at offset $lastMapOffset with ${producers.size} producer ids in ${time.hiResClockMs() - start} ms.") - // do nothing - override def updateParentDir(parentDir: File): Unit = {} + snapshots.put(snapshotFile.offset, snapshotFile) - override protected def removeAndDeleteSnapshot(snapshotOffset: Long): Unit = { - deleteSnapshot(snapshotOffset) + // Update the last snap offset according to the serialized map + lastSnapOffset = lastMapOffset } - - override private[log] def removeAndMarkSnapshotForDeletion(snapshotOffset: Long): Option[SnapshotFile] = { - deleteSnapshot(snapshotOffset) - None + } + + private def writeSnapshot(offset: Long, entries: mutable.Map[Long, ProducerStateEntry]): Unit = { + val buffer = ProducerStateManager.writeSnapshotToBuffer(entries) + val rawSnapshot: Array[Byte] = new Array[Byte](buffer.remaining()) + buffer.get(rawSnapshot) + + val meta = new ElasticPartitionProducerSnapshotMeta(offset, rawSnapshot) + snapshotsMap.put(offset, meta) + persistFun(meta) + } + + private def readSnapshot(file: File): Iterable[ProducerStateEntry] = { + val offset = LocalLog.offsetFromFile(file) + if (!snapshotsMap.contains(offset)) { + throw new CorruptSnapshotException(s"Snapshot not found") } - private def deleteSnapshot(snapshotOffset: Long): Unit = { - snapshots.remove(snapshotOffset) - snapshotsMap.remove(snapshotOffset).foreach( snapshot => { - snapshot.setRawSnapshotData(null) - persistFun(snapshot) - info(s"Deleted producer snapshot file '$snapshotOffset' for partition $topicPartition") - }) + try { + ProducerStateManager.readSnapshotFromBuffer(snapshotsMap(offset).getRawSnapshotData) + } catch { + case e: SchemaException => + throw new CorruptSnapshotException(s"Snapshot failed schema validation: ${e.getMessage}") } + } + + // do nothing + override def updateParentDir(parentDir: File): Unit = {} + + override protected def removeAndDeleteSnapshot(snapshotOffset: Long): Unit = { + deleteSnapshot(snapshotOffset) + } + + override private[log] def removeAndMarkSnapshotForDeletion(snapshotOffset: Long): Option[SnapshotFile] = { + deleteSnapshot(snapshotOffset) + None + } + + private def deleteSnapshot(snapshotOffset: Long): Unit = { + snapshots.remove(snapshotOffset) + snapshotsMap.remove(snapshotOffset).foreach(snapshot => { + snapshot.setRawSnapshotData(null) + persistFun(snapshot) + info(s"Deleted producer snapshot file '$snapshotOffset' for partition $topicPartition") + }) + } } object ElasticProducerStateManager { - def apply( - topicPartition: TopicPartition, - logDir: File, - maxTransactionTimeoutMs: Int, - producerStateManagerConfig: ProducerStateManagerConfig, - time: Time, - snapshotMap: mutable.Map[Long, ElasticPartitionProducerSnapshotMeta], - persistFun: ElasticPartitionProducerSnapshotMeta => Unit - ): ElasticProducerStateManager = { - val stateManager = new ElasticProducerStateManager( - topicPartition, - logDir, - maxTransactionTimeoutMs, - producerStateManagerConfig, - time, - snapshotMap, - persistFun - ) - stateManager - } + def apply( + topicPartition: TopicPartition, + logDir: File, + maxTransactionTimeoutMs: Int, + producerStateManagerConfig: ProducerStateManagerConfig, + time: Time, + snapshotMap: mutable.Map[Long, ElasticPartitionProducerSnapshotMeta], + persistFun: ElasticPartitionProducerSnapshotMeta => Unit + ): ElasticProducerStateManager = { + val stateManager = new ElasticProducerStateManager( + topicPartition, + logDir, + maxTransactionTimeoutMs, + producerStateManagerConfig, + time, + snapshotMap, + persistFun + ) + stateManager + } } diff --git a/core/src/main/scala/kafka/log/es/ElasticRedisClient.java b/core/src/main/scala/kafka/log/es/ElasticRedisClient.java index 3791843941..2619d561a9 100644 --- a/core/src/main/scala/kafka/log/es/ElasticRedisClient.java +++ b/core/src/main/scala/kafka/log/es/ElasticRedisClient.java @@ -17,20 +17,20 @@ package kafka.log.es; +import kafka.log.es.api.AppendResult; +import kafka.log.es.api.Client; +import kafka.log.es.api.CreateStreamOptions; +import kafka.log.es.api.FetchResult; +import kafka.log.es.api.KVClient; +import kafka.log.es.api.KeyValue; +import kafka.log.es.api.OpenStreamOptions; +import kafka.log.es.api.RecordBatch; +import kafka.log.es.api.RecordBatchWithContext; +import kafka.log.es.api.Stream; +import kafka.log.es.api.StreamClient; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import redis.clients.jedis.JedisPooled; -import com.automq.elasticstream.client.api.AppendResult; -import com.automq.elasticstream.client.api.Client; -import com.automq.elasticstream.client.api.CreateStreamOptions; -import com.automq.elasticstream.client.api.FetchResult; -import com.automq.elasticstream.client.api.KVClient; -import com.automq.elasticstream.client.api.KeyValue; -import com.automq.elasticstream.client.api.OpenStreamOptions; -import com.automq.elasticstream.client.api.RecordBatch; -import com.automq.elasticstream.client.api.RecordBatchWithContext; -import com.automq.elasticstream.client.api.Stream; -import com.automq.elasticstream.client.api.StreamClient; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; diff --git a/core/src/main/scala/kafka/log/es/ElasticStreamSlice.java b/core/src/main/scala/kafka/log/es/ElasticStreamSlice.java index 661c9c96a9..cfdd43a2d8 100644 --- a/core/src/main/scala/kafka/log/es/ElasticStreamSlice.java +++ b/core/src/main/scala/kafka/log/es/ElasticStreamSlice.java @@ -17,14 +17,14 @@ package kafka.log.es; -import com.automq.elasticstream.client.api.AppendResult; -import com.automq.elasticstream.client.api.FetchResult; -import com.automq.elasticstream.client.api.RecordBatch; -import com.automq.elasticstream.client.api.Stream; +import kafka.log.es.api.AppendResult; +import kafka.log.es.api.FetchResult; +import kafka.log.es.api.RecordBatch; +import kafka.log.es.api.Stream; +import org.apache.kafka.common.errors.es.SlowFetchHintException; import java.io.IOException; import java.util.concurrent.CompletableFuture; -import org.apache.kafka.common.errors.es.SlowFetchHintException; /** * Elastic stream slice is a slice from elastic stream, the startOffset of a slice is 0. @@ -68,6 +68,7 @@ default FetchResult fetch(long startOffset, long endOffset) throws SlowFetchHint /** * Get slice range which is the relative offset range in stream. + * * @return {@link SliceRange} */ SliceRange sliceRange(); diff --git a/core/src/main/scala/kafka/log/es/ElasticStreamSliceManager.java b/core/src/main/scala/kafka/log/es/ElasticStreamSliceManager.java index 5c3449d398..8fba262b46 100644 --- a/core/src/main/scala/kafka/log/es/ElasticStreamSliceManager.java +++ b/core/src/main/scala/kafka/log/es/ElasticStreamSliceManager.java @@ -17,13 +17,13 @@ package kafka.log.es; -import com.automq.elasticstream.client.api.Stream; +import kafka.log.es.api.Stream; import java.io.IOException; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -import static com.automq.elasticstream.client.utils.Arguments.check; +import static kafka.log.es.utils.Arguments.check; /** * Elastic log dimension stream segment manager. diff --git a/core/src/main/scala/kafka/log/es/ElasticUnifiedLog.scala b/core/src/main/scala/kafka/log/es/ElasticUnifiedLog.scala index 9404c1a5e0..75d8710339 100644 --- a/core/src/main/scala/kafka/log/es/ElasticUnifiedLog.scala +++ b/core/src/main/scala/kafka/log/es/ElasticUnifiedLog.scala @@ -41,9 +41,11 @@ class ElasticUnifiedLog(_logStartOffset: Long, var confirmOffsetChangeListener: Option[() => Unit] = None elasticLog.confirmOffsetChangeListener = Some(() => confirmOffsetChangeListener.map(_.apply())) + def confirmOffset(): LogOffsetMetadata = { elasticLog.confirmOffset.get() } + override private[log] def replaceSegments(newSegments: collection.Seq[LogSegment], oldSegments: collection.Seq[LogSegment]): Unit = { val deletedSegments = elasticLog.replaceSegments(newSegments, oldSegments) deleteProducerSnapshots(deletedSegments, asyncDelete = true) @@ -75,7 +77,7 @@ class ElasticUnifiedLog(_logStartOffset: Long, // only for testing private[log] def removeAndDeleteSegments(segmentsToDelete: Iterable[LogSegment], - asyncDelete: Boolean): Unit = { + asyncDelete: Boolean): Unit = { elasticLog.removeAndDeleteSegments(segmentsToDelete, asyncDelete, LogDeletion(elasticLog)) } @@ -169,8 +171,8 @@ object ElasticUnifiedLog extends Logging { * @return The new LeaderEpochFileCache instance (if created), none otherwise */ private[log] def maybeCreateLeaderEpochCache(topicPartition: TopicPartition, - recordVersion: RecordVersion, - leaderEpochCheckpoint: ElasticLeaderEpochCheckpoint): Option[LeaderEpochFileCache] = { + recordVersion: RecordVersion, + leaderEpochCheckpoint: ElasticLeaderEpochCheckpoint): Option[LeaderEpochFileCache] = { def newLeaderEpochFileCache(): LeaderEpochFileCache = new LeaderEpochFileCache(topicPartition, leaderEpochCheckpoint) diff --git a/core/src/main/scala/kafka/log/es/LazyStream.java b/core/src/main/scala/kafka/log/es/LazyStream.java index 4b7e98448c..12d538afe6 100644 --- a/core/src/main/scala/kafka/log/es/LazyStream.java +++ b/core/src/main/scala/kafka/log/es/LazyStream.java @@ -17,21 +17,21 @@ package kafka.log.es; -import com.automq.elasticstream.client.api.AppendResult; -import com.automq.elasticstream.client.api.CreateStreamOptions; -import com.automq.elasticstream.client.api.FetchResult; -import com.automq.elasticstream.client.api.OpenStreamOptions; -import com.automq.elasticstream.client.api.RecordBatch; -import com.automq.elasticstream.client.api.Stream; -import com.automq.elasticstream.client.api.StreamClient; +import kafka.log.es.api.AppendResult; +import kafka.log.es.api.CreateStreamOptions; +import kafka.log.es.api.FetchResult; +import kafka.log.es.api.OpenStreamOptions; +import kafka.log.es.api.RecordBatch; +import kafka.log.es.api.Stream; +import kafka.log.es.api.StreamClient; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.Collections; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Lazy stream, create stream when append record. diff --git a/core/src/main/scala/kafka/log/es/MemoryClient.java b/core/src/main/scala/kafka/log/es/MemoryClient.java index 50c8272c3d..117a7fae05 100644 --- a/core/src/main/scala/kafka/log/es/MemoryClient.java +++ b/core/src/main/scala/kafka/log/es/MemoryClient.java @@ -17,17 +17,17 @@ package kafka.log.es; -import com.automq.elasticstream.client.api.AppendResult; -import com.automq.elasticstream.client.api.Client; -import com.automq.elasticstream.client.api.CreateStreamOptions; -import com.automq.elasticstream.client.api.FetchResult; -import com.automq.elasticstream.client.api.KVClient; -import com.automq.elasticstream.client.api.KeyValue; -import com.automq.elasticstream.client.api.OpenStreamOptions; -import com.automq.elasticstream.client.api.RecordBatch; -import com.automq.elasticstream.client.api.RecordBatchWithContext; -import com.automq.elasticstream.client.api.Stream; -import com.automq.elasticstream.client.api.StreamClient; +import kafka.log.es.api.AppendResult; +import kafka.log.es.api.Client; +import kafka.log.es.api.CreateStreamOptions; +import kafka.log.es.api.FetchResult; +import kafka.log.es.api.KVClient; +import kafka.log.es.api.KeyValue; +import kafka.log.es.api.OpenStreamOptions; +import kafka.log.es.api.RecordBatch; +import kafka.log.es.api.RecordBatchWithContext; +import kafka.log.es.api.Stream; +import kafka.log.es.api.StreamClient; import java.nio.ByteBuffer; import java.util.ArrayList; diff --git a/core/src/main/scala/kafka/log/es/MetaStream.java b/core/src/main/scala/kafka/log/es/MetaStream.java index feae520d00..a31eced797 100644 --- a/core/src/main/scala/kafka/log/es/MetaStream.java +++ b/core/src/main/scala/kafka/log/es/MetaStream.java @@ -17,11 +17,16 @@ package kafka.log.es; -import com.automq.elasticstream.client.api.AppendResult; -import com.automq.elasticstream.client.api.FetchResult; -import com.automq.elasticstream.client.api.RecordBatch; -import com.automq.elasticstream.client.api.RecordBatchWithContext; -import com.automq.elasticstream.client.api.Stream; +import io.netty.buffer.Unpooled; +import kafka.log.es.api.AppendResult; +import kafka.log.es.api.FetchResult; +import kafka.log.es.api.RecordBatch; +import kafka.log.es.api.RecordBatchWithContext; +import kafka.log.es.api.Stream; +import org.apache.commons.lang3.tuple.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.IOException; import java.nio.ByteBuffer; import java.util.Collections; @@ -36,11 +41,6 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; -import io.netty.buffer.Unpooled; -import org.apache.commons.lang3.tuple.Pair; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** * Meta stream is a wrapper of stream, it is used to record basic info of a topicPartition. * It serves as a kv stream. @@ -125,6 +125,7 @@ public AppendResult appendSync(MetaKeyValue kv) throws IOException { /** * Append a batch of meta key values without trims. + * * @return a future of append result */ private CompletableFuture append0(MetaKeyValue kv) { @@ -147,7 +148,7 @@ public CompletableFuture close() { trimFuture.cancel(true); } return innerStream.close() - .thenAccept(result -> fenced = true); + .thenAccept(result -> fenced = true); } public boolean isFenced() { @@ -164,6 +165,7 @@ public CompletableFuture destroy() { /** * Replay meta stream and return a map of meta keyValues. KeyValues will be cached in metaCache. + * * @return meta keyValues map */ public Map replay() throws IOException { diff --git a/core/src/main/scala/kafka/log/es/RawPayloadRecordBatch.java b/core/src/main/scala/kafka/log/es/RawPayloadRecordBatch.java index 1d30262b32..969163dcb7 100644 --- a/core/src/main/scala/kafka/log/es/RawPayloadRecordBatch.java +++ b/core/src/main/scala/kafka/log/es/RawPayloadRecordBatch.java @@ -17,7 +17,7 @@ package kafka.log.es; -import com.automq.elasticstream.client.api.RecordBatch; +import kafka.log.es.api.RecordBatch; import java.nio.ByteBuffer; import java.util.Collections; diff --git a/core/src/main/scala/kafka/log/es/RecordBatchWithContextWrapper.java b/core/src/main/scala/kafka/log/es/RecordBatchWithContextWrapper.java index db1627889d..51ff883ac5 100644 --- a/core/src/main/scala/kafka/log/es/RecordBatchWithContextWrapper.java +++ b/core/src/main/scala/kafka/log/es/RecordBatchWithContextWrapper.java @@ -17,9 +17,8 @@ package kafka.log.es; -import com.automq.elasticstream.client.DefaultRecordBatch; -import com.automq.elasticstream.client.api.RecordBatch; -import com.automq.elasticstream.client.api.RecordBatchWithContext; +import kafka.log.es.api.RecordBatch; +import kafka.log.es.api.RecordBatchWithContext; import java.nio.ByteBuffer; import java.util.Collections; @@ -66,10 +65,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(); } diff --git a/core/src/main/scala/kafka/log/es/SeparateSlowAndQuickFetchHint.java b/core/src/main/scala/kafka/log/es/SeparateSlowAndQuickFetchHint.java index a233d67deb..e79e68c0ae 100644 --- a/core/src/main/scala/kafka/log/es/SeparateSlowAndQuickFetchHint.java +++ b/core/src/main/scala/kafka/log/es/SeparateSlowAndQuickFetchHint.java @@ -18,6 +18,7 @@ package kafka.log.es; import io.netty.util.concurrent.FastThreadLocal; + import java.util.concurrent.atomic.AtomicBoolean; /** diff --git a/core/src/main/scala/kafka/log/es/SliceRange.java b/core/src/main/scala/kafka/log/es/SliceRange.java index d497d632eb..1d28e6e2e7 100644 --- a/core/src/main/scala/kafka/log/es/SliceRange.java +++ b/core/src/main/scala/kafka/log/es/SliceRange.java @@ -25,7 +25,8 @@ public class SliceRange { @JsonProperty("e") private long end = Offsets.NOOP_OFFSET; - public SliceRange() {} + public SliceRange() { + } public static SliceRange of(long start, long end) { SliceRange sliceRange = new SliceRange(); diff --git a/core/src/main/scala/kafka/log/es/StreamSliceSupplier.java b/core/src/main/scala/kafka/log/es/StreamSliceSupplier.java index f89ebd108d..27cd82ad0c 100644 --- a/core/src/main/scala/kafka/log/es/StreamSliceSupplier.java +++ b/core/src/main/scala/kafka/log/es/StreamSliceSupplier.java @@ -36,6 +36,7 @@ public ElasticStreamSlice get() throws IOException { /** * reset the slice to an open empty slice. This is used in segment index recovery. + * * @return a new open empty slice */ public ElasticStreamSlice reset() throws IOException { diff --git a/core/src/main/scala/kafka/log/es/api/AppendResult.java b/core/src/main/scala/kafka/log/es/api/AppendResult.java new file mode 100644 index 0000000000..92fd6ea783 --- /dev/null +++ b/core/src/main/scala/kafka/log/es/api/AppendResult.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.es.api; + +/** + * Append RecordBatch to stream result. + */ +public interface AppendResult { + + /** + * Get record batch base offset. + * + * @return record batch base offset. + */ + long baseOffset(); + +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/log/es/api/Client.java b/core/src/main/scala/kafka/log/es/api/Client.java new file mode 100644 index 0000000000..04fb6d5042 --- /dev/null +++ b/core/src/main/scala/kafka/log/es/api/Client.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.es.api; + +/** + * Elastic Stream client. + */ +public interface Client { + /** + * Get stream client. + * + * @return {@link StreamClient} + */ + StreamClient streamClient(); + + /** + * Get KV client. + * + * @return {@link KVClient} + */ + KVClient kvClient(); +} diff --git a/core/src/main/scala/kafka/log/es/api/CreateStreamOptions.java b/core/src/main/scala/kafka/log/es/api/CreateStreamOptions.java new file mode 100644 index 0000000000..471b5904e2 --- /dev/null +++ b/core/src/main/scala/kafka/log/es/api/CreateStreamOptions.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.es.api; + +import kafka.log.es.utils.Arguments; + +public class CreateStreamOptions { + private int replicaCount; + private long epoch; + + public static Builder newBuilder() { + return new Builder(); + } + + public int replicaCount() { + return replicaCount; + } + + public long epoch() { + return epoch; + } + + public static class Builder { + private final CreateStreamOptions options = new CreateStreamOptions(); + + public Builder replicaCount(int replicaCount) { + Arguments.check(replicaCount > 0, "replica count should larger than 0"); + options.replicaCount = replicaCount; + return this; + } + + public Builder epoch(long epoch) { + options.epoch = epoch; + return this; + } + + public CreateStreamOptions build() { + return options; + } + + } +} diff --git a/core/src/main/scala/kafka/log/es/api/ElasticStreamClientException.java b/core/src/main/scala/kafka/log/es/api/ElasticStreamClientException.java new file mode 100644 index 0000000000..c018f5608e --- /dev/null +++ b/core/src/main/scala/kafka/log/es/api/ElasticStreamClientException.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.es.api; + +import java.util.concurrent.ExecutionException; + +/** + * All stream client exceptions will list extends ElasticStreamClientException and list here. + */ +public class ElasticStreamClientException extends ExecutionException { + private final int code; + + public ElasticStreamClientException(int code, String str) { + super("code: " + code + ", " + str); + this.code = code; + } + + public int getCode() { + return this.code; + } +} diff --git a/core/src/main/scala/kafka/log/es/api/ErrorCode.java b/core/src/main/scala/kafka/log/es/api/ErrorCode.java new file mode 100644 index 0000000000..882f9b0ce5 --- /dev/null +++ b/core/src/main/scala/kafka/log/es/api/ErrorCode.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.es.api; + +public class ErrorCode { + public static final short OFFSET_OUT_OF_RANGE_BOUNDS = 1463; + public static final short STREAM_ALREADY_CLOSED = 1478; + public static final short EXPIRED_STREAM_EPOCH = 1489; +} diff --git a/core/src/main/scala/kafka/log/es/api/FetchResult.java b/core/src/main/scala/kafka/log/es/api/FetchResult.java new file mode 100644 index 0000000000..eaeddecc5f --- /dev/null +++ b/core/src/main/scala/kafka/log/es/api/FetchResult.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.es.api; + +import java.util.List; + +public interface FetchResult { + + /** + * Get fetched RecordBatch list. + * + * @return {@link RecordBatchWithContext} list. + */ + List recordBatchList(); + + /** + * Free fetch result backend memory. + */ + default void free() { + } +} diff --git a/core/src/main/scala/kafka/log/es/api/KVClient.java b/core/src/main/scala/kafka/log/es/api/KVClient.java new file mode 100644 index 0000000000..a597704c9d --- /dev/null +++ b/core/src/main/scala/kafka/log/es/api/KVClient.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.es.api; + +import java.util.List; +import java.util.concurrent.CompletableFuture; + +/** + * Light KV client, support light & simple kv operations. + */ +public interface KVClient { + /** + * Put key value. + * + * @param keyValues {@link KeyValue} list. + * @return async put result. + */ + CompletableFuture putKV(List keyValues); + + /** + * Get value by key. + * + * @param keys key list. + * @return {@link KeyValue} list. + */ + CompletableFuture> getKV(List keys); + + /** + * Delete key value by key. + * + * @param keys key list. + * @return async delete result. + */ + CompletableFuture delKV(List keys); +} diff --git a/core/src/main/scala/kafka/log/es/api/KeyValue.java b/core/src/main/scala/kafka/log/es/api/KeyValue.java new file mode 100644 index 0000000000..62ad498774 --- /dev/null +++ b/core/src/main/scala/kafka/log/es/api/KeyValue.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.es.api; + + +import java.nio.ByteBuffer; +import java.util.Objects; + +public class KeyValue { + private final String key; + private final ByteBuffer value; + + private KeyValue(String key, ByteBuffer value) { + this.key = key; + this.value = value; + } + + public static KeyValue of(String key, ByteBuffer value) { + return new KeyValue(key, value); + } + + public String key() { + return key; + } + + public ByteBuffer value() { + return value; + } + + @Override + public boolean equals(Object o) { + 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); + } + + @Override + public int hashCode() { + return Objects.hash(key, value); + } +} diff --git a/core/src/main/scala/kafka/log/es/api/OpenStreamOptions.java b/core/src/main/scala/kafka/log/es/api/OpenStreamOptions.java new file mode 100644 index 0000000000..22d09e38e0 --- /dev/null +++ b/core/src/main/scala/kafka/log/es/api/OpenStreamOptions.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.es.api; + +import kafka.log.es.utils.Arguments; + +public class OpenStreamOptions { + private WriteMode writeMode = WriteMode.SINGLE; + private ReadMode readMode = ReadMode.MULTIPLE; + private long epoch; + + public static Builder newBuilder() { + return new Builder(); + } + + public WriteMode writeMode() { + return writeMode; + } + + public ReadMode readMode() { + return readMode; + } + + public long epoch() { + return epoch; + } + + public enum WriteMode { + SINGLE(0), MULTIPLE(1); + + final int code; + + WriteMode(int code) { + this.code = code; + } + + public int getCode() { + return code; + } + } + + public enum ReadMode { + SINGLE(0), MULTIPLE(1); + + final int code; + + ReadMode(int code) { + this.code = code; + } + + public int getCode() { + return code; + } + } + + public static class Builder { + private final OpenStreamOptions options = new OpenStreamOptions(); + + public Builder writeMode(WriteMode writeMode) { + Arguments.isNotNull(writeMode, "WriteMode should be set with SINGLE or MULTIPLE"); + options.writeMode = writeMode; + return this; + } + + public Builder readMode(ReadMode readMode) { + Arguments.isNotNull(readMode, "ReadMode should be set with SINGLE or MULTIPLE"); + options.readMode = readMode; + return this; + } + + public Builder epoch(long epoch) { + options.epoch = epoch; + return this; + } + + public OpenStreamOptions build() { + return options; + } + } +} diff --git a/core/src/main/scala/kafka/log/es/api/RecordBatch.java b/core/src/main/scala/kafka/log/es/api/RecordBatch.java new file mode 100644 index 0000000000..2c4019a385 --- /dev/null +++ b/core/src/main/scala/kafka/log/es/api/RecordBatch.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.es.api; + +import java.nio.ByteBuffer; +import java.util.Map; + +/** + * Record batch. + */ +public interface RecordBatch { + + /** + * Get payload record count. + * + * @return record count. + */ + int count(); + + /** + * Get min timestamp of records. + * + * @return min timestamp of records. + */ + long baseTimestamp(); + + /** + * Get record batch extension properties. + * + * @return batch extension properties. + */ + Map properties(); + + /** + * Get raw payload. + * + * @return raw payload. + */ + ByteBuffer rawPayload(); +} diff --git a/core/src/main/scala/kafka/log/es/api/RecordBatchWithContext.java b/core/src/main/scala/kafka/log/es/api/RecordBatchWithContext.java new file mode 100644 index 0000000000..9735b0664a --- /dev/null +++ b/core/src/main/scala/kafka/log/es/api/RecordBatchWithContext.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.es.api; + +public interface RecordBatchWithContext extends RecordBatch { + + /** + * Get record batch base offset. + * + * @return base offset. + */ + long baseOffset(); + + /** + * Get record batch exclusive last offset. + * + * @return exclusive last offset. + */ + long lastOffset(); +} diff --git a/core/src/main/scala/kafka/log/es/api/Stream.java b/core/src/main/scala/kafka/log/es/api/Stream.java new file mode 100644 index 0000000000..bc1b5c59e1 --- /dev/null +++ b/core/src/main/scala/kafka/log/es/api/Stream.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.es.api; + +import java.util.concurrent.CompletableFuture; + +/** + * Record stream. + */ +public interface Stream { + + /** + * Get stream id + */ + long streamId(); + + /** + * Get stream start offset. + */ + long startOffset(); + + /** + * Get stream next append record offset. + */ + long nextOffset(); + + + /** + * Append recordBatch to stream. + * + * @param recordBatch {@link RecordBatch}. + * @return - complete success with async {@link AppendResult}, when append success. + * - complete exception with {@link ElasticStreamClientException}, when append fail. TODO: specify the exception. + */ + CompletableFuture append(RecordBatch recordBatch); + + /** + * Fetch recordBatch list from stream. Note the startOffset may be in the middle in the first recordBatch. + * + * @param startOffset start offset, if the startOffset in middle of a recordBatch, the recordBatch will be returned. + * @param endOffset exclusive end offset, if the endOffset in middle of a recordBatch, the recordBatch will be returned. + * @param maxBytesHint max fetch data size hint, the real return data size may be larger than maxBytesHint. + * @return - complete success with {@link FetchResult}, when fetch success. + * - complete exception with {@link ElasticStreamClientException}, when startOffset is bigger than stream end offset. + */ + CompletableFuture fetch(long startOffset, long endOffset, int maxBytesHint); + + /** + * Trim stream. + * + * @param newStartOffset new start offset. + * @return - complete success with async {@link Void}, when trim success. + * - complete exception with {@link ElasticStreamClientException}, when trim fail. + */ + CompletableFuture trim(long newStartOffset); + + /** + * Close the stream. + */ + CompletableFuture close(); + + /** + * Destroy stream. + */ + CompletableFuture destroy(); +} diff --git a/core/src/main/scala/kafka/log/es/api/StreamClient.java b/core/src/main/scala/kafka/log/es/api/StreamClient.java new file mode 100644 index 0000000000..e7c94b34d5 --- /dev/null +++ b/core/src/main/scala/kafka/log/es/api/StreamClient.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.es.api; + +import java.util.concurrent.CompletableFuture; + +/** + * Stream client, support stream create and open operation. + */ +public interface StreamClient { + /** + * Create and open stream. + * + * @param options create stream options. + * @return {@link Stream}. + */ + CompletableFuture createAndOpenStream(CreateStreamOptions options); + + /** + * Open stream. + * + * @param streamId stream id. + * @param options open stream options. + * @return {@link Stream}. + */ + CompletableFuture openStream(long streamId, OpenStreamOptions options); +} diff --git a/core/src/main/scala/kafka/log/es/client/ClientFactoryProxy.java b/core/src/main/scala/kafka/log/es/client/ClientFactoryProxy.java index 0da3758894..7b9b07350c 100644 --- a/core/src/main/scala/kafka/log/es/client/ClientFactoryProxy.java +++ b/core/src/main/scala/kafka/log/es/client/ClientFactoryProxy.java @@ -17,18 +17,18 @@ package kafka.log.es.client; -import com.automq.elasticstream.client.api.Client; +import kafka.log.es.api.Client; import java.lang.reflect.Method; public class ClientFactoryProxy { - private static final String PROTOCOL_SEPERATOR = ":"; + private static final String PROTOCOL_SEPARATOR = ":"; private static final String FACTORY_CLASS_FORMAT = "kafka.log.es.client.%s.ClientFactory"; public static Client get(Context context) { String endpoint = context.config.elasticStreamEndpoint(); - String protocal = endpoint.split(PROTOCOL_SEPERATOR)[0]; - String className = String.format(FACTORY_CLASS_FORMAT, protocal); + String protocol = endpoint.split(PROTOCOL_SEPARATOR)[0]; + String className = String.format(FACTORY_CLASS_FORMAT, protocol); try { Class clazz = Class.forName(className); Method method = clazz.getDeclaredMethod("get", Context.class); diff --git a/core/src/main/scala/kafka/log/es/client/es/ClientFactory.java b/core/src/main/scala/kafka/log/es/client/es/ClientFactory.java deleted file mode 100644 index c9e242f7b0..0000000000 --- a/core/src/main/scala/kafka/log/es/client/es/ClientFactory.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.log.es.client.es; - -import com.automq.elasticstream.client.DefaultClientBuilder; -import com.automq.elasticstream.client.api.Client; -import kafka.log.es.AlwaysSuccessClient; -import kafka.log.es.client.Context; - -public class ClientFactory { - private static final String ES_ENDPOINT_PREFIX = "es://"; - - public static Client get(Context context) { - String endpoint = context.config.elasticStreamEndpoint(); - String kvEndpoint = context.config.elasticStreamKvEndpoint(); - if (!kvEndpoint.startsWith(ES_ENDPOINT_PREFIX)) { - throw new IllegalArgumentException("Elastic stream endpoint and kvEndpoint must be the same protocol: " + endpoint + " " + kvEndpoint); - } - Client client = new DefaultClientBuilder() - .endpoint(endpoint.substring(ES_ENDPOINT_PREFIX.length())) - .kvEndpoint(kvEndpoint.substring(ES_ENDPOINT_PREFIX.length())) - .build(); - return new AlwaysSuccessClient(client); - } - -} diff --git a/core/src/main/scala/kafka/log/es/client/memory/ClientFactory.java b/core/src/main/scala/kafka/log/es/client/memory/ClientFactory.java index 91576ead12..d386890d86 100644 --- a/core/src/main/scala/kafka/log/es/client/memory/ClientFactory.java +++ b/core/src/main/scala/kafka/log/es/client/memory/ClientFactory.java @@ -17,10 +17,10 @@ package kafka.log.es.client.memory; -import com.automq.elasticstream.client.api.Client; -import kafka.log.es.client.Context; import kafka.log.es.AlwaysSuccessClient; import kafka.log.es.MemoryClient; +import kafka.log.es.api.Client; +import kafka.log.es.client.Context; public class ClientFactory { diff --git a/core/src/main/scala/kafka/log/es/client/redis/ClientFactory.java b/core/src/main/scala/kafka/log/es/client/redis/ClientFactory.java index fe9c03a886..23ce682232 100644 --- a/core/src/main/scala/kafka/log/es/client/redis/ClientFactory.java +++ b/core/src/main/scala/kafka/log/es/client/redis/ClientFactory.java @@ -17,8 +17,8 @@ package kafka.log.es.client.redis; -import com.automq.elasticstream.client.api.Client; import kafka.log.es.ElasticRedisClient; +import kafka.log.es.api.Client; import kafka.log.es.client.Context; public class ClientFactory { diff --git a/core/src/main/scala/kafka/log/es/client/s3/ClientFactory.java b/core/src/main/scala/kafka/log/es/client/s3/ClientFactory.java index 33a75f280b..d19c0a6ee8 100644 --- a/core/src/main/scala/kafka/log/es/client/s3/ClientFactory.java +++ b/core/src/main/scala/kafka/log/es/client/s3/ClientFactory.java @@ -17,8 +17,8 @@ package kafka.log.es.client.s3; -import com.automq.elasticstream.client.api.Client; import kafka.log.es.AlwaysSuccessClient; +import kafka.log.es.api.Client; import kafka.log.es.client.Context; import kafka.log.s3.DefaultS3Client; import kafka.log.s3.operator.DefaultS3Operator; diff --git a/core/src/main/scala/kafka/log/es/utils/Arguments.java b/core/src/main/scala/kafka/log/es/utils/Arguments.java new file mode 100644 index 0000000000..0eb78eaa5a --- /dev/null +++ b/core/src/main/scala/kafka/log/es/utils/Arguments.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.es.utils; + +public class Arguments { + + public static void check(boolean checkResult, String errorMessage) { + if (!checkResult) { + throw new IllegalArgumentException(errorMessage); + } + } + + public static void isNotNull(Object obj, String errorMessage) { + if (obj == null) { + throw new IllegalArgumentException(errorMessage); + } + } + +} diff --git a/core/src/main/scala/kafka/log/es/utils/Threads.java b/core/src/main/scala/kafka/log/es/utils/Threads.java new file mode 100644 index 0000000000..94cee7d080 --- /dev/null +++ b/core/src/main/scala/kafka/log/es/utils/Threads.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log.es.utils; + +import org.slf4j.Logger; + +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.ThreadFactory; + +public class Threads { + + public static ScheduledExecutorService newSingleThreadScheduledExecutor(ThreadFactory threadFactory, Logger logger) { + return new ScheduledThreadPoolExecutor(1, threadFactory) { + @Override + protected void afterExecute(Runnable r, Throwable t) { + super.afterExecute(r, t); + if (t != null) { + logger.error("[FATAL] Uncaught exception in executor thread {}", Thread.currentThread().getName(), t); + } + } + }; + } + +} diff --git a/core/src/main/scala/kafka/log/s3/ControllerKVClient.java b/core/src/main/scala/kafka/log/s3/ControllerKVClient.java index c3b59b7a28..46d74e7802 100644 --- a/core/src/main/scala/kafka/log/s3/ControllerKVClient.java +++ b/core/src/main/scala/kafka/log/s3/ControllerKVClient.java @@ -17,8 +17,8 @@ package kafka.log.s3; -import com.automq.elasticstream.client.api.KVClient; -import com.automq.elasticstream.client.api.KeyValue; +import kafka.log.es.api.KVClient; +import kafka.log.es.api.KeyValue; import kafka.log.s3.network.ControllerRequestSender; import org.apache.kafka.common.message.DeleteKVRequestData; import org.apache.kafka.common.message.GetKVRequestData; diff --git a/core/src/main/scala/kafka/log/s3/DefaultS3Client.java b/core/src/main/scala/kafka/log/s3/DefaultS3Client.java index 5aa85c1611..184e6a9750 100644 --- a/core/src/main/scala/kafka/log/s3/DefaultS3Client.java +++ b/core/src/main/scala/kafka/log/s3/DefaultS3Client.java @@ -17,9 +17,9 @@ package kafka.log.s3; -import com.automq.elasticstream.client.api.Client; -import com.automq.elasticstream.client.api.KVClient; -import com.automq.elasticstream.client.api.StreamClient; +import kafka.log.es.api.Client; +import kafka.log.es.api.KVClient; +import kafka.log.es.api.StreamClient; import kafka.log.s3.cache.DefaultS3BlockCache; import kafka.log.s3.cache.S3BlockCache; import kafka.log.s3.compact.CompactionManager; diff --git a/core/src/main/scala/kafka/log/s3/S3Storage.java b/core/src/main/scala/kafka/log/s3/S3Storage.java index 13ebbc79d0..ed543c1236 100644 --- a/core/src/main/scala/kafka/log/s3/S3Storage.java +++ b/core/src/main/scala/kafka/log/s3/S3Storage.java @@ -17,7 +17,7 @@ package kafka.log.s3; -import kafka.log.es.FutureUtil; +import kafka.log.es.utils.Threads; import kafka.log.s3.cache.LogCache; import kafka.log.s3.cache.ReadDataBlock; import kafka.log.s3.cache.S3BlockCache; @@ -32,6 +32,7 @@ import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -41,29 +42,32 @@ import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.Executors; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ScheduledExecutorService; +import static kafka.log.es.FutureUtil.exec; +import static kafka.log.es.FutureUtil.propagate; + public class S3Storage implements Storage { private static final Logger LOGGER = LoggerFactory.getLogger(S3Storage.class); + private final long maxWALCacheSize; private final KafkaConfig config; private final WriteAheadLog log; private final LogCache logCache; private final WALCallbackSequencer callbackSequencer = new WALCallbackSequencer(); private final Queue walObjectPrepareQueue = new LinkedList<>(); private final Queue walObjectCommitQueue = new LinkedList<>(); - private final ScheduledExecutorService mainExecutor = Executors.newSingleThreadScheduledExecutor( - ThreadUtils.createThreadFactory("s3-storage-main", false)); - private final ScheduledExecutorService backgroundExecutor = Executors.newSingleThreadScheduledExecutor( - ThreadUtils.createThreadFactory("s3-storage-main", true)); + private final ScheduledExecutorService mainExecutor = Threads.newSingleThreadScheduledExecutor( + ThreadUtils.createThreadFactory("s3-storage-main", false), LOGGER); + private final ScheduledExecutorService backgroundExecutor = Threads.newSingleThreadScheduledExecutor( + ThreadUtils.createThreadFactory("s3-storage-background", true), LOGGER); private final ObjectManager objectManager; private final S3Operator s3Operator; private final S3BlockCache blockCache; public S3Storage(KafkaConfig config, WriteAheadLog log, ObjectManager objectManager, S3BlockCache blockCache, S3Operator s3Operator) { this.config = config; + this.maxWALCacheSize = config.s3WALCacheSize(); this.log = log; this.logCache = new LogCache(config.s3WALObjectSize()); this.objectManager = objectManager; @@ -79,12 +83,23 @@ public void close() { @Override public CompletableFuture append(StreamRecordBatch streamRecord) { - for (;;) { - if (logCache.size() < 2L * 1024 * 1024 * 1024) { + CompletableFuture cf = new CompletableFuture<>(); + acquirePermit(); + // TODO: catch log over capacity exception and retry. + WriteAheadLog.AppendResult appendResult = log.append(streamRecord.encoded()); + WalWriteRequest writeRequest = new WalWriteRequest(streamRecord, appendResult.offset, cf); + handleAppendRequest(writeRequest); + appendResult.future.thenAccept(nil -> handleAppendCallback(writeRequest)); + return cf; + } + + private void acquirePermit() { + for (; ; ) { + if (logCache.size() < maxWALCacheSize) { break; } else { // TODO: log limit - LOGGER.warn("log cache size {} is larger than 2GB, wait 100ms", logCache.size()); + LOGGER.warn("log cache size {} is larger than {}, wait 100ms", maxWALCacheSize, logCache.size()); try { //noinspection BusyWait Thread.sleep(100); @@ -93,19 +108,12 @@ public CompletableFuture append(StreamRecordBatch streamRecord) { } } } - - WriteAheadLog.AppendResult appendResult = log.append(streamRecord.encoded()); - CompletableFuture cf = new CompletableFuture<>(); - WalWriteRequest writeRequest = new WalWriteRequest(streamRecord, appendResult.offset, cf); - callbackSequencer.before(writeRequest); - appendResult.future.thenAccept(nil -> handleAppendCallback(writeRequest)); - return cf; } @Override public CompletableFuture read(long streamId, long startOffset, long endOffset, int maxBytes) { CompletableFuture cf = new CompletableFuture<>(); - mainExecutor.execute(() -> FutureUtil.propagate(read0(streamId, startOffset, endOffset, maxBytes), cf)); + mainExecutor.execute(() -> propagate(read0(streamId, startOffset, endOffset, maxBytes), cf)); return cf; } @@ -132,27 +140,34 @@ public CompletableFuture forceUpload(long streamId) { mainExecutor.execute(() -> { Optional blockOpt = logCache.archiveCurrentBlockIfContains(streamId); if (blockOpt.isPresent()) { - blockOpt.ifPresent(logCacheBlock -> FutureUtil.propagate(uploadWALObject(logCacheBlock), cf)); + blockOpt.ifPresent(logCacheBlock -> propagate(uploadWALObject(logCacheBlock), cf)); } else { cf.complete(null); } + callbackSequencer.tryFree(streamId); }); return cf; } + private void handleAppendRequest(WalWriteRequest request) { + mainExecutor.execute(() -> callbackSequencer.before(request)); + } + private void handleAppendCallback(WalWriteRequest request) { - mainExecutor.execute(() -> { - List waitingAckRequests = callbackSequencer.after(request); - for (WalWriteRequest waitingAckRequest : waitingAckRequests) { - if (logCache.put(waitingAckRequest.record)) { - // cache block is full, trigger WAL object upload. - logCache.setConfirmOffset(callbackSequencer.getWALConfirmOffset()); - LogCache.LogCacheBlock logCacheBlock = logCache.archiveCurrentBlock(); - uploadWALObject(logCacheBlock); - } - waitingAckRequest.cf.complete(null); + mainExecutor.execute(() -> handleAppendCallback0(request)); + } + + private void handleAppendCallback0(WalWriteRequest request) { + List waitingAckRequests = callbackSequencer.after(request); + for (WalWriteRequest waitingAckRequest : waitingAckRequests) { + if (logCache.put(waitingAckRequest.record)) { + // cache block is full, trigger WAL object upload. + logCache.setConfirmOffset(callbackSequencer.getWALConfirmOffset()); + LogCache.LogCacheBlock logCacheBlock = logCache.archiveCurrentBlock(); + uploadWALObject(logCacheBlock); } - }); + waitingAckRequest.cf.complete(null); + } } /** @@ -160,7 +175,7 @@ private void handleAppendCallback(WalWriteRequest request) { */ CompletableFuture uploadWALObject(LogCache.LogCacheBlock logCacheBlock) { CompletableFuture cf = new CompletableFuture<>(); - backgroundExecutor.execute(() -> uploadWALObject0(logCacheBlock, cf)); + backgroundExecutor.execute(() -> exec(() -> uploadWALObject0(logCacheBlock, cf), cf, LOGGER, "uploadWALObject")); return cf; } @@ -224,9 +239,12 @@ private void freeCache(LogCache.LogCacheBlock cacheBlock) { }); } + /** + * WALCallbackSequencer is modified in single thread mainExecutor. + */ static class WALCallbackSequencer { public static final long NOOP_OFFSET = -1L; - private final Map> stream2requests = new ConcurrentHashMap<>(); + private final Map> stream2requests = new HashMap<>(); private final BlockingQueue walRequests = new ArrayBlockingQueue<>(4096); private long walConfirmOffset = NOOP_OFFSET; @@ -238,7 +256,7 @@ public void before(WalWriteRequest request) { walRequests.put(request); Queue streamRequests = stream2requests.computeIfAbsent(request.record.getStreamId(), s -> new LinkedBlockingQueue<>()); streamRequests.add(request); - } catch (InterruptedException ex) { + } catch (Throwable ex) { request.cf.completeExceptionally(ex); } } @@ -276,9 +294,6 @@ public List after(WalWriteRequest request) { } rst.add(streamRequests.poll()); } - if (streamRequests.isEmpty()) { - stream2requests.computeIfPresent(streamId, (id, requests) -> requests.isEmpty() ? null : requests); - } return rst; } @@ -291,6 +306,15 @@ public long getWALConfirmOffset() { return walConfirmOffset; } + /** + * Try free stream related resources. + */ + public void tryFree(long streamId) { + Queue queue = stream2requests.get(streamId); + if (queue != null && queue.isEmpty()) { + stream2requests.remove(streamId, queue); + } + } } static class WALObjectUploadTaskContext { diff --git a/core/src/main/scala/kafka/log/s3/S3Stream.java b/core/src/main/scala/kafka/log/s3/S3Stream.java index 5f0ddd623f..8b4c731bdc 100644 --- a/core/src/main/scala/kafka/log/s3/S3Stream.java +++ b/core/src/main/scala/kafka/log/s3/S3Stream.java @@ -17,17 +17,17 @@ package kafka.log.s3; -import com.automq.elasticstream.client.DefaultAppendResult; -import com.automq.elasticstream.client.api.AppendResult; -import com.automq.elasticstream.client.api.ElasticStreamClientException; -import com.automq.elasticstream.client.api.FetchResult; -import com.automq.elasticstream.client.api.RecordBatch; -import com.automq.elasticstream.client.api.RecordBatchWithContext; -import com.automq.elasticstream.client.api.Stream; -import com.automq.elasticstream.client.flatc.header.ErrorCode; import io.netty.buffer.Unpooled; +import kafka.log.es.DefaultAppendResult; import kafka.log.es.FutureUtil; import kafka.log.es.RecordBatchWithContextWrapper; +import kafka.log.es.api.AppendResult; +import kafka.log.es.api.ElasticStreamClientException; +import kafka.log.es.api.ErrorCode; +import kafka.log.es.api.FetchResult; +import kafka.log.es.api.RecordBatch; +import kafka.log.es.api.RecordBatchWithContext; +import kafka.log.es.api.Stream; import kafka.log.s3.model.StreamRecordBatch; import kafka.log.s3.streams.StreamManager; import org.slf4j.Logger; diff --git a/core/src/main/scala/kafka/log/s3/S3StreamClient.java b/core/src/main/scala/kafka/log/s3/S3StreamClient.java index 768ba06a52..b35eb0731e 100644 --- a/core/src/main/scala/kafka/log/s3/S3StreamClient.java +++ b/core/src/main/scala/kafka/log/s3/S3StreamClient.java @@ -17,10 +17,10 @@ package kafka.log.s3; -import com.automq.elasticstream.client.api.CreateStreamOptions; -import com.automq.elasticstream.client.api.OpenStreamOptions; -import com.automq.elasticstream.client.api.Stream; -import com.automq.elasticstream.client.api.StreamClient; +import kafka.log.es.api.CreateStreamOptions; +import kafka.log.es.api.OpenStreamOptions; +import kafka.log.es.api.Stream; +import kafka.log.es.api.StreamClient; import kafka.log.s3.streams.StreamManager; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/core/src/main/scala/kafka/log/s3/metadata/StreamMetadataManager.java b/core/src/main/scala/kafka/log/s3/metadata/StreamMetadataManager.java index bd4777a276..1851893d3f 100644 --- a/core/src/main/scala/kafka/log/s3/metadata/StreamMetadataManager.java +++ b/core/src/main/scala/kafka/log/s3/metadata/StreamMetadataManager.java @@ -30,6 +30,8 @@ import org.apache.kafka.metadata.stream.S3Object; import org.apache.kafka.metadata.stream.S3ObjectMetadata; import org.apache.kafka.metadata.stream.S3WALObjectMetadata; +import org.apache.kafka.metadata.stream.S3StreamObject; +import org.apache.kafka.metadata.stream.S3StreamObjectMetadata; import org.apache.kafka.metadata.stream.StreamOffsetRange; import org.apache.kafka.raft.OffsetAndEpoch; import org.slf4j.Logger; @@ -177,6 +179,24 @@ public CompletableFuture fetch(long streamId, long startOffset, } } + public CompletableFuture> getStreamObjects(long streamId, long startOffset, long endOffset, int limit) { + synchronized (StreamMetadataManager.this) { + try { + List streamObjects = streamsImage.getStreamObjects(streamId, startOffset, endOffset, limit); + List s3StreamObjectMetadataList = streamObjects.stream().map(object -> { + long committedTimeInMs = objectsImage.getObjectMetadata(object.objectId()).getCommittedTimeInMs(); + return new S3StreamObjectMetadata(object, committedTimeInMs); + }).collect(Collectors.toList()); + return CompletableFuture.completedFuture(s3StreamObjectMetadataList); + } catch (Exception e) { + LOGGER.warn( + "[GetStreamObjects]: stream: {}, startOffset: {}, endOffset: {}, limit: {}, and search in metadataCache failed with exception: {}", + streamId, startOffset, endOffset, limit, e.getMessage()); + return CompletableFuture.failedFuture(e); + } + } + } + // must access thread safe private CompletableFuture pendingFetch(long streamId, long startOffset, long endOffset, int limit) { GetObjectsTask task = GetObjectsTask.of(streamId, startOffset, endOffset, limit); diff --git a/core/src/main/scala/kafka/log/s3/model/StreamRecordBatch.java b/core/src/main/scala/kafka/log/s3/model/StreamRecordBatch.java index c599f75f3c..d208a063aa 100644 --- a/core/src/main/scala/kafka/log/s3/model/StreamRecordBatch.java +++ b/core/src/main/scala/kafka/log/s3/model/StreamRecordBatch.java @@ -17,8 +17,8 @@ package kafka.log.s3.model; -import com.automq.elasticstream.client.api.RecordBatch; import io.netty.buffer.ByteBuf; +import kafka.log.es.api.RecordBatch; import kafka.log.s3.StreamRecordBatchCodec; import java.nio.ByteBuffer; diff --git a/core/src/main/scala/kafka/log/s3/objects/CommitWALObjectRequest.java b/core/src/main/scala/kafka/log/s3/objects/CommitWALObjectRequest.java index 0b97e47824..46445119e2 100644 --- a/core/src/main/scala/kafka/log/s3/objects/CommitWALObjectRequest.java +++ b/core/src/main/scala/kafka/log/s3/objects/CommitWALObjectRequest.java @@ -132,12 +132,12 @@ public void setOrderId(long orderId) { @Override public String toString() { return "CommitWALObjectRequest{" + - "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/core/src/main/scala/kafka/log/s3/objects/ControllerObjectManager.java b/core/src/main/scala/kafka/log/s3/objects/ControllerObjectManager.java index 70d5f14cde..2d445f8358 100644 --- a/core/src/main/scala/kafka/log/s3/objects/ControllerObjectManager.java +++ b/core/src/main/scala/kafka/log/s3/objects/ControllerObjectManager.java @@ -75,18 +75,18 @@ public CompletableFuture prepareObject(int count, long ttl) { @Override public CompletableFuture commitWALObject(CommitWALObjectRequest request) { org.apache.kafka.common.requests.s3.CommitWALObjectRequest.Builder wrapRequestBuilder = new org.apache.kafka.common.requests.s3.CommitWALObjectRequest.Builder( - new CommitWALObjectRequestData() - .setBrokerId(config.brokerId()) - .setOrderId(request.getOrderId()) - .setObjectId(request.getObjectId()) - .setObjectSize(request.getObjectSize()) - .setObjectStreamRanges(request.getStreamRanges() - .stream() - .map(ObjectStreamRange::toObjectStreamRangeInRequest).collect(Collectors.toList())) - .setStreamObjects(request.getStreamObjects() - .stream() - .map(StreamObject::toStreamObjectInRequest).collect(Collectors.toList())) - .setCompactedObjectIds(request.getCompactedObjectIds())); + new CommitWALObjectRequestData() + .setBrokerId(config.brokerId()) + .setOrderId(request.getOrderId()) + .setObjectId(request.getObjectId()) + .setObjectSize(request.getObjectSize()) + .setObjectStreamRanges(request.getStreamRanges() + .stream() + .map(ObjectStreamRange::toObjectStreamRangeInRequest).collect(Collectors.toList())) + .setStreamObjects(request.getStreamObjects() + .stream() + .map(StreamObject::toStreamObjectInRequest).collect(Collectors.toList())) + .setCompactedObjectIds(request.getCompactedObjectIds())); return requestSender.send(wrapRequestBuilder, CommitWALObjectResponseData.class).thenApply(resp -> { Errors code = Errors.forCode(resp.errorCode()); switch (code) { diff --git a/core/src/main/scala/kafka/log/s3/operator/DefaultS3Operator.java b/core/src/main/scala/kafka/log/s3/operator/DefaultS3Operator.java index 738cc05e81..c0b78513c8 100644 --- a/core/src/main/scala/kafka/log/s3/operator/DefaultS3Operator.java +++ b/core/src/main/scala/kafka/log/s3/operator/DefaultS3Operator.java @@ -62,6 +62,7 @@ public class DefaultS3Operator implements S3Operator { private final String bucket; private final S3AsyncClient s3; private static final Timer PART_UPLOAD_COST = new Timer(); + private static final Timer OBJECT_INTO_CLOSE_COST = new Timer(); private static final Timer OBJECT_UPLOAD_COST = new Timer(); private static final Counter OBJECT_UPLOAD_SIZE = new Counter(); private static final AtomicLong LAST_LOG_TIMESTAMP = new AtomicLong(System.currentTimeMillis()); @@ -270,23 +271,24 @@ public CompletableFuture close() { if (closeCf != null) { return closeCf; } - System.out.println("start await close: " + (System.nanoTime() - start) / 1000 / 1000); + OBJECT_INTO_CLOSE_COST.update(System.nanoTime() - start); closeCf = new CompletableFuture<>(); CompletableFuture uploadDoneCf = uploadIdCf.thenCompose(uploadId -> CompletableFuture.allOf(parts.toArray(new CompletableFuture[0]))); uploadDoneCf.thenAccept(nil -> { - System.out.println("start complete: " + (System.nanoTime() - start) / 1000 / 1000); CompletedMultipartUpload multipartUpload = CompletedMultipartUpload.builder().parts(genCompleteParts()).build(); CompleteMultipartUploadRequest request = CompleteMultipartUploadRequest.builder().bucket(bucket).key(path).uploadId(uploadId).multipartUpload(multipartUpload).build(); close0(request); }); closeCf.whenComplete((nil, ex) -> { - System.out.println("complete: " + (System.nanoTime() - start) / 1000 / 1000); OBJECT_UPLOAD_COST.update(System.nanoTime() - start); long now = System.currentTimeMillis(); if (now - LAST_LOG_TIMESTAMP.get() > 10000) { LAST_LOG_TIMESTAMP.set(now); - LOGGER.info("upload s3 metrics, object_timer {}, object_size {}, part_timer {}", - OBJECT_UPLOAD_COST.getAndReset(), OBJECT_UPLOAD_SIZE.getAndReset(), PART_UPLOAD_COST.getAndReset()); + LOGGER.info("upload s3 metrics, object_part_upload_timer {}, object_into_close_timer {}, object_upload_timer {}, object_upload_size {}", + PART_UPLOAD_COST.getAndReset(), + OBJECT_INTO_CLOSE_COST.getAndReset(), + OBJECT_UPLOAD_COST.getAndReset(), + OBJECT_UPLOAD_SIZE.getAndReset()); } }); return closeCf; diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 07f878fa29..62160b9726 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -687,6 +687,8 @@ object KafkaConfig { val S3EndpointProp = "s3.endpoint" val S3RegionProp = "s3.region" val S3BucketProp = "s3.bucket" + val S3WALCacheSizeProp = "s3.wal.cache.size" + val S3WALLogSizeProp = "s3.wal.log.size" val S3WALObjectSizeProp = "s3.wal.object.size" val S3StreamSplitSizeProp = "s3.stream.object.split.size" val S3ObjectBlockSizeProp = "s3.object.block.size" @@ -702,6 +704,9 @@ object KafkaConfig { val S3EndpointDoc = "The S3 endpoint, ex. https://s3.{region}.amazonaws.com." val S3RegionDoc = "The S3 region, ex. us-east-1." val S3BucketDoc = "The S3 bucket, ex. my-bucket." + val S3WALCacheSizeDoc = "The S3 storage max WAL cache size. When WAL cache is full, storage will hang the request, \n" + + "until WAL cache is free by S3 WAL object upload." + val S3WALLogSizeDoc = "The S3 WAL log max size. The value should be larger than s3.wal.cache.size cause of log storage format may not compact." val S3WALObjectSizeDoc = "The S3 WAL object size threshold." val S3StreamSplitSizeDoc = "The S3 stream object split size threshold when upload WAL object or compact object." val S3ObjectBlockSizeDoc = "The S3 object compressed block size threshold." @@ -1516,11 +1521,13 @@ object KafkaConfig { .define(S3EndpointProp, STRING, null, HIGH, S3EndpointDoc) .define(S3RegionProp, STRING, null, HIGH, S3RegionDoc) .define(S3BucketProp, STRING, null, HIGH, S3BucketDoc) - .define(S3WALObjectSizeProp, LONG, 524288000, MEDIUM, S3WALObjectSizeDoc) + .define(S3WALCacheSizeProp, LONG, 1073741824L, MEDIUM, S3WALCacheSizeDoc) + .define(S3WALLogSizeProp, LONG, 2147483648L, MEDIUM, S3WALLogSizeDoc) + .define(S3WALObjectSizeProp, LONG, 524288000L, MEDIUM, S3WALObjectSizeDoc) .define(S3StreamSplitSizeProp, INT, 16777216, MEDIUM, S3StreamSplitSizeDoc) .define(S3ObjectBlockSizeProp, INT, 8388608, MEDIUM, S3ObjectBlockSizeDoc) .define(S3ObjectPartSizeProp, INT, 16777216, MEDIUM, S3ObjectPartSizeDoc) - .define(S3CacheSizeProp, LONG, 1073741824, MEDIUM, S3CacheSizeDoc) + .define(S3CacheSizeProp, LONG, 1073741824L, MEDIUM, S3CacheSizeDoc) .define(S3ObjectCompactionCacheSize, LONG, Defaults.S3ObjectCompactionCacheSize, MEDIUM, S3ObjectCompactionCacheSizeDoc) .define(S3ObjectCompactionNWInBandwidth, LONG, Defaults.S3ObjectCompactionNWInBandwidth, MEDIUM, S3ObjectCompactionNWInBandwidthDoc) .define(S3ObjectCompactionNWOutBandwidth, LONG, Defaults.S3ObjectCompactionNWOutBandwidth, MEDIUM, S3ObjectCompactionNWOutBandwidthDoc) @@ -2066,6 +2073,8 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val s3Endpoint = getString(KafkaConfig.S3EndpointProp) val s3Region = getString(KafkaConfig.S3RegionProp) val s3Bucket = getString(KafkaConfig.S3BucketProp) + val s3WALCacheSize = getLong(KafkaConfig.S3WALCacheSizeProp) + val s3WALLogSize = getLong(KafkaConfig.S3WALLogSizeProp) val s3WALObjectSize = getLong(KafkaConfig.S3WALObjectSizeProp) val s3StreamSplitSize = getInt(KafkaConfig.S3StreamSplitSizeProp) val s3ObjectBlockSize = getInt(KafkaConfig.S3ObjectBlockSizeProp) diff --git a/core/src/test/java/kafka/log/es/AlwaysSuccessClientTest.java b/core/src/test/java/kafka/log/es/AlwaysSuccessClientTest.java index 926942c131..e31f7ec171 100644 --- a/core/src/test/java/kafka/log/es/AlwaysSuccessClientTest.java +++ b/core/src/test/java/kafka/log/es/AlwaysSuccessClientTest.java @@ -17,15 +17,20 @@ package kafka.log.es; -import com.automq.elasticstream.client.api.AppendResult; -import com.automq.elasticstream.client.api.CreateStreamOptions; -import com.automq.elasticstream.client.api.ElasticStreamClientException; -import com.automq.elasticstream.client.api.FetchResult; -import com.automq.elasticstream.client.api.OpenStreamOptions; -import com.automq.elasticstream.client.api.RecordBatch; -import com.automq.elasticstream.client.api.RecordBatchWithContext; -import com.automq.elasticstream.client.api.Stream; -import com.automq.elasticstream.client.api.StreamClient; +import kafka.log.es.api.AppendResult; +import kafka.log.es.api.CreateStreamOptions; +import kafka.log.es.api.ElasticStreamClientException; +import kafka.log.es.api.FetchResult; +import kafka.log.es.api.OpenStreamOptions; +import kafka.log.es.api.RecordBatch; +import kafka.log.es.api.RecordBatchWithContext; +import kafka.log.es.api.Stream; +import kafka.log.es.api.StreamClient; +import org.apache.kafka.common.errors.es.SlowFetchHintException; +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; @@ -42,19 +47,12 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; -import org.apache.kafka.common.errors.es.SlowFetchHintException; -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 static kafka.log.es.AlwaysSuccessClient.HALT_ERROR_CODES; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @Tag("esUnit") class AlwaysSuccessClientTest { - private static final long SLOW_FETCH_TIMEOUT_MILLIS = AlwaysSuccessClient.SLOW_FETCH_TIMEOUT_MILLIS; private AlwaysSuccessClient client; @BeforeEach @@ -91,8 +89,9 @@ public void basicAppendAndFetch() throws ExecutionException, InterruptedExceptio @Test public void testQuickFetch() throws ExecutionException, InterruptedException { MemoryClientWithDelay memoryClientWithDelay = new MemoryClientWithDelay(); - client = new AlwaysSuccessClient(memoryClientWithDelay); - List quickFetchDelayMillisList = List.of(1L, SLOW_FETCH_TIMEOUT_MILLIS / 2); + long slowFetchTimeoutMillis = 1000 * 2; + client = new AlwaysSuccessClient(memoryClientWithDelay, false, slowFetchTimeoutMillis); + List quickFetchDelayMillisList = List.of(1L, slowFetchTimeoutMillis / 2); List payloads = List.of("hello".getBytes(), "world".getBytes()); // test quick fetch @@ -108,7 +107,7 @@ public void testQuickFetch() throws ExecutionException, InterruptedException { .map(payload -> stream.append(RawPayloadRecordBatch.of(ByteBuffer.wrap(payload)))).toArray(CompletableFuture[]::new) ).get(); FetchResult fetched = stream.fetch(0, 100, 1000) - .orTimeout(delay + 100, TimeUnit.MILLISECONDS) + .orTimeout(delay + slowFetchTimeoutMillis / 2, TimeUnit.MILLISECONDS) .get(); checkAppendAndFetch(payloads, fetched); stream.destroy(); @@ -118,10 +117,11 @@ public void testQuickFetch() throws ExecutionException, InterruptedException { @Test public void testSlowFetch() throws ExecutionException, InterruptedException { MemoryClientWithDelay memoryClientWithDelay = new MemoryClientWithDelay(); - client = new AlwaysSuccessClient(memoryClientWithDelay); + long slowFetchTimeoutMillis = 1000 * 2; + client = new AlwaysSuccessClient(memoryClientWithDelay, false, slowFetchTimeoutMillis); List payloads = List.of("hello".getBytes(), "world".getBytes()); - long slowFetchDelay = 500 + SLOW_FETCH_TIMEOUT_MILLIS + SLOW_FETCH_TIMEOUT_MILLIS / 2; + long slowFetchDelay = slowFetchTimeoutMillis * 3 / 2; memoryClientWithDelay.setDelayMillis(slowFetchDelay); Stream stream = client .streamClient() @@ -137,7 +137,7 @@ public void testSlowFetch() throws ExecutionException, InterruptedException { AtomicBoolean gotSlowFetchHintException = new AtomicBoolean(false); try { fetched = stream.fetch(0, 100, 1000) - .orTimeout(SLOW_FETCH_TIMEOUT_MILLIS * 2, TimeUnit.MILLISECONDS) + .orTimeout(slowFetchDelay, TimeUnit.MILLISECONDS) .get(); checkAppendAndFetch(payloads, fetched); } catch (ExecutionException e) { @@ -147,7 +147,7 @@ public void testSlowFetch() throws ExecutionException, InterruptedException { SeparateSlowAndQuickFetchHint.reset(); // It should reuse the fetching future above, therefore only (SLOW_FETCH_TIMEOUT_MILLIS / 2) ms is tolerable. fetched = stream.fetch(0, 100, 1000) - .orTimeout(slowFetchDelay + 100, TimeUnit.MILLISECONDS) + .orTimeout(slowFetchTimeoutMillis - 200, TimeUnit.MILLISECONDS) .get(); } checkAppendAndFetch(payloads, fetched); diff --git a/core/src/test/java/kafka/log/s3/DefaultRecordBatch.java b/core/src/test/java/kafka/log/s3/DefaultRecordBatch.java index a480cc0120..ee717747e6 100644 --- a/core/src/test/java/kafka/log/s3/DefaultRecordBatch.java +++ b/core/src/test/java/kafka/log/s3/DefaultRecordBatch.java @@ -17,7 +17,7 @@ package kafka.log.s3; -import com.automq.elasticstream.client.api.RecordBatch; +import kafka.log.es.api.RecordBatch; import java.nio.ByteBuffer; import java.util.Collections; diff --git a/core/src/test/java/kafka/log/s3/DefaultRecordBatchWithContext.java b/core/src/test/java/kafka/log/s3/DefaultRecordBatchWithContext.java index 0f4b87c9ac..a777d2712f 100644 --- a/core/src/test/java/kafka/log/s3/DefaultRecordBatchWithContext.java +++ b/core/src/test/java/kafka/log/s3/DefaultRecordBatchWithContext.java @@ -17,8 +17,8 @@ package kafka.log.s3; -import com.automq.elasticstream.client.api.RecordBatch; -import com.automq.elasticstream.client.api.RecordBatchWithContext; +import kafka.log.es.api.RecordBatch; +import kafka.log.es.api.RecordBatchWithContext; import java.nio.ByteBuffer; import java.util.Map; @@ -31,6 +31,7 @@ public DefaultRecordBatchWithContext(RecordBatch recordBatch, long baseOffset) { this.recordBatch = recordBatch; this.baseOffset = baseOffset; } + @Override public long baseOffset() { return baseOffset; diff --git a/core/src/test/java/kafka/log/s3/S3StorageTest.java b/core/src/test/java/kafka/log/s3/S3StorageTest.java index 42b0e86c45..a5ba043d10 100644 --- a/core/src/test/java/kafka/log/s3/S3StorageTest.java +++ b/core/src/test/java/kafka/log/s3/S3StorageTest.java @@ -36,9 +36,9 @@ import org.mockito.ArgumentCaptor; import java.util.Collections; -import java.util.LinkedList; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -132,7 +132,7 @@ public void testWALCallbackSequencer() { @Test public void testUploadWALObject_sequence() throws ExecutionException, InterruptedException, TimeoutException { - List> objectIdCfList = new LinkedList<>(); + List> objectIdCfList = new CopyOnWriteArrayList<>(); doAnswer(invocation -> { CompletableFuture objectIdCf = new CompletableFuture<>(); objectIdCfList.add(objectIdCf); @@ -152,7 +152,7 @@ public void testUploadWALObject_sequence() throws ExecutionException, Interrupte // sequence get objectId verify(objectManager, timeout(1000).times(1)).prepareObject(anyInt(), anyLong()); - List> commitCfList = new LinkedList<>(); + List> commitCfList = new CopyOnWriteArrayList<>(); doAnswer(invocation -> { CompletableFuture cf = new CompletableFuture<>(); commitCfList.add(cf); diff --git a/core/src/test/java/kafka/log/s3/S3StreamMemoryTest.java b/core/src/test/java/kafka/log/s3/S3StreamMemoryTest.java index 416e28de52..bcd94b5354 100644 --- a/core/src/test/java/kafka/log/s3/S3StreamMemoryTest.java +++ b/core/src/test/java/kafka/log/s3/S3StreamMemoryTest.java @@ -18,31 +18,13 @@ package kafka.log.s3; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.assertNull; -import static org.junit.jupiter.api.Assertions.assertThrows; - -import com.automq.elasticstream.client.api.AppendResult; -import com.automq.elasticstream.client.api.CreateStreamOptions; -import com.automq.elasticstream.client.api.FetchResult; -import com.automq.elasticstream.client.api.OpenStreamOptions; -import com.automq.elasticstream.client.api.RecordBatch; -import com.automq.elasticstream.client.api.RecordBatchWithContext; -import com.automq.elasticstream.client.api.Stream; - -import java.nio.ByteBuffer; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.concurrent.BrokenBarrierException; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.CyclicBarrier; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.atomic.AtomicLong; - +import kafka.log.es.api.AppendResult; +import kafka.log.es.api.CreateStreamOptions; +import kafka.log.es.api.FetchResult; +import kafka.log.es.api.OpenStreamOptions; +import kafka.log.es.api.RecordBatch; +import kafka.log.es.api.RecordBatchWithContext; +import kafka.log.es.api.Stream; import kafka.log.s3.cache.DefaultS3BlockCache; import kafka.log.s3.cache.S3BlockCache; import kafka.log.s3.memory.MemoryMetadataManager; @@ -59,6 +41,23 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.BrokenBarrierException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicLong; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; + @Tag("S3Unit") public class S3StreamMemoryTest { diff --git a/core/src/test/java/kafka/log/s3/S3StreamTest.java b/core/src/test/java/kafka/log/s3/S3StreamTest.java index 01dfdd6017..8388c6a6d8 100644 --- a/core/src/test/java/kafka/log/s3/S3StreamTest.java +++ b/core/src/test/java/kafka/log/s3/S3StreamTest.java @@ -17,8 +17,8 @@ package kafka.log.s3; -import com.automq.elasticstream.client.api.ElasticStreamClientException; -import com.automq.elasticstream.client.api.FetchResult; +import kafka.log.es.api.ElasticStreamClientException; +import kafka.log.es.api.FetchResult; import kafka.log.s3.cache.ReadDataBlock; import kafka.log.s3.model.StreamRecordBatch; import kafka.log.s3.streams.StreamManager; diff --git a/core/src/test/java/kafka/log/s3/StreamMetadataManagerTest.java b/core/src/test/java/kafka/log/s3/StreamMetadataManagerTest.java index 85971dcce3..4e755fcc33 100644 --- a/core/src/test/java/kafka/log/s3/StreamMetadataManagerTest.java +++ b/core/src/test/java/kafka/log/s3/StreamMetadataManagerTest.java @@ -17,17 +17,6 @@ package kafka.log.s3; -import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.mockito.ArgumentMatchers.any; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import kafka.log.s3.metadata.StreamMetadataManager; import kafka.log.s3.metadata.StreamMetadataManager.StreamMetadataListener; import kafka.server.BrokerServer; @@ -58,6 +47,18 @@ import org.junit.jupiter.api.Timeout; import org.mockito.Mockito; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; + @Timeout(40) @Tag("S3Unit") public class StreamMetadataManagerTest { @@ -97,16 +98,16 @@ public void setUp() { static { S3ObjectsImage objectsImage = new S3ObjectsImage(2L, Map.of( - 0L, new S3Object(0L, 128, null, -1, -1, -1, -1, S3ObjectState.COMMITTED), - 1L, new S3Object(1L, 128, null, -1, -1, -1, -1, S3ObjectState.COMMITTED), - 2L, new S3Object(2L, 128, null, -1, -1, -1, -1, S3ObjectState.COMMITTED) + 0L, new S3Object(0L, 128, null, -1, -1, -1, -1, S3ObjectState.COMMITTED), + 1L, new S3Object(1L, 128, null, -1, -1, -1, -1, S3ObjectState.COMMITTED), + 2L, new S3Object(2L, 128, null, -1, -1, -1, -1, S3ObjectState.COMMITTED) )); Map ranges = Map.of( - 0, new RangeMetadata(STREAM0, 0L, 0, 10L, 100L, BROKER0) + 0, new RangeMetadata(STREAM0, 0L, 0, 10L, 100L, BROKER0) ); Map streamObjects = Map.of( - 0L, new S3StreamObject(0L, 128, STREAM0, 10L, 100L)); + 0L, new S3StreamObject(0L, 128, STREAM0, 10L, 100L)); S3StreamMetadataImage streamImage = new S3StreamMetadataImage(STREAM0, 1L, StreamState.OPENED, 0, 10L, ranges, streamObjects); BrokerS3WALMetadataImage walMetadataImage0 = new BrokerS3WALMetadataImage(BROKER0, new SortedWALObjectsList(List.of( @@ -125,7 +126,7 @@ public void setUp() { streamObjects.put(1L, new S3StreamObject(1L, 128, STREAM0, 100L, 150L)); streamImage = new S3StreamMetadataImage(STREAM0, 2L, StreamState.OPENED, 1, 10L, ranges, streamObjects); streamsImage = new S3StreamsMetadataImage(STREAM0, Map.of(STREAM0, streamImage), - Map.of(BROKER0, BrokerS3WALMetadataImage.EMPTY)); + Map.of(BROKER0, BrokerS3WALMetadataImage.EMPTY)); image1 = new MetadataImage(new MetadataProvenance(1, 1, 1), null, null, null, null, null, null, null, streamsImage, objectsImage, null); ranges = new HashMap<>(ranges); @@ -134,7 +135,7 @@ public void setUp() { streamObjects.put(2L, new S3StreamObject(2L, 128, STREAM0, 150L, 200L)); streamImage = new S3StreamMetadataImage(STREAM0, 3L, StreamState.OPENED, 2, 10L, ranges, streamObjects); streamsImage = new S3StreamsMetadataImage(STREAM0, Map.of(STREAM0, streamImage), - Map.of(BROKER0, BrokerS3WALMetadataImage.EMPTY)); + Map.of(BROKER0, BrokerS3WALMetadataImage.EMPTY)); image2 = new MetadataImage(new MetadataProvenance(2, 2, 2), null, null, null, null, null, null, null, streamsImage, objectsImage, null); } diff --git a/core/src/test/java/kafka/log/s3/StreamObjectCopyerTest.java b/core/src/test/java/kafka/log/s3/StreamObjectCopyerTest.java index 81960541da..12f4317dae 100644 --- a/core/src/test/java/kafka/log/s3/StreamObjectCopyerTest.java +++ b/core/src/test/java/kafka/log/s3/StreamObjectCopyerTest.java @@ -17,10 +17,6 @@ package kafka.log.s3; -import java.util.Iterator; -import java.util.List; -import java.util.concurrent.ExecutionException; - import kafka.log.s3.model.StreamRecordBatch; import kafka.log.s3.operator.MemoryS3Operator; import kafka.log.s3.operator.S3Operator; @@ -30,6 +26,10 @@ 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 static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; diff --git a/core/src/test/scala/unit/kafka/log/es/ElasticLogTest.scala b/core/src/test/scala/unit/kafka/log/es/ElasticLogTest.scala index 7fc9a08a3f..821a245beb 100644 --- a/core/src/test/scala/unit/kafka/log/es/ElasticLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/es/ElasticLogTest.scala @@ -232,14 +232,6 @@ class ElasticLogTest { log.close() } - @Test - def testLogCloseFailureWhenInMemoryBufferClosed(): Unit = { - val keyValues = Seq(KeyValue("abc", "ABC"), KeyValue("de", "DE")) - appendRecords(kvsToRecords(keyValues)) - log.closeHandlers() - assertThrows(classOf[KafkaStorageException], () => log.close()) - } - @Test def testLogCloseHandlers(): Unit = { val keyValues = Seq(KeyValue("abc", "ABC"), KeyValue("de", "DE")) diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 318a50dab9..cb7da00c09 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -129,6 +129,7 @@ versions += [ zstd: "1.5.2-1", elasticstream: "1.0-SNAPSHOT", s3Client: "2.20.127", + commonLang: "3.12.0", ] libs += [ activation: "javax.activation:activation:$versions.activation", @@ -220,6 +221,7 @@ libs += [ mavenArtifact: "org.apache.maven:maven-artifact:$versions.mavenArtifact", zstd: "com.github.luben:zstd-jni:$versions.zstd", httpclient: "org.apache.httpcomponents:httpclient:$versions.httpclient", - esClient: "com.automq.elasticstream:client:$versions.elasticstream", s3Client: "software.amazon.awssdk:s3:$versions.s3Client", + commonLang: "org.apache.commons:commons-lang3:$versions.commonLang", + nettyAll: "io.netty:netty-all:$versions.netty", ] diff --git a/metadata/src/main/java/org/apache/kafka/controller/stream/S3ObjectControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/stream/S3ObjectControlManager.java index 0ba64bb1cf..290898a9ad 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/stream/S3ObjectControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/stream/S3ObjectControlManager.java @@ -35,6 +35,7 @@ import org.apache.kafka.common.metadata.AssignedS3ObjectIdRecord; import org.apache.kafka.common.metadata.RemoveS3ObjectRecord; import org.apache.kafka.common.metadata.S3ObjectRecord; +import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.controller.ControllerRequestContext; import org.apache.kafka.controller.ControllerResult; @@ -154,11 +155,20 @@ public ControllerResult prepareObject(PrepareS3Obje return ControllerResult.atomicOf(records, response); } - public ControllerResult commitObject(long objectId, long objectSize) { + public ControllerResult commitObject(long objectId, long objectSize) { S3Object object = this.objectsMetadata.get(objectId); if (object == null) { log.error("object {} not exist when commit wal object", objectId); - return ControllerResult.of(Collections.emptyList(), false); + return ControllerResult.of(Collections.emptyList(), Errors.OBJECT_NOT_EXIST); + } + // verify the state + if (object.getS3ObjectState() == S3ObjectState.COMMITTED) { + log.warn("object {} already committed", objectId); + return ControllerResult.of(Collections.emptyList(), Errors.REDUNDANT_OPERATION); + } + if (object.getS3ObjectState() != S3ObjectState.PREPARED) { + log.error("object {} is not prepared but try to commit", objectId); + return ControllerResult.of(Collections.emptyList(), Errors.OBJECT_NOT_EXIST); } S3ObjectRecord record = new S3ObjectRecord() .setObjectId(objectId) @@ -168,14 +178,14 @@ public ControllerResult commitObject(long objectId, long objectSize) { .setExpiredTimeInMs(object.getExpiredTimeInMs()) .setCommittedTimeInMs(System.currentTimeMillis()); return ControllerResult.of(List.of( - new ApiMessageAndVersion(record, (short) 0)), true); + new ApiMessageAndVersion(record, (short) 0)), Errors.NONE); } public ControllerResult markDestroyObjects(List objects) { List records = new ArrayList<>(); for (Long objectId : objects) { S3Object object = this.objectsMetadata.get(objectId); - if (object == null) { + if (object == null || object.getS3ObjectState() == S3ObjectState.MARK_DESTROYED) { log.error("object {} not exist when mark destroy object", objectId); // TODO: Maybe we can ignore this situation, because this object is already destroyed ? return ControllerResult.of(Collections.emptyList(), false); diff --git a/metadata/src/main/java/org/apache/kafka/controller/stream/StreamControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/stream/StreamControlManager.java index 2b965a02f2..a7f44ef9f8 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/stream/StreamControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/stream/StreamControlManager.java @@ -362,19 +362,25 @@ public ControllerResult commitWALObject(CommitWALOb long objectSize = data.objectSize(); long orderId = data.orderId(); List streamRanges = data.objectStreamRanges(); + List compactedObjectIds = data.compactedObjectIds(); // commit object - ControllerResult commitResult = this.s3ObjectControlManager.commitObject(objectId, objectSize); - if (!commitResult.response()) { - log.error("object {} not exist when commit wal object", objectId); + ControllerResult commitResult = this.s3ObjectControlManager.commitObject(objectId, objectSize); + if (commitResult.response() == Errors.OBJECT_NOT_EXIST) { + log.error("[CommitWALObject]: object {} not exist when commit wal object", objectId); resp.setErrorCode(Errors.OBJECT_NOT_EXIST.code()); return ControllerResult.of(Collections.emptyList(), resp); } + if (commitResult.response() == Errors.REDUNDANT_OPERATION) { + // regard it as redundant commit operation, just return success + log.warn("[CommitWALObject]: object {} already committed", objectId); + return ControllerResult.of(Collections.emptyList(), resp); + } records.addAll(commitResult.records()); // mark destroy compacted object - if (data.compactedObjectIds() != null && !data.compactedObjectIds().isEmpty()) { - ControllerResult destroyResult = this.s3ObjectControlManager.markDestroyObjects(data.compactedObjectIds()); + if (compactedObjectIds != null && !compactedObjectIds.isEmpty()) { + ControllerResult destroyResult = this.s3ObjectControlManager.markDestroyObjects(compactedObjectIds); if (!destroyResult.response()) { - log.error("Mark destroy compacted objects {} failed", String.join(",", data.compactedObjectIds().toArray(new String[0]))); + log.error("[CommitWALObject]: Mark destroy compacted objects: {} failed", compactedObjectIds); resp.setErrorCode(Errors.STREAM_INNER_ERROR.code()); return ControllerResult.of(Collections.emptyList(), resp); } @@ -408,15 +414,64 @@ public ControllerResult commitWALObject(CommitWALOb long endOffset = obj.endOffset(); records.add(new S3StreamObject(obj.objectId(), obj.objectSize(), streamId, startOffset, endOffset).toRecord()); }); - // generate compacted objects' remove record - data.compactedObjectIds().forEach(id -> records.add(new ApiMessageAndVersion(new RemoveWALObjectRecord() - .setObjectId(id), (short) 0))); - log.info("[CommitWALObject]: broker: {} commit wal object {} success", brokerId, objectId); + if (compactedObjectIds != null && !compactedObjectIds.isEmpty()) { + // generate compacted objects' remove record + compactedObjectIds.forEach(id -> records.add(new ApiMessageAndVersion(new RemoveWALObjectRecord() + .setObjectId(id), (short) 0))); + } + log.info("[CommitWALObject]: broker: {} commit wal object: {} success, compacted objects: {}", brokerId, objectId, compactedObjectIds); return ControllerResult.atomicOf(records, resp); } public ControllerResult commitStreamObject(CommitStreamObjectRequestData data) { - throw new UnsupportedOperationException(); + long streamObjectId = data.objectId(); + long streamId = data.streamId(); + long startOffset = data.startOffset(); + long endOffset = data.endOffset(); + long objectSize = data.objectSize(); + List sourceObjectIds = data.sourceObjectIds(); + List records = new ArrayList<>(); + CommitStreamObjectResponseData resp = new CommitStreamObjectResponseData(); + + // commit object + ControllerResult commitResult = this.s3ObjectControlManager.commitObject(streamObjectId, objectSize); + if (commitResult.response() == Errors.OBJECT_NOT_EXIST) { + log.error("[CommitStreamObject]: object {} not exist when commit stream object", streamObjectId); + resp.setErrorCode(Errors.OBJECT_NOT_EXIST.code()); + return ControllerResult.of(Collections.emptyList(), resp); + } + if (commitResult.response() == Errors.REDUNDANT_OPERATION) { + // regard it as redundant commit operation, just return success + log.warn("[CommitStreamObject]: object {} already committed", streamObjectId); + return ControllerResult.of(Collections.emptyList(), resp); + } + + // mark destroy compacted object + if (sourceObjectIds != null && !sourceObjectIds.isEmpty()) { + ControllerResult destroyResult = this.s3ObjectControlManager.markDestroyObjects(sourceObjectIds); + if (!destroyResult.response()) { + log.error("[CommitStreamObject]: Mark destroy compacted objects: {} failed", sourceObjectIds); + resp.setErrorCode(Errors.STREAM_INNER_ERROR.code()); + return ControllerResult.of(Collections.emptyList(), resp); + } + } + + // generate stream object record + records.add(new ApiMessageAndVersion(new S3StreamObjectRecord() + .setObjectId(streamObjectId) + .setStreamId(streamId) + .setObjectSize(objectSize) + .setStartOffset(startOffset) + .setEndOffset(endOffset), (short) 0)); + + // generate compacted objects' remove record + if (sourceObjectIds != null && !sourceObjectIds.isEmpty()) { + sourceObjectIds.forEach(id -> records.add(new ApiMessageAndVersion(new RemoveS3StreamObjectRecord() + .setObjectId(id) + .setStreamId(streamId), (short) 0))); + } + log.info("[CommitStreamObject]: stream object: {} commit success, compacted objects: {}", streamObjectId, sourceObjectIds); + return ControllerResult.atomicOf(records, resp); } public GetStreamsOffsetResponseData getStreamsOffset(GetStreamsOffsetRequestData data) { @@ -541,6 +596,7 @@ public void replay(RemoveWALObjectRecord record) { } walMetadata.walObjects.remove(objectId); } + public void replay(S3StreamObjectRecord record) { long objectId = record.objectId(); long streamId = record.streamId(); diff --git a/metadata/src/main/java/org/apache/kafka/image/S3StreamsMetadataImage.java b/metadata/src/main/java/org/apache/kafka/image/S3StreamsMetadataImage.java index deea17ea55..6516d7870b 100644 --- a/metadata/src/main/java/org/apache/kafka/image/S3StreamsMetadataImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/S3StreamsMetadataImage.java @@ -99,6 +99,34 @@ public InRangeObjects getObjects(long streamId, long startOffset, long endOffset return new InRangeObjects(streamId, startOffset, realEndOffset, objects); } + /** + * Get stream objects in range [startOffset, endOffset) with limit. + * It will throw IllegalArgumentException if limit or streamId is invalid. + * @param streamId stream id + * @param startOffset inclusive start offset of the stream + * @param endOffset exclusive end offset of the stream + * @param limit max number of stream objects to return + * @return stream objects + */ + public List getStreamObjects(long streamId, long startOffset, long endOffset, int limit) { + if (limit <= 0) { + throw new IllegalArgumentException("limit must be positive"); + } + S3StreamMetadataImage stream = streamsMetadata.get(streamId); + if (stream == null) { + throw new IllegalArgumentException("stream not found"); + } + Map streamObjectsMetadata = stream.getStreamObjects(); + if (streamObjectsMetadata == null || streamObjectsMetadata.isEmpty()) { + return Collections.emptyList(); + } + return streamObjectsMetadata.values().stream().filter(obj -> { + long objectStartOffset = obj.streamOffsetRange().getStartOffset(); + long objectEndOffset = obj.streamOffsetRange().getEndOffset(); + return objectStartOffset < endOffset && objectEndOffset > startOffset; + }).sorted(Comparator.comparing(S3StreamObject::streamOffsetRange)).limit(limit).collect(Collectors.toCollection(ArrayList::new)); + } + public List getWALObjects(int brokerId) { BrokerS3WALMetadataImage wal = brokerWALMetadata.get(brokerId); if (wal == null) { @@ -170,17 +198,12 @@ private Queue rangeOfStreamObjects() { // TODO: refactor to make stream objects in order if (streamObjectsMetadata != null && !streamObjectsMetadata.isEmpty()) { return streamObjectsMetadata.values().stream().filter(obj -> { - long objectStartOffset = obj.streamIndex().getStartOffset(); - long objectEndOffset = obj.streamIndex().getEndOffset(); + long objectStartOffset = obj.streamOffsetRange().getStartOffset(); + long objectEndOffset = obj.streamOffsetRange().getEndOffset(); return objectStartOffset < endOffset && objectEndOffset > startOffset; - }).sorted(new Comparator() { - @Override - public int compare(S3StreamObject o1, S3StreamObject o2) { - return o1.objectId() > o2.objectId() ? 1 : -1; - } - }).map(obj -> { - long startOffset = obj.streamIndex().getStartOffset(); - long endOffset = obj.streamIndex().getEndOffset(); + }).sorted(Comparator.comparingLong(S3StreamObject::objectId)).map(obj -> { + long startOffset = obj.streamOffsetRange().getStartOffset(); + long endOffset = obj.streamOffsetRange().getEndOffset(); return new ObjectStreamRange(obj.objectId(), obj.objectType(), startOffset, endOffset); }).collect(Collectors.toCollection(LinkedList::new)); } diff --git a/metadata/src/main/java/org/apache/kafka/metadata/stream/S3StreamObject.java b/metadata/src/main/java/org/apache/kafka/metadata/stream/S3StreamObject.java index 87b0e71cdc..944ed903ae 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/stream/S3StreamObject.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/stream/S3StreamObject.java @@ -27,22 +27,26 @@ public class S3StreamObject { private final long objectSize; - private final StreamOffsetRange streamIndex; + private final StreamOffsetRange streamOffsetRange; public S3StreamObject(long objectId, long objectSize, long streamId, long startOffset, long endOffset) { this.objectId = objectId; this.objectSize = objectSize; - this.streamIndex = new StreamOffsetRange(streamId, startOffset, endOffset); + this.streamOffsetRange = new StreamOffsetRange(streamId, startOffset, endOffset); } - public StreamOffsetRange streamIndex() { - return streamIndex; + public StreamOffsetRange streamOffsetRange() { + return streamOffsetRange; } public long objectId() { return objectId; } + public long objectSize() { + return objectSize; + } + public S3ObjectType objectType() { return S3ObjectType.STREAM; } @@ -50,9 +54,9 @@ public S3ObjectType objectType() { public ApiMessageAndVersion toRecord() { return new ApiMessageAndVersion(new S3StreamObjectRecord() .setObjectId(objectId) - .setStreamId(streamIndex.getStreamId()) - .setStartOffset(streamIndex.getStartOffset()) - .setEndOffset(streamIndex.getEndOffset()), (short) 0); + .setStreamId(streamOffsetRange.getStreamId()) + .setStartOffset(streamOffsetRange.getStartOffset()) + .setEndOffset(streamOffsetRange.getEndOffset()), (short) 0); } public static S3StreamObject of(S3StreamObjectRecord record) { diff --git a/metadata/src/main/java/org/apache/kafka/metadata/stream/S3StreamObjectMetadata.java b/metadata/src/main/java/org/apache/kafka/metadata/stream/S3StreamObjectMetadata.java new file mode 100644 index 0000000000..81e9228937 --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/metadata/stream/S3StreamObjectMetadata.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.metadata.stream; + +public class S3StreamObjectMetadata implements Comparable { + private final S3StreamObject s3StreamObject; + private final long timestamp; + + public S3StreamObjectMetadata(S3StreamObject s3StreamObject, long timestamp) { + this.s3StreamObject = s3StreamObject; + this.timestamp = timestamp; + } + + public long startOffset() { + return s3StreamObject.streamOffsetRange().getStartOffset(); + } + + public long endOffset() { + return s3StreamObject.streamOffsetRange().getEndOffset(); + } + + public long streamId() { + return s3StreamObject.streamOffsetRange().getStreamId(); + } + + public long objectId() { + return s3StreamObject.objectId(); + } + + public long timestamp() { + return timestamp; + } + + @Override + public int compareTo(S3StreamObjectMetadata o) { + return s3StreamObject.streamOffsetRange().compareTo(o.s3StreamObject.streamOffsetRange()); + } +} diff --git a/metadata/src/main/java/org/apache/kafka/metadata/stream/StreamOffsetRange.java b/metadata/src/main/java/org/apache/kafka/metadata/stream/StreamOffsetRange.java index e405a9d732..581659a79b 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/stream/StreamOffsetRange.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/stream/StreamOffsetRange.java @@ -54,7 +54,10 @@ public long getEndOffset() { @Override public int compareTo(StreamOffsetRange o) { int res = Long.compare(this.streamId, o.streamId); - return res == 0 ? Long.compare(this.startOffset, o.startOffset) : res; + if (res != 0) + return res; + res = Long.compare(this.startOffset, o.startOffset); + return res == 0 ? Long.compare(this.endOffset, o.endOffset) : res; } public StreamIndex toRecordStreamIndex() { diff --git a/metadata/src/test/java/org/apache/kafka/controller/S3ObjectControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/S3ObjectControlManagerTest.java index 062153e0a8..b191ae3e61 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/S3ObjectControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/S3ObjectControlManagerTest.java @@ -18,7 +18,6 @@ package org.apache.kafka.controller; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; @@ -158,18 +157,23 @@ public void testCommitObject() { prepareOneObject(60 * 1000); // 2. commit an object which not exist in controller - ControllerResult result1 = manager.commitObject(1, 1024); - assertFalse(result1.response()); + ControllerResult result1 = manager.commitObject(1, 1024); + assertEquals(Errors.OBJECT_NOT_EXIST, result1.response()); assertEquals(0, result1.records().size()); // 3. commit an valid object - ControllerResult result2 = manager.commitObject(0, 1024); - assertTrue(result2.response()); + ControllerResult result2 = manager.commitObject(0, 1024); + assertEquals(Errors.NONE, result2.response()); assertEquals(1, result2.records().size()); S3ObjectRecord record = (S3ObjectRecord) result2.records().get(0).message(); manager.replay(record); - // 4. verify the object is committed + // 4. commit again + ControllerResult result3 = manager.commitObject(0, 1024); + assertEquals(Errors.REDUNDANT_OPERATION, result3.response()); + assertEquals(0, result3.records().size()); + + // 5. verify the object is committed assertEquals(1, manager.objectsMetadata().size()); S3Object object = manager.objectsMetadata().get(0L); assertEquals(S3ObjectState.COMMITTED, object.getS3ObjectState()); diff --git a/metadata/src/test/java/org/apache/kafka/controller/StreamControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/StreamControlManagerTest.java index 7bf57e3f3d..e8cafaeedf 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/StreamControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/StreamControlManagerTest.java @@ -29,6 +29,8 @@ import java.util.stream.Collectors; import org.apache.kafka.common.message.CloseStreamRequestData; import org.apache.kafka.common.message.CloseStreamResponseData; +import org.apache.kafka.common.message.CommitStreamObjectRequestData; +import org.apache.kafka.common.message.CommitStreamObjectResponseData; import org.apache.kafka.common.message.CommitWALObjectRequestData; import org.apache.kafka.common.message.CommitWALObjectRequestData.ObjectStreamRange; import org.apache.kafka.common.message.CommitWALObjectRequestData.StreamObject; @@ -271,9 +273,9 @@ public void testCommitWalBasic() { Mockito.when(objectControlManager.commitObject(anyLong(), anyLong())).then(ink -> { long objectId = ink.getArgument(0); if (objectId == 1) { - return ControllerResult.of(Collections.emptyList(), false); + return ControllerResult.of(Collections.emptyList(), Errors.OBJECT_NOT_EXIST); } - return ControllerResult.of(Collections.emptyList(), true); + return ControllerResult.of(Collections.emptyList(), Errors.NONE); }); // 1. create and open stream_0 CreateStreamRequestData request0 = new CreateStreamRequestData(); @@ -374,7 +376,7 @@ private void createAndOpenStream(int brokerId, long epoch) { @Test public void testCommitWalCompacted() { - Mockito.when(objectControlManager.commitObject(anyLong(), anyLong())).thenReturn(ControllerResult.of(Collections.emptyList(), true)); + Mockito.when(objectControlManager.commitObject(anyLong(), anyLong())).thenReturn(ControllerResult.of(Collections.emptyList(), Errors.NONE)); Mockito.when(objectControlManager.markDestroyObjects(anyList())).thenReturn(ControllerResult.of(Collections.emptyList(), true)); // 1. create and open stream_0 and stream_1 @@ -447,7 +449,8 @@ public void testCommitWalCompacted() { assertEquals(0L, streamsOffset.streamsOffset().get(1).startOffset()); assertEquals(300L, streamsOffset.streamsOffset().get(1).endOffset()); - // 6. commit a second level wal object which compact wal_0 and wal_1 + // 6. commit an invalid wal object which contains the destroyed or not exist wal object + Mockito.when(objectControlManager.markDestroyObjects(anyList())).thenReturn(ControllerResult.of(Collections.emptyList(), false)); List streamRanges2 = List.of( new ObjectStreamRange() .setStreamId(STREAM0) @@ -465,12 +468,25 @@ public void testCommitWalCompacted() { .setBrokerId(BROKER0) .setObjectSize(999) .setObjectStreamRanges(streamRanges2) - .setCompactedObjectIds(List.of(0L, 1L)); + .setCompactedObjectIds(List.of(0L, 1L, 10L)); ControllerResult result6 = manager.commitWALObject(commitRequest2); + assertEquals(Errors.STREAM_INNER_ERROR.code(), result6.response().errorCode()); + assertEquals(0, result6.records().size()); + Mockito.when(objectControlManager.markDestroyObjects(anyList())).thenReturn(ControllerResult.of(Collections.emptyList(), true)); + + // 7. commit a second level wal object which compact wal_0 and wal_1 + commitRequest2 = new CommitWALObjectRequestData() + .setObjectId(2L) + .setOrderId(0L) + .setBrokerId(BROKER0) + .setObjectSize(999) + .setObjectStreamRanges(streamRanges2) + .setCompactedObjectIds(List.of(0L, 1L)); + result6 = manager.commitWALObject(commitRequest2); assertEquals(Errors.NONE.code(), result6.response().errorCode()); replay(manager, result6.records()); - // 7. fetch range end offset + // 8. fetch range end offset streamsOffset = manager.getStreamsOffset(request); assertEquals(2, streamsOffset.streamsOffset().size()); assertEquals(STREAM0, streamsOffset.streamsOffset().get(0).streamId()); @@ -480,15 +496,16 @@ public void testCommitWalCompacted() { assertEquals(0L, streamsOffset.streamsOffset().get(1).startOffset()); assertEquals(300L, streamsOffset.streamsOffset().get(1).endOffset()); - // 8. verify compacted wal objects is removed + // 9. verify compacted wal objects is removed assertEquals(1, manager.brokersMetadata().get(BROKER0).walObjects().size()); assertEquals(2, manager.brokersMetadata().get(BROKER0).walObjects().get(2L).objectId()); assertEquals(0, manager.brokersMetadata().get(BROKER0).walObjects().get(2L).orderId()); + } @Test public void testCommitWalWithStreamObject() { - Mockito.when(objectControlManager.commitObject(anyLong(), anyLong())).thenReturn(ControllerResult.of(Collections.emptyList(), true)); + Mockito.when(objectControlManager.commitObject(anyLong(), anyLong())).thenReturn(ControllerResult.of(Collections.emptyList(), Errors.NONE)); Mockito.when(objectControlManager.markDestroyObjects(anyList())).thenReturn(ControllerResult.of(Collections.emptyList(), true)); // 1. create and open stream_0 and stream_1 @@ -536,6 +553,109 @@ public void testCommitWalWithStreamObject() { assertEquals(1, manager.streamsMetadata().get(STREAM1).streamObjects().size()); } + @Test + public void testCommitStreamObject() { + Mockito.when(objectControlManager.commitObject(anyLong(), anyLong())).thenReturn(ControllerResult.of(Collections.emptyList(), Errors.NONE)); + Mockito.when(objectControlManager.markDestroyObjects(anyList())).thenReturn(ControllerResult.of(Collections.emptyList(), true)); + + // 1. create and open stream_0 and stream_1 + createAndOpenStream(BROKER0, EPOCH0); + createAndOpenStream(BROKER0, EPOCH0); + + // 2. commit a wal with stream_0 and a stream object with stream_1 that is split out from wal + List streamRanges0 = List.of( + new ObjectStreamRange() + .setStreamId(STREAM0) + .setStreamEpoch(EPOCH0) + .setStartOffset(0L) + .setEndOffset(100L)); + CommitWALObjectRequestData commitRequest0 = new CommitWALObjectRequestData() + .setObjectId(0L) + .setOrderId(0L) + .setBrokerId(BROKER0) + .setObjectSize(999) + .setObjectStreamRanges(streamRanges0) + .setStreamObjects(List.of( + new StreamObject() + .setStreamId(STREAM1) + .setObjectId(1L) + .setObjectSize(999) + .setStartOffset(0L) + .setEndOffset(200L) + )); + ControllerResult result0 = manager.commitWALObject(commitRequest0); + assertEquals(Errors.NONE.code(), result0.response().errorCode()); + replay(manager, result0.records()); + + // 3. commit a wal with stream_0 and a stream object with stream_1 that is split out from wal + List streamRanges1 = List.of( + new ObjectStreamRange() + .setStreamId(STREAM0) + .setStreamEpoch(EPOCH0) + .setStartOffset(100L) + .setEndOffset(200L)); + CommitWALObjectRequestData commitRequest1 = new CommitWALObjectRequestData() + .setObjectId(2L) + .setOrderId(1L) + .setBrokerId(BROKER0) + .setObjectSize(999) + .setObjectStreamRanges(streamRanges1) + .setStreamObjects(List.of( + new StreamObject() + .setStreamId(STREAM1) + .setObjectId(3L) + .setObjectSize(999) + .setStartOffset(200L) + .setEndOffset(400L) + )); + ControllerResult result1 = manager.commitWALObject(commitRequest1); + assertEquals(Errors.NONE.code(), result1.response().errorCode()); + replay(manager, result1.records()); + + // 4. compact these two stream objects + CommitStreamObjectRequestData streamObjectRequest = new CommitStreamObjectRequestData() + .setObjectId(4L) + .setStreamId(STREAM1) + .setStartOffset(0L) + .setEndOffset(400L) + .setObjectSize(999) + .setSourceObjectIds(List.of(1L, 3L)); + ControllerResult result2 = manager.commitStreamObject(streamObjectRequest); + assertEquals(Errors.NONE.code(), result2.response().errorCode()); + replay(manager, result2.records()); + + // 5. fetch stream offset range + GetStreamsOffsetRequestData request = new GetStreamsOffsetRequestData() + .setStreamIds(List.of(STREAM0, STREAM1)); + GetStreamsOffsetResponseData streamsOffset = manager.getStreamsOffset(request); + assertEquals(2, streamsOffset.streamsOffset().size()); + assertEquals(STREAM0, streamsOffset.streamsOffset().get(0).streamId()); + assertEquals(0L, streamsOffset.streamsOffset().get(0).startOffset()); + assertEquals(200L, streamsOffset.streamsOffset().get(0).endOffset()); + assertEquals(STREAM1, streamsOffset.streamsOffset().get(1).streamId()); + assertEquals(0L, streamsOffset.streamsOffset().get(1).startOffset()); + assertEquals(400L, streamsOffset.streamsOffset().get(1).endOffset()); + + // 6. compact a stream object from invalid source object + Mockito.when(objectControlManager.markDestroyObjects(anyList())).thenReturn(ControllerResult.of(Collections.emptyList(), false)); + streamObjectRequest = new CommitStreamObjectRequestData() + .setObjectId(5L) + .setStreamId(STREAM1) + .setStartOffset(400L) + .setEndOffset(1000L) + .setObjectSize(999) + .setSourceObjectIds(List.of(10L)); + result2 = manager.commitStreamObject(streamObjectRequest); + assertEquals(Errors.STREAM_INNER_ERROR.code(), result2.response().errorCode()); + replay(manager, result2.records()); + + // 6. verify stream objects + assertEquals(1, manager.streamsMetadata().get(STREAM1).streamObjects().size()); + assertEquals(4L, manager.streamsMetadata().get(STREAM1).streamObjects().get(4L).objectId()); + assertEquals(0L, manager.streamsMetadata().get(STREAM1).streamObjects().get(4L).streamOffsetRange().getStartOffset()); + assertEquals(400L, manager.streamsMetadata().get(STREAM1).streamObjects().get(4L).streamOffsetRange().getEndOffset()); + } + private void commitFirstLevelWalObject(long objectId, long orderId, long streamId, long startOffset, long endOffset, long epoch, int brokerId) { List streamRanges0 = List.of(new ObjectStreamRange() .setStreamId(streamId) @@ -586,7 +706,7 @@ private void replay(StreamControlManager manager, List rec case S3_STREAM_OBJECT_RECORD: manager.replay((S3StreamObjectRecord) message); break; - case REMOVE_S3_OBJECT_RECORD: + case REMOVE_S3_STREAM_OBJECT_RECORD: manager.replay((RemoveS3StreamObjectRecord) message); break; default: From 68c732e35528da09209fd86fa7ec822c3e5131ad Mon Sep 17 00:00:00 2001 From: Shichao Nie Date: Fri, 8 Sep 2023 17:30:53 +0800 Subject: [PATCH 4/7] feat: support configurable compaction interval Signed-off-by: Shichao Nie --- .../log/s3/compact/CompactionManager.java | 4 +- .../main/scala/kafka/server/KafkaConfig.scala | 41 +++++++++++-------- .../s3/compact/CompactionAnalyzerTest.java | 6 +-- .../log/s3/compact/CompactionManagerTest.java | 10 ----- .../log/s3/compact/CompactionTestBase.java | 5 +-- 5 files changed, 31 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/kafka/log/s3/compact/CompactionManager.java b/core/src/main/scala/kafka/log/s3/compact/CompactionManager.java index 92af43ee44..a9a2050355 100644 --- a/core/src/main/scala/kafka/log/s3/compact/CompactionManager.java +++ b/core/src/main/scala/kafka/log/s3/compact/CompactionManager.java @@ -53,12 +53,14 @@ public class CompactionManager { private final long compactionCacheSize; private final double executionScoreThreshold; private final long streamSplitSize; + private final int compactionInterval; private final TokenBucketThrottle networkInThrottle; public CompactionManager(KafkaConfig config, ObjectManager objectManager, StreamMetadataManager streamMetadataManager, S3Operator s3Operator) { this.objectManager = objectManager; this.streamMetadataManager = streamMetadataManager; this.s3Operator = s3Operator; + this.compactionInterval = config.s3ObjectCompactionInterval(); this.compactionCacheSize = config.s3ObjectCompactionCacheSize(); this.executionScoreThreshold = config.s3ObjectCompactionExecutionScoreThreshold(); this.streamSplitSize = config.s3ObjectCompactionStreamSplitSize(); @@ -69,7 +71,7 @@ public CompactionManager(KafkaConfig config, ObjectManager objectManager, Stream } public void start() { - this.executorService.scheduleWithFixedDelay(this::compact, 0, 600, TimeUnit.SECONDS); + this.executorService.scheduleWithFixedDelay(this::compact, 0, this.compactionInterval, TimeUnit.MINUTES); } public void shutdown() { diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 62160b9726..a4daa95062 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -311,6 +311,7 @@ object Defaults { val QuorumRetryBackoffMs = RaftConfig.DEFAULT_QUORUM_RETRY_BACKOFF_MS /** ********* Kafka on S3 Configuration *********/ + val S3ObjectCompactionInterval: Int = 30 // 30min val S3ObjectCompactionCacheSize: Long = 2 * 1024 * 1024 * 1024 // 2GB val S3ObjectCompactionNWInBandwidth: Long = 50 * 1024 * 1024 // 50MB/s val S3ObjectCompactionNWOutBandwidth: Long = 50 * 1024 * 1024 // 50MB/s @@ -694,12 +695,13 @@ object KafkaConfig { val S3ObjectBlockSizeProp = "s3.object.block.size" val S3ObjectPartSizeProp = "s3.object.part.size" val S3CacheSizeProp = "s3.cache.size" - val S3ObjectCompactionCacheSize = "s3.object.compaction.cache.size" - val S3ObjectCompactionNWInBandwidth = "s3.object.compaction.network.in.bandwidth" - val S3ObjectCompactionNWOutBandwidth = "s3.object.compaction.network.out.bandwidth" - val S3ObjectCompactionUploadConcurrency = "s3.object.compaction.upload.concurrency" - val S3ObjectCompactionExecutionScoreThreshold = "s3.object.compaction.execution.score.threshold" - val S3ObjectCompactionStreamSplitSize = "s3.object.compaction.stream.split.size" + val S3ObjectCompactionIntervalProp = "s3.object.compaction.interval.minutes" + val S3ObjectCompactionCacheSizeProp = "s3.object.compaction.cache.size" + val S3ObjectCompactionNWInBandwidthProp = "s3.object.compaction.network.in.bandwidth" + val S3ObjectCompactionNWOutBandwidthProp = "s3.object.compaction.network.out.bandwidth" + val S3ObjectCompactionUploadConcurrencyProp = "s3.object.compaction.upload.concurrency" + val S3ObjectCompactionExecutionScoreThresholdProp = "s3.object.compaction.execution.score.threshold" + val S3ObjectCompactionStreamSplitSizeProp = "s3.object.compaction.stream.split.size" val S3EndpointDoc = "The S3 endpoint, ex. https://s3.{region}.amazonaws.com." val S3RegionDoc = "The S3 region, ex. us-east-1." @@ -712,6 +714,7 @@ object KafkaConfig { val S3ObjectBlockSizeDoc = "The S3 object compressed block size threshold." val S3ObjectPartSizeDoc = "The S3 object multi-part upload part size threshold." val S3CacheSizeDoc = "The S3 block cache size in MiB." + val S3ObjectCompactionIntervalDoc = "The execution interval of S3 object compaction in minutes." val S3ObjectCompactionCacheSizeDoc = "The S3 object compaction cache size in Bytes." val S3ObjectCompactionNWInBandwidthDoc = "The S3 object compaction network in bandwidth in Bytes/s." val S3ObjectCompactionNWOutBandwidthDoc = "The S3 object compaction network out bandwidth in Bytes/s." @@ -1528,12 +1531,13 @@ object KafkaConfig { .define(S3ObjectBlockSizeProp, INT, 8388608, MEDIUM, S3ObjectBlockSizeDoc) .define(S3ObjectPartSizeProp, INT, 16777216, MEDIUM, S3ObjectPartSizeDoc) .define(S3CacheSizeProp, LONG, 1073741824L, MEDIUM, S3CacheSizeDoc) - .define(S3ObjectCompactionCacheSize, LONG, Defaults.S3ObjectCompactionCacheSize, MEDIUM, S3ObjectCompactionCacheSizeDoc) - .define(S3ObjectCompactionNWInBandwidth, LONG, Defaults.S3ObjectCompactionNWInBandwidth, MEDIUM, S3ObjectCompactionNWInBandwidthDoc) - .define(S3ObjectCompactionNWOutBandwidth, LONG, Defaults.S3ObjectCompactionNWOutBandwidth, MEDIUM, S3ObjectCompactionNWOutBandwidthDoc) - .define(S3ObjectCompactionUploadConcurrency, INT, Defaults.S3ObjectCompactionUploadConcurrency, MEDIUM, S3ObjectCompactionUploadConcurrencyDoc) - .define(S3ObjectCompactionExecutionScoreThreshold, DOUBLE, Defaults.S3ObjectCompactionExecutionScoreThreshold, MEDIUM, S3ObjectCompactionExecutionScoreThresholdDoc) - .define(S3ObjectCompactionStreamSplitSize, LONG, Defaults.S3ObjectCompactionStreamSplitSize, MEDIUM, S3ObjectCompactionStreamSplitSizeDoc) + .define(S3ObjectCompactionIntervalProp, INT, Defaults.S3ObjectCompactionInterval, MEDIUM, S3ObjectCompactionIntervalDoc) + .define(S3ObjectCompactionCacheSizeProp, LONG, Defaults.S3ObjectCompactionCacheSize, MEDIUM, S3ObjectCompactionCacheSizeDoc) + .define(S3ObjectCompactionNWInBandwidthProp, LONG, Defaults.S3ObjectCompactionNWInBandwidth, MEDIUM, S3ObjectCompactionNWInBandwidthDoc) + .define(S3ObjectCompactionNWOutBandwidthProp, LONG, Defaults.S3ObjectCompactionNWOutBandwidth, MEDIUM, S3ObjectCompactionNWOutBandwidthDoc) + .define(S3ObjectCompactionUploadConcurrencyProp, INT, Defaults.S3ObjectCompactionUploadConcurrency, MEDIUM, S3ObjectCompactionUploadConcurrencyDoc) + .define(S3ObjectCompactionExecutionScoreThresholdProp, DOUBLE, Defaults.S3ObjectCompactionExecutionScoreThreshold, MEDIUM, S3ObjectCompactionExecutionScoreThresholdDoc) + .define(S3ObjectCompactionStreamSplitSizeProp, LONG, Defaults.S3ObjectCompactionStreamSplitSize, MEDIUM, S3ObjectCompactionStreamSplitSizeDoc) .define(S3CacheSizeProp, INT, 1024, MEDIUM, S3CacheSizeDoc) // Kafka on S3 inject end } @@ -2080,12 +2084,13 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val s3ObjectBlockSize = getInt(KafkaConfig.S3ObjectBlockSizeProp) val s3ObjectPartSize = getInt(KafkaConfig.S3ObjectPartSizeProp) val s3CacheSize = getLong(KafkaConfig.S3CacheSizeProp) - val s3ObjectCompactionCacheSize = getLong(KafkaConfig.S3ObjectCompactionCacheSize) - val s3ObjectCompactionNWInBandwidth = getLong(KafkaConfig.S3ObjectCompactionNWInBandwidth) - val s3ObjectCompactionNWOutBandwidth = getLong(KafkaConfig.S3ObjectCompactionNWInBandwidth) - val s3ObjectCompactionUploadConcurrency = getInt(KafkaConfig.S3ObjectCompactionUploadConcurrency) - val s3ObjectCompactionExecutionScoreThreshold = getDouble(KafkaConfig.S3ObjectCompactionExecutionScoreThreshold) - val s3ObjectCompactionStreamSplitSize = getLong(KafkaConfig.S3ObjectCompactionStreamSplitSize) + val s3ObjectCompactionInterval = getInt(KafkaConfig.S3ObjectCompactionIntervalProp) + val s3ObjectCompactionCacheSize = getLong(KafkaConfig.S3ObjectCompactionCacheSizeProp) + val s3ObjectCompactionNWInBandwidth = getLong(KafkaConfig.S3ObjectCompactionNWInBandwidthProp) + val s3ObjectCompactionNWOutBandwidth = getLong(KafkaConfig.S3ObjectCompactionNWInBandwidthProp) + val s3ObjectCompactionUploadConcurrency = getInt(KafkaConfig.S3ObjectCompactionUploadConcurrencyProp) + val s3ObjectCompactionExecutionScoreThreshold = getDouble(KafkaConfig.S3ObjectCompactionExecutionScoreThresholdProp) + val s3ObjectCompactionStreamSplitSize = getLong(KafkaConfig.S3ObjectCompactionStreamSplitSizeProp) // Kafka on S3 inject end def addReconfigurable(reconfigurable: Reconfigurable): Unit = { diff --git a/core/src/test/java/kafka/log/s3/compact/CompactionAnalyzerTest.java b/core/src/test/java/kafka/log/s3/compact/CompactionAnalyzerTest.java index 288c829cd9..f8af0576b7 100644 --- a/core/src/test/java/kafka/log/s3/compact/CompactionAnalyzerTest.java +++ b/core/src/test/java/kafka/log/s3/compact/CompactionAnalyzerTest.java @@ -97,9 +97,9 @@ public void testBuildCompactedObject1() { List compactedObjectBuilders = compactionAnalyzer.buildCompactedObjects(S3_WAL_OBJECT_METADATA_LIST); 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)), + .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)) diff --git a/core/src/test/java/kafka/log/s3/compact/CompactionManagerTest.java b/core/src/test/java/kafka/log/s3/compact/CompactionManagerTest.java index 8b23e3f4c9..5f4867dacc 100644 --- a/core/src/test/java/kafka/log/s3/compact/CompactionManagerTest.java +++ b/core/src/test/java/kafka/log/s3/compact/CompactionManagerTest.java @@ -19,27 +19,17 @@ import kafka.log.s3.compact.objects.CompactionType; import kafka.log.s3.compact.objects.StreamDataBlock; -import kafka.log.s3.memory.MemoryMetadataManager; import kafka.log.s3.metadata.StreamMetadataManager; import kafka.log.s3.objects.CommitWALObjectRequest; -import kafka.log.s3.objects.StreamObject; -import kafka.log.s3.operator.MemoryS3Operator; import kafka.server.KafkaConfig; -import org.apache.kafka.metadata.stream.S3ObjectMetadata; -import org.apache.kafka.metadata.stream.S3ObjectType; -import org.apache.kafka.metadata.stream.S3WALObject; import org.apache.kafka.metadata.stream.S3WALObjectMetadata; -import org.apache.kafka.metadata.stream.StreamOffsetRange; 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.Mockito; -import java.util.Collections; import java.util.List; -import java.util.Map; -import java.util.concurrent.CompletableFuture; import java.util.stream.Collectors; public class CompactionManagerTest extends CompactionTestBase { diff --git a/core/src/test/java/kafka/log/s3/compact/CompactionTestBase.java b/core/src/test/java/kafka/log/s3/compact/CompactionTestBase.java index 084df2a4a2..857265e46f 100644 --- a/core/src/test/java/kafka/log/s3/compact/CompactionTestBase.java +++ b/core/src/test/java/kafka/log/s3/compact/CompactionTestBase.java @@ -17,7 +17,6 @@ package kafka.log.s3.compact; -import com.automq.elasticstream.client.flatc.header.ObjectMetadata; import kafka.log.s3.ObjectWriter; import kafka.log.s3.TestUtils; import kafka.log.s3.compact.objects.CompactedObject; @@ -53,7 +52,7 @@ public class CompactionTestBase { protected static final List S3_WAL_OBJECT_METADATA_LIST = new ArrayList<>(); protected MemoryMetadataManager objectManager; protected S3Operator s3Operator; - + public void setUp() throws Exception { objectManager = new MemoryMetadataManager(); objectManager.start(); @@ -118,7 +117,7 @@ public void setUp() throws Exception { S3_WAL_OBJECT_METADATA_LIST.add(new S3WALObjectMetadata(walObject, objectMetadata)); }).join(); } - + public void tearDown() { S3_WAL_OBJECT_METADATA_LIST.clear(); objectManager.shutdown(); From 286c7433af27a078d67882d609026a65865ba072 Mon Sep 17 00:00:00 2001 From: Shichao Nie Date: Fri, 8 Sep 2023 18:03:32 +0800 Subject: [PATCH 5/7] fix: remove duplicat config key Signed-off-by: Shichao Nie --- core/src/main/scala/kafka/server/KafkaConfig.scala | 5 ++++- .../java/kafka/log/s3/compact/CompactionAnalyzerTest.java | 2 ++ .../java/kafka/log/s3/compact/CompactionManagerTest.java | 4 ++++ 3 files changed, 10 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index a4daa95062..7029b015ef 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -318,6 +318,7 @@ object Defaults { val S3ObjectCompactionUploadConcurrency: Int = 8 val S3ObjectCompactionExecutionScoreThreshold: Double = 0.5 val S3ObjectCompactionStreamSplitSize: Long = 16 * 1024 * 1024 // 16MB + val S3ObjectCompactionForceSplitPeriod: Int = 60 // 60min } object KafkaConfig { @@ -702,6 +703,7 @@ object KafkaConfig { val S3ObjectCompactionUploadConcurrencyProp = "s3.object.compaction.upload.concurrency" val S3ObjectCompactionExecutionScoreThresholdProp = "s3.object.compaction.execution.score.threshold" val S3ObjectCompactionStreamSplitSizeProp = "s3.object.compaction.stream.split.size" + val S3ObjectCompactionForceSplitPeriodProp = "s3.object.compaction.force.split.time" val S3EndpointDoc = "The S3 endpoint, ex. https://s3.{region}.amazonaws.com." val S3RegionDoc = "The S3 region, ex. us-east-1." @@ -721,6 +723,7 @@ object KafkaConfig { val S3ObjectCompactionUploadConcurrencyDoc = "The S3 object compaction upload concurrency." val S3ObjectCompactionExecutionScoreThresholdDoc = "The S3 object compaction execution score threshold." val S3ObjectCompactionStreamSplitSizeDoc = "The S3 object compaction stream split size threshold in Bytes." + val S3ObjectCompactionForceSplitPeriodDoc = "The S3 object compaction force split period in minutes." // Kafka on S3 inject end @@ -1538,7 +1541,7 @@ object KafkaConfig { .define(S3ObjectCompactionUploadConcurrencyProp, INT, Defaults.S3ObjectCompactionUploadConcurrency, MEDIUM, S3ObjectCompactionUploadConcurrencyDoc) .define(S3ObjectCompactionExecutionScoreThresholdProp, DOUBLE, Defaults.S3ObjectCompactionExecutionScoreThreshold, MEDIUM, S3ObjectCompactionExecutionScoreThresholdDoc) .define(S3ObjectCompactionStreamSplitSizeProp, LONG, Defaults.S3ObjectCompactionStreamSplitSize, MEDIUM, S3ObjectCompactionStreamSplitSizeDoc) - .define(S3CacheSizeProp, INT, 1024, MEDIUM, S3CacheSizeDoc) + .define(S3ObjectCompactionForceSplitPeriodProp, INT, Defaults.S3ObjectCompactionForceSplitPeriod, MEDIUM, S3ObjectCompactionForceSplitPeriodDoc) // Kafka on S3 inject end } diff --git a/core/src/test/java/kafka/log/s3/compact/CompactionAnalyzerTest.java b/core/src/test/java/kafka/log/s3/compact/CompactionAnalyzerTest.java index f8af0576b7..6ed80b78fe 100644 --- a/core/src/test/java/kafka/log/s3/compact/CompactionAnalyzerTest.java +++ b/core/src/test/java/kafka/log/s3/compact/CompactionAnalyzerTest.java @@ -31,11 +31,13 @@ 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.Map; +@Timeout(30) @Tag("S3Unit") public class CompactionAnalyzerTest extends CompactionTestBase { diff --git a/core/src/test/java/kafka/log/s3/compact/CompactionManagerTest.java b/core/src/test/java/kafka/log/s3/compact/CompactionManagerTest.java index 5f4867dacc..8de7d4627d 100644 --- a/core/src/test/java/kafka/log/s3/compact/CompactionManagerTest.java +++ b/core/src/test/java/kafka/log/s3/compact/CompactionManagerTest.java @@ -26,12 +26,16 @@ 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 java.util.List; import java.util.stream.Collectors; +@Timeout(30) +@Tag("S3Unit") public class CompactionManagerTest extends CompactionTestBase { private static final int BROKER0 = 0; private StreamMetadataManager streamMetadataManager; From d72ee95e4a5373a1f4621bf7ab3da432796282d4 Mon Sep 17 00:00:00 2001 From: Shichao Nie Date: Mon, 11 Sep 2023 11:29:09 +0800 Subject: [PATCH 6/7] feat: Add force split interface in compaction Signed-off-by: Shichao Nie --- .../log/s3/compact/CompactionManager.java | 39 +++++++++++++++++-- .../log/s3/compact/CompactionUploader.java | 8 ++-- .../s3/compact/operator/DataBlockWriter.java | 6 ++- .../log/s3/operator/MemoryS3Operator.java | 4 +- .../main/scala/kafka/server/KafkaConfig.scala | 1 + .../kafka/metadata/stream/S3WALObject.java | 14 ++++++- 6 files changed, 60 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/kafka/log/s3/compact/CompactionManager.java b/core/src/main/scala/kafka/log/s3/compact/CompactionManager.java index a9a2050355..a1c8be58c2 100644 --- a/core/src/main/scala/kafka/log/s3/compact/CompactionManager.java +++ b/core/src/main/scala/kafka/log/s3/compact/CompactionManager.java @@ -54,6 +54,7 @@ public class CompactionManager { private final double executionScoreThreshold; private final long streamSplitSize; private final int compactionInterval; + private final int forceSplitObjectPeriod; private final TokenBucketThrottle networkInThrottle; public CompactionManager(KafkaConfig config, ObjectManager objectManager, StreamMetadataManager streamMetadataManager, S3Operator s3Operator) { @@ -64,6 +65,7 @@ public CompactionManager(KafkaConfig config, ObjectManager objectManager, Stream this.compactionCacheSize = config.s3ObjectCompactionCacheSize(); this.executionScoreThreshold = config.s3ObjectCompactionExecutionScoreThreshold(); this.streamSplitSize = config.s3ObjectCompactionStreamSplitSize(); + this.forceSplitObjectPeriod = config.s3ObjectCompactionForceSplitPeriod(); this.networkInThrottle = new TokenBucketThrottle(config.s3ObjectCompactionNWInBandwidth()); this.uploader = new CompactionUploader(objectManager, s3Operator, config); this.compactionAnalyzer = new CompactionAnalyzer(compactionCacheSize, executionScoreThreshold, streamSplitSize, s3Operator); @@ -71,7 +73,16 @@ public CompactionManager(KafkaConfig config, ObjectManager objectManager, Stream } public void start() { - this.executorService.scheduleWithFixedDelay(this::compact, 0, this.compactionInterval, TimeUnit.MINUTES); + this.executorService.scheduleWithFixedDelay(() -> { + long start = System.currentTimeMillis(); + this.compact() + .thenAccept(result -> LOGGER.info("Compaction complete, cost {} ms, result {}", + System.currentTimeMillis() - start, result)) + .exceptionally(ex -> { + LOGGER.error("Compaction failed, cost {} ms, ", System.currentTimeMillis() - start, ex); + return null; + }); + }, 0, this.compactionInterval, TimeUnit.MINUTES); } public void shutdown() { @@ -82,14 +93,20 @@ public void shutdown() { public CompletableFuture compact() { List s3ObjectMetadata = this.streamMetadataManager.getWALObjects(); + Map> objectMetadataFilterMap = s3ObjectMetadata.stream() + .collect(Collectors.partitioningBy(e -> (System.currentTimeMillis() - e.getWalObject().getTimestamp()) + >= this.forceSplitObjectPeriod)); + // force split objects that exists for too long + splitWALObjects(objectMetadataFilterMap.get(true)); + try { - List compactionPlans = this.compactionAnalyzer.analyze(s3ObjectMetadata); + List compactionPlans = this.compactionAnalyzer.analyze(objectMetadataFilterMap.get(false)); if (compactionPlans.isEmpty()) { return CompletableFuture.completedFuture(CompactResult.SKIPPED); } CommitWALObjectRequest request = buildCompactRequest(compactionPlans, s3ObjectMetadata); return objectManager.commitWALObject(request).thenApply(nil -> { - LOGGER.info("Compaction success, WAL object id: {}, size: {}, stream object num: {}", + LOGGER.info("Commit compact request succeed, WAL object id: {}, size: {}, stream object num: {}", request.getObjectId(), request.getObjectSize(), request.getStreamObjects().size()); return CompactResult.SUCCESS; }); @@ -100,6 +117,22 @@ public CompletableFuture compact() { } + public CompletableFuture forceSplitAll() { + CompletableFuture cf = new CompletableFuture<>(); + this.executorService.execute(() -> splitWALObjects(this.streamMetadataManager.getWALObjects()) + .thenAccept(v -> cf.complete(null)) + .exceptionally(ex -> { + cf.completeExceptionally(ex); + return null; + }) + ); + return cf; + } + + CompletableFuture splitWALObjects(List objectMetadataList) { + return CompletableFuture.completedFuture(null); + } + CommitWALObjectRequest buildCompactRequest(List compactionPlans, List s3ObjectMetadata) throws IllegalArgumentException { CommitWALObjectRequest request = new CommitWALObjectRequest(); diff --git a/core/src/main/scala/kafka/log/s3/compact/CompactionUploader.java b/core/src/main/scala/kafka/log/s3/compact/CompactionUploader.java index f0a6593e9f..6c913d1d37 100644 --- a/core/src/main/scala/kafka/log/s3/compact/CompactionUploader.java +++ b/core/src/main/scala/kafka/log/s3/compact/CompactionUploader.java @@ -32,15 +32,15 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; public class CompactionUploader { private final static Logger LOGGER = LoggerFactory.getLogger(CompactionUploader.class); private final ObjectManager objectManager; private final TokenBucketThrottle throttle; - private final ScheduledExecutorService executorService; + private final ExecutorService executorService; private final S3Operator s3Operator; private final KafkaConfig kafkaConfig; private CompletableFuture walObjectIdCf = null; @@ -52,8 +52,8 @@ public CompactionUploader(ObjectManager objectManager, S3Operator s3Operator, Ka this.s3Operator = s3Operator; this.kafkaConfig = kafkaConfig; this.throttle = new TokenBucketThrottle(kafkaConfig.s3ObjectCompactionNWOutBandwidth()); - this.executorService = Executors.newScheduledThreadPool(kafkaConfig.s3ObjectCompactionUploadConcurrency(), - ThreadUtils.createThreadFactory("compaction-uploader", true)); + this.executorService = Executors.newFixedThreadPool(kafkaConfig.s3ObjectCompactionUploadConcurrency(), + ThreadUtils.createThreadFactory("compaction-uploader-%d", true)); } public void stop() { diff --git a/core/src/main/scala/kafka/log/s3/compact/operator/DataBlockWriter.java b/core/src/main/scala/kafka/log/s3/compact/operator/DataBlockWriter.java index 82c8cb9abd..c2b222d0fd 100644 --- a/core/src/main/scala/kafka/log/s3/compact/operator/DataBlockWriter.java +++ b/core/src/main/scala/kafka/log/s3/compact/operator/DataBlockWriter.java @@ -116,7 +116,7 @@ class IndexBlock { public IndexBlock() { position = nextDataBlockPosition; - buf = Unpooled.buffer(1024 * 1024); + buf = Unpooled.buffer(calculateIndexBlockSize()); buf.writeInt(completedBlocks.size()); // block count // block index for (StreamDataBlock block : completedBlocks) { @@ -134,6 +134,10 @@ public IndexBlock() { } } + private int calculateIndexBlockSize() { + return 4 + completedBlocks.size() * 40; + } + public ByteBuf buffer() { return buf.duplicate(); } diff --git a/core/src/main/scala/kafka/log/s3/operator/MemoryS3Operator.java b/core/src/main/scala/kafka/log/s3/operator/MemoryS3Operator.java index 91c66b207f..856789008d 100644 --- a/core/src/main/scala/kafka/log/s3/operator/MemoryS3Operator.java +++ b/core/src/main/scala/kafka/log/s3/operator/MemoryS3Operator.java @@ -22,12 +22,12 @@ import kafka.log.es.FutureUtil; import software.amazon.awssdk.services.s3.model.CompletedPart; -import java.util.HashMap; import java.util.Map; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; public class MemoryS3Operator implements S3Operator { - private final Map storage = new HashMap<>(); + private final Map storage = new ConcurrentHashMap<>(); @Override public void close() { diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 7029b015ef..e6cae25363 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -2094,6 +2094,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val s3ObjectCompactionUploadConcurrency = getInt(KafkaConfig.S3ObjectCompactionUploadConcurrencyProp) val s3ObjectCompactionExecutionScoreThreshold = getDouble(KafkaConfig.S3ObjectCompactionExecutionScoreThresholdProp) val s3ObjectCompactionStreamSplitSize = getLong(KafkaConfig.S3ObjectCompactionStreamSplitSizeProp) + val s3ObjectCompactionForceSplitPeriod = getInt(KafkaConfig.S3ObjectCompactionForceSplitPeriodProp) // Kafka on S3 inject end def addReconfigurable(reconfigurable: Reconfigurable): Unit = { diff --git a/metadata/src/main/java/org/apache/kafka/metadata/stream/S3WALObject.java b/metadata/src/main/java/org/apache/kafka/metadata/stream/S3WALObject.java index 69af4385ab..4c730ad090 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/stream/S3WALObject.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/stream/S3WALObject.java @@ -41,11 +41,18 @@ public class S3WALObject implements Comparable { private final S3ObjectType objectType = S3ObjectType.UNKNOWN; + private final long timestamp; + public S3WALObject(long objectId, int brokerId, final Map> streamsIndex, long orderId) { + this(objectId, brokerId, streamsIndex, orderId, System.currentTimeMillis()); + } + + public S3WALObject(long objectId, int brokerId, final Map> streamsIndex, long orderId, long timestamp) { this.orderId = orderId; this.objectId = objectId; this.brokerId = brokerId; this.streamsIndex = streamsIndex; + this.timestamp = timestamp; } public boolean intersect(long streamId, long startOffset, long endOffset) { @@ -77,9 +84,8 @@ public static S3WALObject of(WALObjectRecord record) { Map> collect = record.streamsIndex().stream() .map(index -> new StreamOffsetRange(index.streamId(), index.startOffset(), index.endOffset())) .collect(Collectors.groupingBy(StreamOffsetRange::getStreamId)); - S3WALObject s3WalObject = new S3WALObject(record.objectId(), record.brokerId(), + return new S3WALObject(record.objectId(), record.brokerId(), collect, record.orderId()); - return s3WalObject; } public Integer brokerId() { @@ -98,6 +104,10 @@ public long orderId() { return orderId; } + public long getTimestamp() { + return timestamp; + } + @Override public boolean equals(Object o) { if (this == o) { From 67def3fe8db064e72180a024f55b89945091afdc Mon Sep 17 00:00:00 2001 From: Shichao Nie Date: Mon, 11 Sep 2023 12:28:10 +0800 Subject: [PATCH 7/7] fix: fix unit test Signed-off-by: Shichao Nie --- .../src/test/java/kafka/log/s3/StreamMetadataManagerTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/java/kafka/log/s3/StreamMetadataManagerTest.java b/core/src/test/java/kafka/log/s3/StreamMetadataManagerTest.java index 5cb158a68f..5019889eef 100644 --- a/core/src/test/java/kafka/log/s3/StreamMetadataManagerTest.java +++ b/core/src/test/java/kafka/log/s3/StreamMetadataManagerTest.java @@ -216,8 +216,8 @@ public void testFetch() throws Exception { public void testGetWALObjects() { this.streamMetadataListener.onChange(null, image0); List objectMetadata = this.manager.getWALObjects(); - List expected = List.of(new S3ObjectMetadata(1L, 128, S3ObjectType.UNKNOWN), - new S3ObjectMetadata(2L, 128, S3ObjectType.UNKNOWN)); + List expected = List.of(new S3ObjectMetadata(1L, 128, S3ObjectType.WAL), + new S3ObjectMetadata(2L, 128, S3ObjectType.WAL)); // compare objectMetadata with expected assertEquals(expected.size(), objectMetadata.size()); for (int i = 0; i < expected.size(); i++) {