Skip to content

Commit

Permalink
fix(s3stream): confirm requests after putting them into logCache (#961)
Browse files Browse the repository at this point in the history
Signed-off-by: Ning Yu <[email protected]>
  • Loading branch information
Chillax-0v0 authored Mar 13, 2024
1 parent ebf8257 commit 9182360
Show file tree
Hide file tree
Showing 3 changed files with 26 additions and 7 deletions.
8 changes: 5 additions & 3 deletions s3stream/src/main/java/com/automq/stream/s3/S3Storage.java
Original file line number Diff line number Diff line change
Expand Up @@ -545,7 +545,9 @@ private void handleAppendCallback0(WalWriteRequest request) {
waitingAckRequests = callbackSequencer.after(request);
waitingAckRequests.forEach(r -> r.record.retain());
for (WalWriteRequest waitingAckRequest : waitingAckRequests) {
if (deltaWALCache.put(waitingAckRequest.record)) {
boolean full = deltaWALCache.put(waitingAckRequest.record);
waitingAckRequest.confirmed = true;
if (full) {
// cache block is full, trigger WAL upload.
uploadDeltaWAL();
}
Expand Down Expand Up @@ -766,7 +768,7 @@ synchronized private long calculate() {
}
WalWriteRequest request = wrapper.request;
assert request.offset != NOOP_OFFSET;
if (!request.persisted) {
if (!request.confirmed) {
minUnconfirmedOffset = Math.min(minUnconfirmedOffset, request.offset);
}
}
Expand All @@ -783,7 +785,7 @@ synchronized private long calculate() {
break;
}
WalWriteRequest request = wrapper.request;
if (request.persisted && request.offset < minUnconfirmedOffset) {
if (request.confirmed && request.offset < minUnconfirmedOffset) {
confirmedOffset = Math.max(confirmedOffset, request.offset);
iterator.remove();
}
Expand Down
17 changes: 17 additions & 0 deletions s3stream/src/main/java/com/automq/stream/s3/WalWriteRequest.java
Original file line number Diff line number Diff line change
Expand Up @@ -11,17 +11,33 @@

package com.automq.stream.s3;

import com.automq.stream.s3.cache.LogCache;
import com.automq.stream.s3.context.AppendContext;
import com.automq.stream.s3.model.StreamRecordBatch;
import com.automq.stream.s3.wal.WriteAheadLog;
import java.util.concurrent.CompletableFuture;

public class WalWriteRequest implements Comparable<WalWriteRequest> {
final StreamRecordBatch record;
final AppendContext context;
final CompletableFuture<Void> cf;
long offset;
/**
* Whether the record has been persisted to the {@link WriteAheadLog}
* When a continuous series of records IN A STREAM have been persisted to the WAL, they can be uploaded to S3.
*
* @see S3Storage.WALCallbackSequencer
*/
boolean persisted;

/**
* Whether the record has been put to the {@link LogCache}
* When a continuous series of records have been persisted to the WAL and uploaded to S3, they can be trimmed.
*
* @see S3Storage.WALConfirmOffsetCalculator
*/
boolean confirmed;

public WalWriteRequest(StreamRecordBatch record, long offset, CompletableFuture<Void> cf) {
this(record, offset, cf, AppendContext.DEFAULT);
}
Expand All @@ -44,6 +60,7 @@ public String toString() {
"record=" + record +
", offset=" + offset +
", persisted=" + persisted +
", confirmed=" + confirmed +
'}';
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -135,19 +135,19 @@ public void testWALConfirmOffsetCalculator() {
calc.update();
assertEquals(-1L, calc.get());

r0.persisted = true;
r0.confirmed = true;
calc.update();
assertEquals(0L, calc.get());

r3.persisted = true;
r3.confirmed = true;
calc.update();
assertEquals(0L, calc.get());

r1.persisted = true;
r1.confirmed = true;
calc.update();
assertEquals(1L, calc.get());

r2.persisted = true;
r2.confirmed = true;
calc.update();
assertEquals(3L, calc.get());
}
Expand Down

0 comments on commit 9182360

Please sign in to comment.