Skip to content

Commit

Permalink
feat(wal):reduce concurrent conflicts between block write operations …
Browse files Browse the repository at this point in the history
…and poll operations (AutoMQ#1550)
  • Loading branch information
CLFutureX committed Jul 12, 2024
1 parent b46383f commit 0847d72
Show file tree
Hide file tree
Showing 2 changed files with 21 additions and 11 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ public class BlockBatch {
private final Collection<Block> blocks;
private final long startOffset;
private final long endOffset;
private final long blockBatchSize;

public BlockBatch(Collection<Block> blocks) {
assert !blocks.isEmpty();
Expand All @@ -34,9 +33,6 @@ public BlockBatch(Collection<Block> blocks) {
.map(b -> b.startOffset() + b.size())
.max(Long::compareTo)
.orElseThrow();
this.blockBatchSize = blocks.stream()
.mapToLong(Block::size)
.sum();
}

public long startOffset() {
Expand All @@ -51,10 +47,6 @@ public Collection<Block> blocks() {
return Collections.unmodifiableCollection(blocks);
}

public long blockBatchSize(){
return blockBatchSize;
}

public Iterator<CompletableFuture<AppendResult.CallbackResult>> futures() {
return new Iterator<>() {
private final Iterator<Block> blockIterator = blocks.iterator();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,10 +23,10 @@
import java.util.Collection;
import java.util.LinkedList;
import java.util.Queue;
import java.util.PriorityQueue;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.PriorityBlockingQueue;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
Expand Down Expand Up @@ -71,7 +71,7 @@ public class SlidingWindowService {
/**
* Blocks that are being written.
*/
private final Queue<Long> writingBlocks = new PriorityBlockingQueue<>();
private final Queue<Long> writingBlocks = new PriorityQueue<>();
/**
* Whether the service is initialized.
* After the service is initialized, data in {@link #windowCoreData} is valid.
Expand Down Expand Up @@ -106,6 +106,11 @@ public class SlidingWindowService {
*/
private volatile long lastWriteTimeNanos = 0;

/**
* The maximum offset currently written into writeBlocks.*
*/
private long maxWriteBlockOffset = 0;

public SlidingWindowService(WALChannel walChannel, int ioThreadNums, long upperLimit, long scaleUnit,
long blockSoftLimit, int writeRateLimit, WALHeaderFlusher flusher) {
this.walChannel = walChannel;
Expand Down Expand Up @@ -323,6 +328,7 @@ private BlockBatch pollBlocksLocked() {

BlockBatch blockBatch = new BlockBatch(blocks);
writingBlocks.add(blockBatch.startOffset());
maxWriteBlockOffset = blockBatch.endOffset();

return blockBatch;
}
Expand All @@ -331,10 +337,22 @@ private BlockBatch pollBlocksLocked() {
* Finish the given block batch, and return the start offset of the first block which has not been flushed yet.
*/
private long wroteBlocks(BlockBatch wroteBlocks) {
this.pollBlockLock.lock();
try {
return wroteBlocksLocked(wroteBlocks);
} finally {
this.pollBlockLock.unlock();
}
}
/**
* Finish the given block batch, and return the start offset of the first block which has not been flushed yet.
* Note: this method is NOT thread safe, and it should be called with {@link #pollBlockLock} locked.
*/
private long wroteBlocksLocked(BlockBatch wroteBlocks) {
boolean removed = writingBlocks.remove(wroteBlocks.startOffset());
assert removed;
if (writingBlocks.isEmpty()) {
return wroteBlocks.startOffset() + WALUtil.alignLargeByBlockSize(wroteBlocks.blockBatchSize());
return this.maxWriteBlockOffset;
}
return writingBlocks.peek();
}
Expand Down

0 comments on commit 0847d72

Please sign in to comment.