You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rocketmq.apache.org by GitBox <gi...@apache.org> on 2023/01/13 15:19:50 UTC

[GitHub] [rocketmq] aaron-ai commented on a diff in pull request #5875: [ISSUE #5874] implement file queue for tiered storage

aaron-ai commented on code in PR #5875:
URL: https://github.com/apache/rocketmq/pull/5875#discussion_r1069541515


##########
tieredstore/src/main/java/org/apache/rocketmq/store/tiered/common/TieredStoreExecutor.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.rocketmq.store.tiered.common;
+
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import org.apache.rocketmq.common.ThreadFactoryImpl;
+
+public class TieredStoreExecutor {
+    private static final int QUEUE_CAPACITY = 10000;
+    private static final BlockingQueue<Runnable> DISPATCH_THREAD_POOL_QUEUE;
+    public static final ExecutorService DISPATCH_EXECUTOR;
+    public static final ScheduledExecutorService COMMON_SCHEDULED_EXECUTOR;
+
+    public static final ScheduledExecutorService COMMIT_EXECUTOR;
+
+    public static final ScheduledExecutorService CLEAN_EXPIRED_FILE_EXECUTOR;
+
+    private static final BlockingQueue<Runnable> FETCH_DATA_THREAD_POOL_QUEUE;
+    public static final ExecutorService FETCH_DATA_EXECUTOR;
+
+    private static final BlockingQueue<Runnable> COMPACT_INDEX_FILE_THREAD_POOL_QUEUE;
+    public static final ExecutorService COMPACT_INDEX_FILE_EXECUTOR;
+
+    static {
+        DISPATCH_THREAD_POOL_QUEUE = new LinkedBlockingQueue<>(QUEUE_CAPACITY);
+        DISPATCH_EXECUTOR = new ThreadPoolExecutor(

Review Comment:
   Can we construct the thread pool directly when declaring it?



##########
tieredstore/src/main/java/org/apache/rocketmq/store/tiered/common/TieredMessageStoreConfig.java:
##########
@@ -17,9 +17,126 @@
 package org.apache.rocketmq.store.tiered.common;
 
 import java.io.File;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
 
 public class TieredMessageStoreConfig {
+    private String brokerName = localHostName();
+    private String brokerClusterName = "DefaultCluster";
+    private TieredStorageLevel tieredStorageLevel = TieredStorageLevel.NOT_IN_DISK;
+    public enum TieredStorageLevel {
+        DISABLE(0),
+        NOT_IN_DISK(1),
+        NOT_IN_MEM(2),
+        FORCE(3);
+
+        private final int value;
+
+        TieredStorageLevel(int value) {
+            this.value = value;
+        }
+
+        public int getValue() {
+            return value;
+        }
+
+        public static TieredStorageLevel valueOf(int value) {
+            switch (value) {
+                case 1:
+                    return NOT_IN_DISK;
+                case 2:
+                    return NOT_IN_MEM;
+                case 3:
+                    return FORCE;
+                default:
+                    return DISABLE;
+            }
+        }
+
+        public boolean isEnable() {
+            return this.value > 0;
+        }
+
+        public boolean check(TieredStorageLevel targetLevel) {
+            return this.value >= targetLevel.value;
+        }
+    }
+
     private String storePathRootDir = System.getProperty("user.home") + File.separator + "store";
+    private boolean messageIndexEnable = true;
+
+    // CommitLog file size, default is 1G
+    private long tieredStoreCommitLogMaxSize = 1024 * 1024 * 1024;
+    // ConsumeQueue file size, default is 100M
+    private long tieredStoreConsumeQueueMaxSize = 100 * 1024 * 1024;
+    private int tieredStoreIndexFileMaxHashSlotNum = 5000000;
+    private int tieredStoreIndexFileMaxIndexNum = 5000000 * 4;
+    // index file will force rolling to next file after idle specified time, default is 3h
+    private int tieredStoreIndexFileRollingIdleInterval = 3 * 60 * 60 * 1000;
+    private String tieredMetadataServiceProvider = "org.apache.rocketmq.store.tiered.metadata.TieredMetadataManager";
+    private String tieredBackendServiceProvider = "";
+    // file reserved time, default is 72 hour
+    private int tieredStoreFileReservedTime = 72;

Review Comment:
   Maybe It's a good idea to add time unit on the configuration name



##########
tieredstore/src/main/java/org/apache/rocketmq/store/tiered/metadata/TieredMetadataStore.java:
##########
@@ -29,10 +35,32 @@ public interface TieredStoreMetadataStore {
     void updateTopicStatus(String topic, int status);

Review Comment:
   blank line for new method.



##########
tieredstore/src/main/java/org/apache/rocketmq/store/tiered/container/TieredFileSegment.java:
##########
@@ -0,0 +1,529 @@
+/*
+ * 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.rocketmq.store.tiered.container;
+
+import com.google.common.base.Stopwatch;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.apache.rocketmq.logging.org.slf4j.Logger;
+import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
+import org.apache.rocketmq.store.tiered.common.AppendResult;
+import org.apache.rocketmq.store.tiered.common.TieredMessageStoreConfig;
+import org.apache.rocketmq.store.tiered.exception.TieredStoreErrorCode;
+import org.apache.rocketmq.store.tiered.exception.TieredStoreException;
+import org.apache.rocketmq.store.tiered.util.MessageBufferUtil;
+import org.apache.rocketmq.store.tiered.util.TieredStoreUtil;
+
+public abstract class TieredFileSegment implements Comparable<TieredFileSegment> {
+    private static final Logger LOGGER = LoggerFactory.getLogger(TieredStoreUtil.TIERED_STORE_LOGGER_NAME);
+    private volatile boolean closed = false;
+    private final ReentrantLock bufferLock = new ReentrantLock();
+    private final Semaphore commitLock = new Semaphore(1);
+    private List<ByteBuffer> uploadBufferList = new ArrayList<>();
+    private boolean full;
+    protected final FileSegmentType fileType;
+    protected final MessageQueue messageQueue;
+    protected final TieredMessageStoreConfig storeConfig;
+    protected final long baseOffset;
+    private volatile long commitPosition;
+    private volatile long appendPosition;
+    private final long maxSize;
+    private long beginTimestamp = Long.MAX_VALUE;
+    private long endTimestamp = Long.MAX_VALUE;
+    // only used in commitLog
+    private long commitMsgQueueOffset = 0;
+    private ByteBuffer codaBuffer;
+
+    private CompletableFuture<Boolean> inflightCommitRequest = CompletableFuture.completedFuture(false);
+
+    public TieredFileSegment(FileSegmentType fileType, MessageQueue messageQueue, long baseOffset,
+        TieredMessageStoreConfig storeConfig) {
+        this.fileType = fileType;
+        this.messageQueue = messageQueue;
+        this.storeConfig = storeConfig;
+        this.baseOffset = baseOffset;
+        this.commitPosition = 0;
+        this.appendPosition = 0;
+        switch (fileType) {
+            case COMMIT_LOG:
+                this.maxSize = storeConfig.getTieredStoreCommitLogMaxSize();
+                break;
+            case CONSUME_QUEUE:
+                this.maxSize = storeConfig.getTieredStoreConsumeQueueMaxSize();
+                break;
+            case INDEX:
+                this.maxSize = Long.MAX_VALUE;
+                break;
+            default:
+                throw new IllegalArgumentException("Unsupported file type: " + fileType);
+        }
+    }
+
+    @Override
+    public int compareTo(TieredFileSegment o) {
+        return Long.compare(this.baseOffset, o.baseOffset);
+    }
+
+    public long getBaseOffset() {
+        return baseOffset;
+    }
+
+    public long getCommitOffset() {
+        return baseOffset + commitPosition;
+    }
+
+    public long getCommitPosition() {
+        return commitPosition;
+    }
+
+    public long getCommitMsgQueueOffset() {
+        return commitMsgQueueOffset;
+    }
+
+    public long getMaxOffset() {
+        return baseOffset + appendPosition;
+    }
+
+    public long getMaxSize() {
+        return maxSize;
+    }
+
+    public long getBeginTimestamp() {
+        return beginTimestamp;
+    }
+
+    protected void setBeginTimestamp(long beginTimestamp) {
+        this.beginTimestamp = beginTimestamp;
+    }
+
+    public long getEndTimestamp() {
+        return endTimestamp;
+    }
+
+    protected void setEndTimestamp(long endTimestamp) {
+        this.endTimestamp = endTimestamp;
+    }
+
+    public boolean isFull() {
+        return full;
+    }
+
+    public void setFull() {
+        setFull(true);
+    }
+
+    public void setFull(boolean appendCoda) {
+        bufferLock.lock();
+        try {
+            full = true;
+            if (fileType == FileSegmentType.COMMIT_LOG && appendCoda) {
+                appendCoda();
+            }
+        } finally {
+            bufferLock.unlock();
+        }
+    }
+
+    public boolean isClosed() {
+        return closed;
+    }
+
+    public void close() {
+        closed = true;
+    }
+
+    public FileSegmentType getFileType() {
+        return fileType;
+    }
+
+    public MessageQueue getMessageQueue() {
+        return messageQueue;
+    }
+
+    public void initPosition(long pos) {
+        this.commitPosition = pos;
+        this.appendPosition = pos;
+    }
+
+    private List<ByteBuffer> rollingUploadBuffer() {
+        bufferLock.lock();
+        try {
+            List<ByteBuffer> tmp = uploadBufferList;
+            uploadBufferList = new ArrayList<>();
+            return tmp;
+        } finally {
+            bufferLock.unlock();
+        }
+    }
+
+    private void sendBackBuffer(TieredFileSegmentInputStream inputStream) {
+        bufferLock.lock();
+        try {
+            List<ByteBuffer> tmpBufferList = inputStream.getUploadBufferList();
+            for (ByteBuffer buffer : tmpBufferList) {
+                buffer.rewind();
+            }
+            tmpBufferList.addAll(uploadBufferList);
+            uploadBufferList = tmpBufferList;
+            if (inputStream.getCodaBuffer() != null) {
+                codaBuffer.rewind();
+            }
+        } finally {
+            bufferLock.unlock();
+        }
+    }
+
+    public AppendResult append(ByteBuffer byteBuf, long timeStamp) {
+        if (closed) {
+            return AppendResult.FILE_CLOSE;
+        }
+        bufferLock.lock();
+        try {
+            if (full || codaBuffer != null) {
+                return AppendResult.FILE_FULL;
+            }
+
+            if (fileType == FileSegmentType.INDEX) {
+                beginTimestamp = byteBuf.getLong(TieredIndexFile.INDEX_FILE_HEADER_BEGIN_TIME_STAMP_POSITION);
+                endTimestamp = byteBuf.getLong(TieredIndexFile.INDEX_FILE_HEADER_END_TIME_STAMP_POSITION);
+                appendPosition += byteBuf.remaining();
+                uploadBufferList.add(byteBuf);
+                setFull();
+                return AppendResult.SUCCESS;
+            }
+
+            if (appendPosition + byteBuf.remaining() > maxSize) {
+                setFull();
+                return AppendResult.FILE_FULL;
+            }
+            if (uploadBufferList.size() > storeConfig.getTieredStoreGroupCommitCount()
+                || appendPosition - commitPosition > storeConfig.getTieredStoreGroupCommitSize()) {
+                commitAsync();
+            }
+            if (uploadBufferList.size() > storeConfig.getTieredStoreMaxGroupCommitCount()) {
+                LOGGER.debug("TieredFileSegment#append: buffer full: file: {}, upload buffer size: {}",
+                    getPath(), uploadBufferList.size());
+                return AppendResult.BUFFER_FULL;
+            }
+            if (timeStamp != Long.MAX_VALUE) {
+                endTimestamp = timeStamp;
+                if (beginTimestamp == Long.MAX_VALUE) {
+                    beginTimestamp = timeStamp;
+                }
+            }
+            appendPosition += byteBuf.remaining();
+            uploadBufferList.add(byteBuf);
+            return AppendResult.SUCCESS;
+        } finally {
+            bufferLock.unlock();
+        }
+    }
+
+    private void appendCoda() {
+        if (codaBuffer != null) {
+            return;
+        }
+        codaBuffer = ByteBuffer.allocate(TieredCommitLog.CODA_SIZE);
+        codaBuffer.putInt(TieredCommitLog.CODA_SIZE);
+        codaBuffer.putInt(TieredCommitLog.BLANK_MAGIC_CODE);
+        codaBuffer.putLong(endTimestamp);
+        codaBuffer.flip();
+        appendPosition += TieredCommitLog.CODA_SIZE;
+    }
+
+    public ByteBuffer read(long position, int length) {
+        return readAsync(position, length).join();
+    }
+
+    public CompletableFuture<ByteBuffer> readAsync(long position, int length) {
+        if (position < 0 || length < 0) {
+            throw new TieredStoreException(TieredStoreErrorCode.ILLEGAL_PARAM, "position or length is negative");
+        }
+        if (length == 0) {
+            throw new TieredStoreException(TieredStoreErrorCode.ILLEGAL_PARAM, "length is zero");
+        }
+        if (position + length > commitPosition) {
+            LOGGER.warn("TieredFileSegment#readAsync request position + length is greater than commit position," +
+                    " correct length using commit position, file: {}, request position: {}, commit position:{}, change length from {} to {}",
+                getPath(), position, commitPosition, length, commitPosition - position);
+            length = (int) (commitPosition - position);
+            if (length == 0) {
+                throw new TieredStoreException(TieredStoreErrorCode.NO_NEW_DATA, "request position is equal to commit position");
+            }
+            if (fileType == FileSegmentType.CONSUME_QUEUE && length % TieredConsumeQueue.CONSUME_QUEUE_STORE_UNIT_SIZE != 0) {
+                throw new TieredStoreException(TieredStoreErrorCode.ILLEGAL_PARAM, "position and length is illegal");
+            }
+        }
+        return read0(position, length);
+    }
+
+    public boolean needCommit() {
+        return appendPosition > commitPosition;
+    }
+
+    public boolean commit() {
+        if (closed) {
+            return false;
+        }
+        Boolean result = commitAsync().join();
+        if (!result) {
+            result = inflightCommitRequest.join();
+        }
+        return result;
+    }
+
+    public CompletableFuture<Boolean> commitAsync() {
+        if (closed) {
+            return CompletableFuture.completedFuture(false);
+        }
+        Stopwatch stopwatch = Stopwatch.createStarted();
+        if (!needCommit()) {
+            return CompletableFuture.completedFuture(true);
+        }
+        try {
+            int permits = commitLock.drainPermits();
+            if (permits <= 0) {
+                return CompletableFuture.completedFuture(false);
+            }
+        } catch (Exception e) {
+            return CompletableFuture.completedFuture(false);
+        }
+        List<ByteBuffer> bufferList = rollingUploadBuffer();
+        int bufferSize = 0;
+        for (ByteBuffer buffer : bufferList) {
+            bufferSize += buffer.remaining();
+        }
+        if (codaBuffer != null) {
+            bufferSize += codaBuffer.remaining();
+        }
+        if (bufferSize == 0) {
+            return CompletableFuture.completedFuture(true);
+        }
+        TieredFileSegmentInputStream inputStream = new TieredFileSegmentInputStream(fileType, baseOffset + commitPosition, bufferList, codaBuffer, bufferSize);
+        int finalBufferSize = bufferSize;
+        try {
+            inflightCommitRequest = commit0(inputStream, commitPosition, bufferSize, fileType != FileSegmentType.INDEX)
+                .thenApply(result -> {
+                    if (result) {
+                        if (fileType == FileSegmentType.COMMIT_LOG && bufferList.size() > 0) {
+                            commitMsgQueueOffset = MessageBufferUtil.getQueueOffset(bufferList.get(bufferList.size() - 1));
+                        }
+                        commitPosition += finalBufferSize;
+                        return true;
+                    }
+                    sendBackBuffer(inputStream);
+                    return false;
+                }).exceptionally(e -> handleCommitException(inputStream, e))
+                .whenComplete((result, e) -> {
+                    if (commitLock.availablePermits() == 0) {
+                        LOGGER.debug("TieredFileSegment#commitAsync: commit cost: {}ms, file: {}, item count: {}, buffer size: {}", stopwatch.elapsed(TimeUnit.MILLISECONDS), getPath(), bufferList.size(), finalBufferSize);
+                        commitLock.release();
+                    } else {
+                        LOGGER.error("[Bug]TieredFileSegment#commitAsync: commit lock is already released: available permits: {}", commitLock.availablePermits());
+                    }
+                });
+            return inflightCommitRequest;
+        } catch (Exception e) {
+            handleCommitException(inputStream, e);
+            if (commitLock.availablePermits() == 0) {
+                LOGGER.debug("TieredFileSegment#commitAsync: commit cost: {}ms, file: {}, item count: {}, buffer size: {}", stopwatch.elapsed(TimeUnit.MILLISECONDS), getPath(), bufferList.size(), finalBufferSize);
+                commitLock.release();
+            } else {
+                LOGGER.error("[Bug]TieredFileSegment#commitAsync: commit lock is already released: available permits: {}", commitLock.availablePermits());
+            }
+        }
+        return CompletableFuture.completedFuture(false);
+    }
+
+    private boolean handleCommitException(TieredFileSegmentInputStream inputStream, Throwable e) {
+        Throwable cause = e.getCause() != null ? e.getCause() : e;
+        sendBackBuffer(inputStream);
+        long realSize = 0;
+        if (cause instanceof TieredStoreException && ((TieredStoreException) cause).getPosition() > 0) {
+            realSize = ((TieredStoreException) cause).getPosition();
+        }
+        if (realSize <= 0) {
+            realSize = getSize();
+        }
+        if (realSize > 0 && realSize > commitPosition) {
+            LOGGER.error("TieredFileSegment#handleCommitException: commit failed: file: {}, try to fix position: origin: {}, real: {}", getPath(), commitPosition, realSize, cause);
+            // TODO check if this diff part is uploaded to OSS
+            long diff = appendPosition - commitPosition;
+            commitPosition = realSize;
+            appendPosition = realSize + diff;
+            // TODO check if appendPosition is large than maxOffset
+        } else if (realSize < commitPosition) {
+            LOGGER.error("[Bug]TieredFileSegment#handleCommitException: commit failed: file: {}, can not fix position: origin: {}, real: {}", getPath(), commitPosition, realSize, cause);
+        }
+        return false;
+    }
+
+    public enum FileSegmentType {
+        COMMIT_LOG(0),
+        CONSUME_QUEUE(1),
+        INDEX(2);
+
+        private int type;
+
+        FileSegmentType(int type) {
+            this.type = type;
+        }
+
+        public int getType() {
+            return type;
+        }
+
+        public static FileSegmentType valueOf(int type) {
+            switch (type) {
+                case 0:
+                    return COMMIT_LOG;
+                case 1:
+                    return CONSUME_QUEUE;
+                case 2:
+                    return INDEX;
+                default:
+                    throw new IllegalStateException("Unexpected value: " + type);
+            }
+        }
+    }
+
+    protected static class TieredFileSegmentInputStream extends InputStream {
+
+        private final FileSegmentType fileType;
+        private final List<ByteBuffer> uploadBufferList;
+        private int bufferReadIndex = 0;
+        private int readOffset = 0;
+        // only used in commitLog
+        private long commitLogOffset;
+        private final ByteBuffer commitLogOffsetBuffer = ByteBuffer.allocate(8);
+        private final ByteBuffer codaBuffer;
+        private ByteBuffer curBuffer;
+        private final int contentLength;
+        private int readBytes = 0;
+
+        public TieredFileSegmentInputStream(FileSegmentType fileType, long startOffset,
+            List<ByteBuffer> uploadBufferList, ByteBuffer codaBuffer, int contentLength) {
+            this.fileType = fileType;
+            this.commitLogOffset = startOffset;
+            this.commitLogOffsetBuffer.putLong(0, startOffset);
+            this.uploadBufferList = uploadBufferList;
+            this.codaBuffer = codaBuffer;
+            this.contentLength = contentLength;
+            if (uploadBufferList.size() > 0) {
+                this.curBuffer = uploadBufferList.get(0);
+            }
+            if (fileType == FileSegmentType.INDEX && uploadBufferList.size() != 1) {
+                LOGGER.error("[Bug]TieredFileSegmentInputStream: index file must have only one buffer");
+            }
+        }
+
+        public List<ByteBuffer> getUploadBufferList() {
+            return uploadBufferList;
+        }
+
+        public ByteBuffer getCodaBuffer() {
+            return codaBuffer;
+        }
+
+        @Override
+        public int available() {
+            return contentLength - readBytes;
+        }
+
+        @Override
+        public int read() {
+            if (bufferReadIndex >= uploadBufferList.size()) {
+                return readCoda();
+            }
+
+            int res;
+            switch (fileType) {
+                case COMMIT_LOG:
+                    if (readOffset >= curBuffer.remaining()) {
+                        bufferReadIndex++;
+                        if (bufferReadIndex >= uploadBufferList.size()) {
+                            return readCoda();
+                        }
+                        curBuffer = uploadBufferList.get(bufferReadIndex);
+                        commitLogOffset += readOffset;
+                        commitLogOffsetBuffer.putLong(0, commitLogOffset);
+                        readOffset = 0;
+                    }
+                    if (readOffset >= MessageBufferUtil.PHYSICAL_OFFSET_POSITION && readOffset < MessageBufferUtil.SYS_FLAG_OFFSET_POSITION) {
+                        res = commitLogOffsetBuffer.get(readOffset - MessageBufferUtil.PHYSICAL_OFFSET_POSITION) & 0xff;
+                        readOffset++;
+                    } else {
+                        res = curBuffer.get(readOffset++) & 0xff;
+                    }
+                    break;
+                case CONSUME_QUEUE:
+                    if (!curBuffer.hasRemaining()) {
+                        bufferReadIndex++;
+                        if (bufferReadIndex >= uploadBufferList.size()) {
+                            return -1;
+                        }
+                        curBuffer = uploadBufferList.get(bufferReadIndex);
+                    }
+                    res = curBuffer.get() & 0xff;
+                    break;
+                case INDEX:
+                    if (!curBuffer.hasRemaining()) {
+                        return -1;
+                    }
+                    res = curBuffer.get() & 0xff;
+                    break;
+                default:
+                    throw new IllegalStateException("unknown file type");
+            }
+            readBytes++;
+            return res;
+        }
+
+        private int readCoda() {
+            if (fileType != FileSegmentType.COMMIT_LOG || codaBuffer == null) {
+                return -1;
+            }
+            if (!codaBuffer.hasRemaining()) {
+                return -1;
+            }
+            readBytes++;
+            return codaBuffer.get() & 0xff;
+        }
+    }
+
+    public abstract String getPath();
+
+    public abstract long getSize();
+
+    protected abstract boolean exists();
+
+    protected abstract void createFile();
+
+    protected abstract void destroyFile();
+
+    protected abstract CompletableFuture<ByteBuffer> read0(long position, int length);

Review Comment:
   Add comments for the abstract method.



##########
tieredstore/src/main/java/org/apache/rocketmq/store/tiered/container/TieredFileSegment.java:
##########
@@ -0,0 +1,529 @@
+/*
+ * 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.rocketmq.store.tiered.container;
+
+import com.google.common.base.Stopwatch;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.apache.rocketmq.logging.org.slf4j.Logger;
+import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
+import org.apache.rocketmq.store.tiered.common.AppendResult;
+import org.apache.rocketmq.store.tiered.common.TieredMessageStoreConfig;
+import org.apache.rocketmq.store.tiered.exception.TieredStoreErrorCode;
+import org.apache.rocketmq.store.tiered.exception.TieredStoreException;
+import org.apache.rocketmq.store.tiered.util.MessageBufferUtil;
+import org.apache.rocketmq.store.tiered.util.TieredStoreUtil;
+
+public abstract class TieredFileSegment implements Comparable<TieredFileSegment> {
+    private static final Logger LOGGER = LoggerFactory.getLogger(TieredStoreUtil.TIERED_STORE_LOGGER_NAME);
+    private volatile boolean closed = false;
+    private final ReentrantLock bufferLock = new ReentrantLock();
+    private final Semaphore commitLock = new Semaphore(1);
+    private List<ByteBuffer> uploadBufferList = new ArrayList<>();
+    private boolean full;
+    protected final FileSegmentType fileType;
+    protected final MessageQueue messageQueue;
+    protected final TieredMessageStoreConfig storeConfig;
+    protected final long baseOffset;
+    private volatile long commitPosition;
+    private volatile long appendPosition;
+    private final long maxSize;
+    private long beginTimestamp = Long.MAX_VALUE;
+    private long endTimestamp = Long.MAX_VALUE;
+    // only used in commitLog
+    private long commitMsgQueueOffset = 0;
+    private ByteBuffer codaBuffer;
+
+    private CompletableFuture<Boolean> inflightCommitRequest = CompletableFuture.completedFuture(false);
+
+    public TieredFileSegment(FileSegmentType fileType, MessageQueue messageQueue, long baseOffset,
+        TieredMessageStoreConfig storeConfig) {
+        this.fileType = fileType;
+        this.messageQueue = messageQueue;
+        this.storeConfig = storeConfig;
+        this.baseOffset = baseOffset;
+        this.commitPosition = 0;
+        this.appendPosition = 0;
+        switch (fileType) {
+            case COMMIT_LOG:
+                this.maxSize = storeConfig.getTieredStoreCommitLogMaxSize();
+                break;
+            case CONSUME_QUEUE:
+                this.maxSize = storeConfig.getTieredStoreConsumeQueueMaxSize();
+                break;
+            case INDEX:
+                this.maxSize = Long.MAX_VALUE;
+                break;
+            default:
+                throw new IllegalArgumentException("Unsupported file type: " + fileType);
+        }
+    }
+
+    @Override
+    public int compareTo(TieredFileSegment o) {
+        return Long.compare(this.baseOffset, o.baseOffset);
+    }
+
+    public long getBaseOffset() {
+        return baseOffset;
+    }
+
+    public long getCommitOffset() {
+        return baseOffset + commitPosition;
+    }
+
+    public long getCommitPosition() {
+        return commitPosition;
+    }
+
+    public long getCommitMsgQueueOffset() {
+        return commitMsgQueueOffset;
+    }
+
+    public long getMaxOffset() {
+        return baseOffset + appendPosition;
+    }
+
+    public long getMaxSize() {
+        return maxSize;
+    }
+
+    public long getBeginTimestamp() {
+        return beginTimestamp;
+    }
+
+    protected void setBeginTimestamp(long beginTimestamp) {
+        this.beginTimestamp = beginTimestamp;
+    }
+
+    public long getEndTimestamp() {
+        return endTimestamp;
+    }
+
+    protected void setEndTimestamp(long endTimestamp) {
+        this.endTimestamp = endTimestamp;
+    }
+
+    public boolean isFull() {
+        return full;
+    }
+
+    public void setFull() {
+        setFull(true);
+    }
+
+    public void setFull(boolean appendCoda) {
+        bufferLock.lock();
+        try {
+            full = true;
+            if (fileType == FileSegmentType.COMMIT_LOG && appendCoda) {
+                appendCoda();
+            }
+        } finally {
+            bufferLock.unlock();
+        }
+    }
+
+    public boolean isClosed() {
+        return closed;
+    }
+
+    public void close() {
+        closed = true;
+    }
+
+    public FileSegmentType getFileType() {
+        return fileType;
+    }
+
+    public MessageQueue getMessageQueue() {
+        return messageQueue;
+    }
+
+    public void initPosition(long pos) {
+        this.commitPosition = pos;
+        this.appendPosition = pos;
+    }
+
+    private List<ByteBuffer> rollingUploadBuffer() {
+        bufferLock.lock();
+        try {
+            List<ByteBuffer> tmp = uploadBufferList;
+            uploadBufferList = new ArrayList<>();
+            return tmp;
+        } finally {
+            bufferLock.unlock();
+        }
+    }
+
+    private void sendBackBuffer(TieredFileSegmentInputStream inputStream) {
+        bufferLock.lock();
+        try {
+            List<ByteBuffer> tmpBufferList = inputStream.getUploadBufferList();
+            for (ByteBuffer buffer : tmpBufferList) {
+                buffer.rewind();
+            }
+            tmpBufferList.addAll(uploadBufferList);
+            uploadBufferList = tmpBufferList;
+            if (inputStream.getCodaBuffer() != null) {
+                codaBuffer.rewind();
+            }
+        } finally {
+            bufferLock.unlock();
+        }
+    }
+
+    public AppendResult append(ByteBuffer byteBuf, long timeStamp) {
+        if (closed) {
+            return AppendResult.FILE_CLOSE;
+        }
+        bufferLock.lock();
+        try {
+            if (full || codaBuffer != null) {
+                return AppendResult.FILE_FULL;
+            }
+
+            if (fileType == FileSegmentType.INDEX) {
+                beginTimestamp = byteBuf.getLong(TieredIndexFile.INDEX_FILE_HEADER_BEGIN_TIME_STAMP_POSITION);
+                endTimestamp = byteBuf.getLong(TieredIndexFile.INDEX_FILE_HEADER_END_TIME_STAMP_POSITION);
+                appendPosition += byteBuf.remaining();
+                uploadBufferList.add(byteBuf);
+                setFull();
+                return AppendResult.SUCCESS;
+            }
+
+            if (appendPosition + byteBuf.remaining() > maxSize) {
+                setFull();
+                return AppendResult.FILE_FULL;
+            }
+            if (uploadBufferList.size() > storeConfig.getTieredStoreGroupCommitCount()
+                || appendPosition - commitPosition > storeConfig.getTieredStoreGroupCommitSize()) {
+                commitAsync();
+            }
+            if (uploadBufferList.size() > storeConfig.getTieredStoreMaxGroupCommitCount()) {
+                LOGGER.debug("TieredFileSegment#append: buffer full: file: {}, upload buffer size: {}",
+                    getPath(), uploadBufferList.size());
+                return AppendResult.BUFFER_FULL;
+            }
+            if (timeStamp != Long.MAX_VALUE) {
+                endTimestamp = timeStamp;
+                if (beginTimestamp == Long.MAX_VALUE) {
+                    beginTimestamp = timeStamp;
+                }
+            }
+            appendPosition += byteBuf.remaining();
+            uploadBufferList.add(byteBuf);
+            return AppendResult.SUCCESS;
+        } finally {
+            bufferLock.unlock();
+        }
+    }
+
+    private void appendCoda() {
+        if (codaBuffer != null) {
+            return;
+        }
+        codaBuffer = ByteBuffer.allocate(TieredCommitLog.CODA_SIZE);
+        codaBuffer.putInt(TieredCommitLog.CODA_SIZE);
+        codaBuffer.putInt(TieredCommitLog.BLANK_MAGIC_CODE);
+        codaBuffer.putLong(endTimestamp);
+        codaBuffer.flip();
+        appendPosition += TieredCommitLog.CODA_SIZE;
+    }
+
+    public ByteBuffer read(long position, int length) {
+        return readAsync(position, length).join();
+    }
+
+    public CompletableFuture<ByteBuffer> readAsync(long position, int length) {
+        if (position < 0 || length < 0) {
+            throw new TieredStoreException(TieredStoreErrorCode.ILLEGAL_PARAM, "position or length is negative");

Review Comment:
   The `future` here implies that there may be an exception, could we return `future` directly?



##########
tieredstore/src/main/java/org/apache/rocketmq/store/tiered/container/TieredFileSegment.java:
##########
@@ -0,0 +1,529 @@
+/*
+ * 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.rocketmq.store.tiered.container;
+
+import com.google.common.base.Stopwatch;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.apache.rocketmq.logging.org.slf4j.Logger;
+import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
+import org.apache.rocketmq.store.tiered.common.AppendResult;
+import org.apache.rocketmq.store.tiered.common.TieredMessageStoreConfig;
+import org.apache.rocketmq.store.tiered.exception.TieredStoreErrorCode;
+import org.apache.rocketmq.store.tiered.exception.TieredStoreException;
+import org.apache.rocketmq.store.tiered.util.MessageBufferUtil;
+import org.apache.rocketmq.store.tiered.util.TieredStoreUtil;
+
+public abstract class TieredFileSegment implements Comparable<TieredFileSegment> {
+    private static final Logger LOGGER = LoggerFactory.getLogger(TieredStoreUtil.TIERED_STORE_LOGGER_NAME);
+    private volatile boolean closed = false;
+    private final ReentrantLock bufferLock = new ReentrantLock();
+    private final Semaphore commitLock = new Semaphore(1);
+    private List<ByteBuffer> uploadBufferList = new ArrayList<>();
+    private boolean full;
+    protected final FileSegmentType fileType;
+    protected final MessageQueue messageQueue;
+    protected final TieredMessageStoreConfig storeConfig;
+    protected final long baseOffset;
+    private volatile long commitPosition;
+    private volatile long appendPosition;
+    private final long maxSize;
+    private long beginTimestamp = Long.MAX_VALUE;
+    private long endTimestamp = Long.MAX_VALUE;
+    // only used in commitLog
+    private long commitMsgQueueOffset = 0;
+    private ByteBuffer codaBuffer;
+
+    private CompletableFuture<Boolean> inflightCommitRequest = CompletableFuture.completedFuture(false);
+
+    public TieredFileSegment(FileSegmentType fileType, MessageQueue messageQueue, long baseOffset,
+        TieredMessageStoreConfig storeConfig) {
+        this.fileType = fileType;
+        this.messageQueue = messageQueue;
+        this.storeConfig = storeConfig;
+        this.baseOffset = baseOffset;
+        this.commitPosition = 0;
+        this.appendPosition = 0;
+        switch (fileType) {
+            case COMMIT_LOG:
+                this.maxSize = storeConfig.getTieredStoreCommitLogMaxSize();
+                break;
+            case CONSUME_QUEUE:
+                this.maxSize = storeConfig.getTieredStoreConsumeQueueMaxSize();
+                break;
+            case INDEX:
+                this.maxSize = Long.MAX_VALUE;
+                break;
+            default:
+                throw new IllegalArgumentException("Unsupported file type: " + fileType);
+        }
+    }
+
+    @Override
+    public int compareTo(TieredFileSegment o) {
+        return Long.compare(this.baseOffset, o.baseOffset);
+    }
+
+    public long getBaseOffset() {
+        return baseOffset;
+    }
+
+    public long getCommitOffset() {
+        return baseOffset + commitPosition;
+    }
+
+    public long getCommitPosition() {
+        return commitPosition;
+    }
+
+    public long getCommitMsgQueueOffset() {
+        return commitMsgQueueOffset;
+    }
+
+    public long getMaxOffset() {
+        return baseOffset + appendPosition;
+    }
+
+    public long getMaxSize() {
+        return maxSize;
+    }
+
+    public long getBeginTimestamp() {
+        return beginTimestamp;
+    }
+
+    protected void setBeginTimestamp(long beginTimestamp) {
+        this.beginTimestamp = beginTimestamp;
+    }
+
+    public long getEndTimestamp() {
+        return endTimestamp;
+    }
+
+    protected void setEndTimestamp(long endTimestamp) {
+        this.endTimestamp = endTimestamp;
+    }
+
+    public boolean isFull() {
+        return full;
+    }
+
+    public void setFull() {
+        setFull(true);
+    }
+
+    public void setFull(boolean appendCoda) {
+        bufferLock.lock();
+        try {
+            full = true;
+            if (fileType == FileSegmentType.COMMIT_LOG && appendCoda) {
+                appendCoda();
+            }
+        } finally {
+            bufferLock.unlock();
+        }
+    }
+
+    public boolean isClosed() {
+        return closed;
+    }
+
+    public void close() {
+        closed = true;
+    }
+
+    public FileSegmentType getFileType() {
+        return fileType;
+    }
+
+    public MessageQueue getMessageQueue() {
+        return messageQueue;
+    }
+
+    public void initPosition(long pos) {
+        this.commitPosition = pos;
+        this.appendPosition = pos;
+    }
+
+    private List<ByteBuffer> rollingUploadBuffer() {
+        bufferLock.lock();
+        try {
+            List<ByteBuffer> tmp = uploadBufferList;
+            uploadBufferList = new ArrayList<>();
+            return tmp;
+        } finally {
+            bufferLock.unlock();
+        }
+    }
+
+    private void sendBackBuffer(TieredFileSegmentInputStream inputStream) {
+        bufferLock.lock();
+        try {
+            List<ByteBuffer> tmpBufferList = inputStream.getUploadBufferList();
+            for (ByteBuffer buffer : tmpBufferList) {
+                buffer.rewind();
+            }
+            tmpBufferList.addAll(uploadBufferList);
+            uploadBufferList = tmpBufferList;
+            if (inputStream.getCodaBuffer() != null) {
+                codaBuffer.rewind();
+            }
+        } finally {
+            bufferLock.unlock();
+        }
+    }
+
+    public AppendResult append(ByteBuffer byteBuf, long timeStamp) {
+        if (closed) {
+            return AppendResult.FILE_CLOSE;
+        }
+        bufferLock.lock();
+        try {
+            if (full || codaBuffer != null) {
+                return AppendResult.FILE_FULL;
+            }
+
+            if (fileType == FileSegmentType.INDEX) {
+                beginTimestamp = byteBuf.getLong(TieredIndexFile.INDEX_FILE_HEADER_BEGIN_TIME_STAMP_POSITION);
+                endTimestamp = byteBuf.getLong(TieredIndexFile.INDEX_FILE_HEADER_END_TIME_STAMP_POSITION);
+                appendPosition += byteBuf.remaining();
+                uploadBufferList.add(byteBuf);
+                setFull();
+                return AppendResult.SUCCESS;
+            }
+
+            if (appendPosition + byteBuf.remaining() > maxSize) {
+                setFull();
+                return AppendResult.FILE_FULL;
+            }
+            if (uploadBufferList.size() > storeConfig.getTieredStoreGroupCommitCount()
+                || appendPosition - commitPosition > storeConfig.getTieredStoreGroupCommitSize()) {
+                commitAsync();
+            }
+            if (uploadBufferList.size() > storeConfig.getTieredStoreMaxGroupCommitCount()) {
+                LOGGER.debug("TieredFileSegment#append: buffer full: file: {}, upload buffer size: {}",
+                    getPath(), uploadBufferList.size());
+                return AppendResult.BUFFER_FULL;
+            }
+            if (timeStamp != Long.MAX_VALUE) {
+                endTimestamp = timeStamp;
+                if (beginTimestamp == Long.MAX_VALUE) {
+                    beginTimestamp = timeStamp;
+                }
+            }
+            appendPosition += byteBuf.remaining();
+            uploadBufferList.add(byteBuf);
+            return AppendResult.SUCCESS;
+        } finally {
+            bufferLock.unlock();
+        }
+    }
+
+    private void appendCoda() {
+        if (codaBuffer != null) {
+            return;
+        }
+        codaBuffer = ByteBuffer.allocate(TieredCommitLog.CODA_SIZE);
+        codaBuffer.putInt(TieredCommitLog.CODA_SIZE);
+        codaBuffer.putInt(TieredCommitLog.BLANK_MAGIC_CODE);
+        codaBuffer.putLong(endTimestamp);
+        codaBuffer.flip();
+        appendPosition += TieredCommitLog.CODA_SIZE;
+    }
+
+    public ByteBuffer read(long position, int length) {
+        return readAsync(position, length).join();
+    }
+
+    public CompletableFuture<ByteBuffer> readAsync(long position, int length) {
+        if (position < 0 || length < 0) {
+            throw new TieredStoreException(TieredStoreErrorCode.ILLEGAL_PARAM, "position or length is negative");
+        }
+        if (length == 0) {
+            throw new TieredStoreException(TieredStoreErrorCode.ILLEGAL_PARAM, "length is zero");
+        }
+        if (position + length > commitPosition) {
+            LOGGER.warn("TieredFileSegment#readAsync request position + length is greater than commit position," +
+                    " correct length using commit position, file: {}, request position: {}, commit position:{}, change length from {} to {}",
+                getPath(), position, commitPosition, length, commitPosition - position);
+            length = (int) (commitPosition - position);
+            if (length == 0) {
+                throw new TieredStoreException(TieredStoreErrorCode.NO_NEW_DATA, "request position is equal to commit position");
+            }
+            if (fileType == FileSegmentType.CONSUME_QUEUE && length % TieredConsumeQueue.CONSUME_QUEUE_STORE_UNIT_SIZE != 0) {
+                throw new TieredStoreException(TieredStoreErrorCode.ILLEGAL_PARAM, "position and length is illegal");
+            }
+        }
+        return read0(position, length);
+    }
+
+    public boolean needCommit() {
+        return appendPosition > commitPosition;
+    }
+
+    public boolean commit() {
+        if (closed) {
+            return false;
+        }
+        Boolean result = commitAsync().join();
+        if (!result) {
+            result = inflightCommitRequest.join();
+        }
+        return result;
+    }
+
+    public CompletableFuture<Boolean> commitAsync() {
+        if (closed) {
+            return CompletableFuture.completedFuture(false);
+        }
+        Stopwatch stopwatch = Stopwatch.createStarted();
+        if (!needCommit()) {
+            return CompletableFuture.completedFuture(true);
+        }
+        try {
+            int permits = commitLock.drainPermits();
+            if (permits <= 0) {
+                return CompletableFuture.completedFuture(false);
+            }
+        } catch (Exception e) {
+            return CompletableFuture.completedFuture(false);
+        }
+        List<ByteBuffer> bufferList = rollingUploadBuffer();
+        int bufferSize = 0;
+        for (ByteBuffer buffer : bufferList) {
+            bufferSize += buffer.remaining();
+        }
+        if (codaBuffer != null) {
+            bufferSize += codaBuffer.remaining();
+        }
+        if (bufferSize == 0) {
+            return CompletableFuture.completedFuture(true);
+        }
+        TieredFileSegmentInputStream inputStream = new TieredFileSegmentInputStream(fileType, baseOffset + commitPosition, bufferList, codaBuffer, bufferSize);
+        int finalBufferSize = bufferSize;
+        try {
+            inflightCommitRequest = commit0(inputStream, commitPosition, bufferSize, fileType != FileSegmentType.INDEX)
+                .thenApply(result -> {
+                    if (result) {
+                        if (fileType == FileSegmentType.COMMIT_LOG && bufferList.size() > 0) {
+                            commitMsgQueueOffset = MessageBufferUtil.getQueueOffset(bufferList.get(bufferList.size() - 1));
+                        }
+                        commitPosition += finalBufferSize;
+                        return true;
+                    }
+                    sendBackBuffer(inputStream);
+                    return false;
+                }).exceptionally(e -> handleCommitException(inputStream, e))
+                .whenComplete((result, e) -> {
+                    if (commitLock.availablePermits() == 0) {
+                        LOGGER.debug("TieredFileSegment#commitAsync: commit cost: {}ms, file: {}, item count: {}, buffer size: {}", stopwatch.elapsed(TimeUnit.MILLISECONDS), getPath(), bufferList.size(), finalBufferSize);
+                        commitLock.release();
+                    } else {
+                        LOGGER.error("[Bug]TieredFileSegment#commitAsync: commit lock is already released: available permits: {}", commitLock.availablePermits());
+                    }
+                });
+            return inflightCommitRequest;
+        } catch (Exception e) {
+            handleCommitException(inputStream, e);
+            if (commitLock.availablePermits() == 0) {
+                LOGGER.debug("TieredFileSegment#commitAsync: commit cost: {}ms, file: {}, item count: {}, buffer size: {}", stopwatch.elapsed(TimeUnit.MILLISECONDS), getPath(), bufferList.size(), finalBufferSize);
+                commitLock.release();
+            } else {
+                LOGGER.error("[Bug]TieredFileSegment#commitAsync: commit lock is already released: available permits: {}", commitLock.availablePermits());
+            }
+        }
+        return CompletableFuture.completedFuture(false);
+    }
+
+    private boolean handleCommitException(TieredFileSegmentInputStream inputStream, Throwable e) {
+        Throwable cause = e.getCause() != null ? e.getCause() : e;
+        sendBackBuffer(inputStream);
+        long realSize = 0;
+        if (cause instanceof TieredStoreException && ((TieredStoreException) cause).getPosition() > 0) {
+            realSize = ((TieredStoreException) cause).getPosition();
+        }
+        if (realSize <= 0) {
+            realSize = getSize();
+        }
+        if (realSize > 0 && realSize > commitPosition) {
+            LOGGER.error("TieredFileSegment#handleCommitException: commit failed: file: {}, try to fix position: origin: {}, real: {}", getPath(), commitPosition, realSize, cause);
+            // TODO check if this diff part is uploaded to OSS

Review Comment:
   The `check` here could be an independent method or we could implement it in `#commit0`.



##########
tieredstore/src/main/java/org/apache/rocketmq/store/tiered/container/TieredFileSegment.java:
##########
@@ -0,0 +1,529 @@
+/*
+ * 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.rocketmq.store.tiered.container;
+
+import com.google.common.base.Stopwatch;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.apache.rocketmq.logging.org.slf4j.Logger;
+import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
+import org.apache.rocketmq.store.tiered.common.AppendResult;
+import org.apache.rocketmq.store.tiered.common.TieredMessageStoreConfig;
+import org.apache.rocketmq.store.tiered.exception.TieredStoreErrorCode;
+import org.apache.rocketmq.store.tiered.exception.TieredStoreException;
+import org.apache.rocketmq.store.tiered.util.MessageBufferUtil;
+import org.apache.rocketmq.store.tiered.util.TieredStoreUtil;
+
+public abstract class TieredFileSegment implements Comparable<TieredFileSegment> {
+    private static final Logger LOGGER = LoggerFactory.getLogger(TieredStoreUtil.TIERED_STORE_LOGGER_NAME);
+    private volatile boolean closed = false;
+    private final ReentrantLock bufferLock = new ReentrantLock();
+    private final Semaphore commitLock = new Semaphore(1);
+    private List<ByteBuffer> uploadBufferList = new ArrayList<>();
+    private boolean full;
+    protected final FileSegmentType fileType;
+    protected final MessageQueue messageQueue;
+    protected final TieredMessageStoreConfig storeConfig;
+    protected final long baseOffset;
+    private volatile long commitPosition;
+    private volatile long appendPosition;
+    private final long maxSize;
+    private long beginTimestamp = Long.MAX_VALUE;
+    private long endTimestamp = Long.MAX_VALUE;
+    // only used in commitLog
+    private long commitMsgQueueOffset = 0;
+    private ByteBuffer codaBuffer;
+
+    private CompletableFuture<Boolean> inflightCommitRequest = CompletableFuture.completedFuture(false);
+
+    public TieredFileSegment(FileSegmentType fileType, MessageQueue messageQueue, long baseOffset,
+        TieredMessageStoreConfig storeConfig) {
+        this.fileType = fileType;
+        this.messageQueue = messageQueue;
+        this.storeConfig = storeConfig;
+        this.baseOffset = baseOffset;
+        this.commitPosition = 0;
+        this.appendPosition = 0;
+        switch (fileType) {
+            case COMMIT_LOG:
+                this.maxSize = storeConfig.getTieredStoreCommitLogMaxSize();
+                break;
+            case CONSUME_QUEUE:
+                this.maxSize = storeConfig.getTieredStoreConsumeQueueMaxSize();
+                break;
+            case INDEX:
+                this.maxSize = Long.MAX_VALUE;
+                break;
+            default:
+                throw new IllegalArgumentException("Unsupported file type: " + fileType);
+        }
+    }
+
+    @Override
+    public int compareTo(TieredFileSegment o) {
+        return Long.compare(this.baseOffset, o.baseOffset);
+    }
+
+    public long getBaseOffset() {
+        return baseOffset;
+    }
+
+    public long getCommitOffset() {
+        return baseOffset + commitPosition;
+    }
+
+    public long getCommitPosition() {
+        return commitPosition;
+    }
+
+    public long getCommitMsgQueueOffset() {
+        return commitMsgQueueOffset;
+    }
+
+    public long getMaxOffset() {
+        return baseOffset + appendPosition;
+    }
+
+    public long getMaxSize() {
+        return maxSize;
+    }
+
+    public long getBeginTimestamp() {
+        return beginTimestamp;
+    }
+
+    protected void setBeginTimestamp(long beginTimestamp) {
+        this.beginTimestamp = beginTimestamp;
+    }
+
+    public long getEndTimestamp() {
+        return endTimestamp;
+    }
+
+    protected void setEndTimestamp(long endTimestamp) {
+        this.endTimestamp = endTimestamp;
+    }
+
+    public boolean isFull() {
+        return full;
+    }
+
+    public void setFull() {
+        setFull(true);
+    }
+
+    public void setFull(boolean appendCoda) {
+        bufferLock.lock();
+        try {
+            full = true;
+            if (fileType == FileSegmentType.COMMIT_LOG && appendCoda) {
+                appendCoda();
+            }
+        } finally {
+            bufferLock.unlock();
+        }
+    }
+
+    public boolean isClosed() {
+        return closed;
+    }
+
+    public void close() {
+        closed = true;
+    }
+
+    public FileSegmentType getFileType() {
+        return fileType;
+    }
+
+    public MessageQueue getMessageQueue() {
+        return messageQueue;
+    }
+
+    public void initPosition(long pos) {
+        this.commitPosition = pos;
+        this.appendPosition = pos;
+    }
+
+    private List<ByteBuffer> rollingUploadBuffer() {
+        bufferLock.lock();
+        try {
+            List<ByteBuffer> tmp = uploadBufferList;
+            uploadBufferList = new ArrayList<>();
+            return tmp;
+        } finally {
+            bufferLock.unlock();
+        }
+    }
+
+    private void sendBackBuffer(TieredFileSegmentInputStream inputStream) {
+        bufferLock.lock();
+        try {
+            List<ByteBuffer> tmpBufferList = inputStream.getUploadBufferList();
+            for (ByteBuffer buffer : tmpBufferList) {
+                buffer.rewind();
+            }
+            tmpBufferList.addAll(uploadBufferList);
+            uploadBufferList = tmpBufferList;
+            if (inputStream.getCodaBuffer() != null) {
+                codaBuffer.rewind();
+            }
+        } finally {
+            bufferLock.unlock();
+        }
+    }
+
+    public AppendResult append(ByteBuffer byteBuf, long timeStamp) {
+        if (closed) {
+            return AppendResult.FILE_CLOSE;
+        }
+        bufferLock.lock();
+        try {
+            if (full || codaBuffer != null) {
+                return AppendResult.FILE_FULL;
+            }
+
+            if (fileType == FileSegmentType.INDEX) {
+                beginTimestamp = byteBuf.getLong(TieredIndexFile.INDEX_FILE_HEADER_BEGIN_TIME_STAMP_POSITION);
+                endTimestamp = byteBuf.getLong(TieredIndexFile.INDEX_FILE_HEADER_END_TIME_STAMP_POSITION);
+                appendPosition += byteBuf.remaining();
+                uploadBufferList.add(byteBuf);
+                setFull();
+                return AppendResult.SUCCESS;
+            }
+
+            if (appendPosition + byteBuf.remaining() > maxSize) {
+                setFull();
+                return AppendResult.FILE_FULL;
+            }
+            if (uploadBufferList.size() > storeConfig.getTieredStoreGroupCommitCount()
+                || appendPosition - commitPosition > storeConfig.getTieredStoreGroupCommitSize()) {
+                commitAsync();
+            }
+            if (uploadBufferList.size() > storeConfig.getTieredStoreMaxGroupCommitCount()) {
+                LOGGER.debug("TieredFileSegment#append: buffer full: file: {}, upload buffer size: {}",
+                    getPath(), uploadBufferList.size());
+                return AppendResult.BUFFER_FULL;
+            }
+            if (timeStamp != Long.MAX_VALUE) {
+                endTimestamp = timeStamp;
+                if (beginTimestamp == Long.MAX_VALUE) {
+                    beginTimestamp = timeStamp;
+                }
+            }
+            appendPosition += byteBuf.remaining();
+            uploadBufferList.add(byteBuf);
+            return AppendResult.SUCCESS;
+        } finally {
+            bufferLock.unlock();
+        }
+    }
+
+    private void appendCoda() {
+        if (codaBuffer != null) {
+            return;
+        }
+        codaBuffer = ByteBuffer.allocate(TieredCommitLog.CODA_SIZE);
+        codaBuffer.putInt(TieredCommitLog.CODA_SIZE);
+        codaBuffer.putInt(TieredCommitLog.BLANK_MAGIC_CODE);
+        codaBuffer.putLong(endTimestamp);
+        codaBuffer.flip();
+        appendPosition += TieredCommitLog.CODA_SIZE;
+    }
+
+    public ByteBuffer read(long position, int length) {
+        return readAsync(position, length).join();
+    }
+
+    public CompletableFuture<ByteBuffer> readAsync(long position, int length) {
+        if (position < 0 || length < 0) {
+            throw new TieredStoreException(TieredStoreErrorCode.ILLEGAL_PARAM, "position or length is negative");
+        }
+        if (length == 0) {
+            throw new TieredStoreException(TieredStoreErrorCode.ILLEGAL_PARAM, "length is zero");
+        }
+        if (position + length > commitPosition) {
+            LOGGER.warn("TieredFileSegment#readAsync request position + length is greater than commit position," +
+                    " correct length using commit position, file: {}, request position: {}, commit position:{}, change length from {} to {}",
+                getPath(), position, commitPosition, length, commitPosition - position);
+            length = (int) (commitPosition - position);
+            if (length == 0) {
+                throw new TieredStoreException(TieredStoreErrorCode.NO_NEW_DATA, "request position is equal to commit position");
+            }
+            if (fileType == FileSegmentType.CONSUME_QUEUE && length % TieredConsumeQueue.CONSUME_QUEUE_STORE_UNIT_SIZE != 0) {
+                throw new TieredStoreException(TieredStoreErrorCode.ILLEGAL_PARAM, "position and length is illegal");
+            }
+        }
+        return read0(position, length);
+    }
+
+    public boolean needCommit() {
+        return appendPosition > commitPosition;
+    }
+
+    public boolean commit() {
+        if (closed) {
+            return false;
+        }
+        Boolean result = commitAsync().join();
+        if (!result) {
+            result = inflightCommitRequest.join();
+        }
+        return result;
+    }
+
+    public CompletableFuture<Boolean> commitAsync() {
+        if (closed) {
+            return CompletableFuture.completedFuture(false);
+        }
+        Stopwatch stopwatch = Stopwatch.createStarted();
+        if (!needCommit()) {
+            return CompletableFuture.completedFuture(true);
+        }
+        try {
+            int permits = commitLock.drainPermits();
+            if (permits <= 0) {
+                return CompletableFuture.completedFuture(false);
+            }
+        } catch (Exception e) {
+            return CompletableFuture.completedFuture(false);
+        }
+        List<ByteBuffer> bufferList = rollingUploadBuffer();
+        int bufferSize = 0;
+        for (ByteBuffer buffer : bufferList) {
+            bufferSize += buffer.remaining();
+        }
+        if (codaBuffer != null) {
+            bufferSize += codaBuffer.remaining();
+        }
+        if (bufferSize == 0) {
+            return CompletableFuture.completedFuture(true);
+        }
+        TieredFileSegmentInputStream inputStream = new TieredFileSegmentInputStream(fileType, baseOffset + commitPosition, bufferList, codaBuffer, bufferSize);
+        int finalBufferSize = bufferSize;
+        try {
+            inflightCommitRequest = commit0(inputStream, commitPosition, bufferSize, fileType != FileSegmentType.INDEX)
+                .thenApply(result -> {
+                    if (result) {
+                        if (fileType == FileSegmentType.COMMIT_LOG && bufferList.size() > 0) {
+                            commitMsgQueueOffset = MessageBufferUtil.getQueueOffset(bufferList.get(bufferList.size() - 1));
+                        }
+                        commitPosition += finalBufferSize;
+                        return true;
+                    }
+                    sendBackBuffer(inputStream);
+                    return false;
+                }).exceptionally(e -> handleCommitException(inputStream, e))
+                .whenComplete((result, e) -> {
+                    if (commitLock.availablePermits() == 0) {
+                        LOGGER.debug("TieredFileSegment#commitAsync: commit cost: {}ms, file: {}, item count: {}, buffer size: {}", stopwatch.elapsed(TimeUnit.MILLISECONDS), getPath(), bufferList.size(), finalBufferSize);
+                        commitLock.release();
+                    } else {
+                        LOGGER.error("[Bug]TieredFileSegment#commitAsync: commit lock is already released: available permits: {}", commitLock.availablePermits());
+                    }
+                });
+            return inflightCommitRequest;
+        } catch (Exception e) {
+            handleCommitException(inputStream, e);
+            if (commitLock.availablePermits() == 0) {
+                LOGGER.debug("TieredFileSegment#commitAsync: commit cost: {}ms, file: {}, item count: {}, buffer size: {}", stopwatch.elapsed(TimeUnit.MILLISECONDS), getPath(), bufferList.size(), finalBufferSize);
+                commitLock.release();
+            } else {
+                LOGGER.error("[Bug]TieredFileSegment#commitAsync: commit lock is already released: available permits: {}", commitLock.availablePermits());
+            }
+        }
+        return CompletableFuture.completedFuture(false);
+    }
+
+    private boolean handleCommitException(TieredFileSegmentInputStream inputStream, Throwable e) {
+        Throwable cause = e.getCause() != null ? e.getCause() : e;
+        sendBackBuffer(inputStream);
+        long realSize = 0;
+        if (cause instanceof TieredStoreException && ((TieredStoreException) cause).getPosition() > 0) {
+            realSize = ((TieredStoreException) cause).getPosition();
+        }
+        if (realSize <= 0) {
+            realSize = getSize();
+        }
+        if (realSize > 0 && realSize > commitPosition) {
+            LOGGER.error("TieredFileSegment#handleCommitException: commit failed: file: {}, try to fix position: origin: {}, real: {}", getPath(), commitPosition, realSize, cause);
+            // TODO check if this diff part is uploaded to OSS

Review Comment:
   OSS? tiered-storage should not be limited to the specific implementation.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org