diff --git a/core/src/main/scala/kafka/log/s3/ObjectWriter.java b/core/src/main/scala/kafka/log/s3/ObjectWriter.java index 386bcf0f8f..891f94a723 100644 --- a/core/src/main/scala/kafka/log/s3/ObjectWriter.java +++ b/core/src/main/scala/kafka/log/s3/ObjectWriter.java @@ -21,7 +21,6 @@ import io.netty.buffer.CompositeByteBuf; import io.netty.buffer.Unpooled; import kafka.log.s3.model.StreamRecordBatch; -import kafka.log.s3.objects.ObjectManager; import kafka.log.s3.objects.ObjectStreamRange; import kafka.log.s3.operator.S3Operator; import kafka.log.s3.operator.Writer; @@ -37,15 +36,13 @@ // TODO: memory optimization public class ObjectWriter { - private int blockSizeThreshold; - private int partSizeThreshold; - private final S3Operator s3Operator; + private final int blockSizeThreshold; + private final int partSizeThreshold; private final List waitingUploadBlocks; private final List completedBlocks; private IndexBlock indexBlock; private final Writer writer; private final long objectId; - private final String objectKey; private long nextDataBlockPosition; private long size; @@ -54,8 +51,7 @@ public class ObjectWriter { public ObjectWriter(long objectId, S3Operator s3Operator, int blockSizeThreshold, int partSizeThreshold) { this.objectId = objectId; - this.objectKey = ObjectUtils.genKey(0, "todocluster", objectId); - this.s3Operator = s3Operator; + String objectKey = ObjectUtils.genKey(0, "todocluster", objectId); this.blockSizeThreshold = blockSizeThreshold; this.partSizeThreshold = partSizeThreshold; waitingUploadBlocks = new LinkedList<>(); diff --git a/core/src/main/scala/kafka/log/s3/S3Wal.java b/core/src/main/scala/kafka/log/s3/S3Wal.java index aed67da1a1..f2b375ef04 100644 --- a/core/src/main/scala/kafka/log/s3/S3Wal.java +++ b/core/src/main/scala/kafka/log/s3/S3Wal.java @@ -48,8 +48,7 @@ public class S3Wal implements Wal { public S3Wal(ObjectManager objectManager, S3Operator s3Operator) { writeBuffer = new ArrayBlockingQueue<>(16384); walBatchWriteTask = new WalBatchWriteTask(objectManager, s3Operator); - minorCompactTask = new MinorCompactTask(5L * 1024 * 1024 * 1024, 60, 16 * 1024 * 1024 - , objectManager, s3Operator); + minorCompactTask = new MinorCompactTask(5L * 1024 * 1024 * 1024, 60, 16 * 1024 * 1024, objectManager, s3Operator); this.objectManager = objectManager; this.s3Operator = s3Operator; } diff --git a/core/src/main/scala/kafka/log/s3/SingleWalObjectWriteTask.java b/core/src/main/scala/kafka/log/s3/SingleWalObjectWriteTask.java index a649b0937b..c2406b9a01 100644 --- a/core/src/main/scala/kafka/log/s3/SingleWalObjectWriteTask.java +++ b/core/src/main/scala/kafka/log/s3/SingleWalObjectWriteTask.java @@ -24,7 +24,6 @@ import kafka.log.s3.objects.CommitWalObjectResponse; import kafka.log.s3.objects.ObjectManager; import kafka.log.s3.operator.S3Operator; -import kafka.log.s3.utils.ObjectUtils; import java.util.Collections; import java.util.HashSet;