Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
7 changes: 4 additions & 3 deletions core/src/main/scala/kafka/log/s3/DefaultS3Client.java
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import kafka.log.s3.operator.S3Operator;
import kafka.log.s3.streams.ControllerStreamManager;
import kafka.log.s3.streams.StreamManager;
import kafka.log.s3.wal.MemoryWriteAheadLog;
import kafka.server.BrokerServer;
import kafka.server.BrokerToControllerChannelManager;
import kafka.server.KafkaConfig;
Expand All @@ -44,7 +45,7 @@ public class DefaultS3Client implements Client {

private final S3Operator operator;

private final Wal wal;
private final Storage storage;

private final S3BlockCache blockCache;

Expand All @@ -64,9 +65,9 @@ public DefaultS3Client(BrokerServer brokerServer, KafkaConfig config, S3Operator
this.requestSender = new ControllerRequestSender(channelManager);
this.streamManager = new ControllerStreamManager(this.requestSender, config);
this.objectManager = new ControllerObjectManager(this.requestSender, this.metadataManager, this.config);
this.wal = new S3Wal(objectManager, operator);
this.blockCache = new DefaultS3BlockCache(objectManager, operator);
this.streamClient = new S3StreamClient(this.streamManager, this.wal, this.blockCache, this.objectManager);
this.storage = new S3Storage(new MemoryWriteAheadLog(), objectManager, blockCache, operator);
this.streamClient = new S3StreamClient(this.streamManager, this.storage);
this.kvClient = new KVClientImpl();
}

Expand Down
61 changes: 61 additions & 0 deletions core/src/main/scala/kafka/log/s3/FlatStreamRecordBatch.java
Original file line number Diff line number Diff line change
@@ -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.s3;

import io.netty.buffer.ByteBuf;
import kafka.log.s3.model.StreamRecordBatch;

public class FlatStreamRecordBatch implements Comparable<FlatStreamRecordBatch> {
public long streamId;
public long epoch;
public long baseOffset;
public int count;
public ByteBuf encodedBuf;

public static FlatStreamRecordBatch from(StreamRecordBatch streamRecord) {
FlatStreamRecordBatch self = new FlatStreamRecordBatch();
self.streamId = streamRecord.getStreamId();
self.epoch = streamRecord.getEpoch();
self.baseOffset = streamRecord.getBaseOffset();
self.count = streamRecord.getRecordBatch().count();
self.encodedBuf = StreamRecordBatchCodec.encode(streamRecord);
return self;
}

public long lastOffset() {
return baseOffset + count;
}

public ByteBuf encodedBuf() {
return encodedBuf.duplicate();
}

@Override
public int compareTo(FlatStreamRecordBatch o) {
@SuppressWarnings("DuplicatedCode")
int rst = Long.compare(streamId, o.streamId);
if (rst != 0) {
return rst;
}
rst = Long.compare(epoch, o.epoch);
if (rst != 0) {
return rst;
}
return Long.compare(baseOffset, o.baseOffset);
}
}
229 changes: 0 additions & 229 deletions core/src/main/scala/kafka/log/s3/MinorCompactTask.java

This file was deleted.

19 changes: 9 additions & 10 deletions core/src/main/scala/kafka/log/s3/ObjectWriter.java
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@
import io.netty.buffer.ByteBuf;
import io.netty.buffer.CompositeByteBuf;
import io.netty.buffer.Unpooled;
import kafka.log.s3.model.StreamRecordBatch;
import kafka.log.s3.objects.ObjectStreamRange;
import kafka.log.s3.operator.S3Operator;
import kafka.log.s3.operator.Writer;
Expand Down Expand Up @@ -63,7 +62,7 @@ public ObjectWriter(long objectId, S3Operator s3Operator) {
this(objectId, s3Operator, 16 * 1024 * 1024, 32 * 1024 * 1024);
}

public void write(StreamRecordBatch record) {
public void write(FlatStreamRecordBatch record) {
if (dataBlock == null) {
dataBlock = new DataBlock(nextDataBlockPosition);
}
Expand Down Expand Up @@ -159,7 +158,7 @@ public DataBlock(long position) {
streamRanges = new LinkedList<>();
}

public boolean write(StreamRecordBatch record) {
public boolean write(FlatStreamRecordBatch record) {
try {
recordCount++;
return write0(record);
Expand All @@ -169,17 +168,17 @@ public boolean write(StreamRecordBatch record) {
}
}

public boolean write0(StreamRecordBatch record) throws IOException {
if (streamRange == null || streamRange.getStreamId() != record.getStreamId()) {
public boolean write0(FlatStreamRecordBatch record) throws IOException {
if (streamRange == null || streamRange.getStreamId() != record.streamId) {
streamRange = new ObjectStreamRange();
streamRange.setStreamId(record.getStreamId());
streamRange.setEpoch(record.getEpoch());
streamRange.setStartOffset(record.getBaseOffset());
streamRange.setStreamId(record.streamId);
streamRange.setEpoch(record.epoch);
streamRange.setStartOffset(record.baseOffset);
streamRanges.add(streamRange);
}
streamRange.setEndOffset(record.getBaseOffset() + record.getRecordBatch().count());
streamRange.setEndOffset(record.lastOffset());

ByteBuf recordBuf = StreamRecordBatchCodec.encode(record);
ByteBuf recordBuf = record.encodedBuf();
out.write(recordBuf.array(), recordBuf.arrayOffset(), recordBuf.readableBytes());
recordBuf.release();
blockSize += recordBuf.readableBytes();
Expand Down
Loading