Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[ISSUE #8334] fix: irreducible reference count of mapped file caused by held tiered storage's dispatch #8335

Open
wants to merge 3 commits into
base: develop
Choose a base branch
from
Open
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
Original file line number Diff line number Diff line change
Expand Up @@ -184,27 +184,35 @@ public CompletableFuture<Boolean> doScheduleDispatch(FlatFileInterface flatFile,

ConsumeQueueInterface consumeQueue = defaultStore.getConsumeQueue(topic, queueId);
CqUnit cqUnit = consumeQueue.get(currentOffset);
SelectMappedBufferResult message =
defaultStore.selectOneMessageByOffset(cqUnit.getPos(), cqUnit.getSize());
boolean timeout = MessageFormatUtil.getStoreTimeStamp(message.getByteBuffer()) +
storeConfig.getTieredStoreGroupCommitTimeout() < System.currentTimeMillis();
boolean bufferFull = maxOffsetInQueue - currentOffset > storeConfig.getTieredStoreGroupCommitCount();

if (!timeout && !bufferFull && !force) {
log.debug("MessageDispatcher#dispatch hold, topic={}, queueId={}, offset={}-{}, current={}, remain={}",
topic, queueId, minOffsetInQueue, maxOffsetInQueue, currentOffset, maxOffsetInQueue - currentOffset);
return CompletableFuture.completedFuture(false);
} else {
if (MessageFormatUtil.getStoreTimeStamp(message.getByteBuffer()) +
TimeUnit.MINUTES.toMillis(5) < System.currentTimeMillis()) {
log.warn("MessageDispatcher#dispatch behind too much, topic={}, queueId={}, offset={}-{}, current={}, remain={}",
topic, queueId, minOffsetInQueue, maxOffsetInQueue, currentOffset, maxOffsetInQueue - currentOffset);

SelectMappedBufferResult message = null;
boolean timeout;
boolean bufferFull;
try {
message = defaultStore.selectOneMessageByOffset(cqUnit.getPos(), cqUnit.getSize());
timeout = MessageFormatUtil.getStoreTimeStamp(message.getByteBuffer()) +
storeConfig.getTieredStoreGroupCommitTimeout() < System.currentTimeMillis();
bufferFull = maxOffsetInQueue - currentOffset > storeConfig.getTieredStoreGroupCommitCount();

if (!timeout && !bufferFull && !force) {
log.debug("MessageDispatcher#dispatch hold, topic={}, queueId={}, offset={}-{}, current={}, remain={}",
topic, queueId, minOffsetInQueue, maxOffsetInQueue, currentOffset, maxOffsetInQueue - currentOffset);
return CompletableFuture.completedFuture(false);
} else {
log.info("MessageDispatcher#dispatch, topic={}, queueId={}, offset={}-{}, current={}, remain={}",
topic, queueId, minOffsetInQueue, maxOffsetInQueue, currentOffset, maxOffsetInQueue - currentOffset);
if (MessageFormatUtil.getStoreTimeStamp(message.getByteBuffer()) +
TimeUnit.MINUTES.toMillis(5) < System.currentTimeMillis()) {
log.warn("MessageDispatcher#dispatch behind too much, topic={}, queueId={}, offset={}-{}, current={}, remain={}",
topic, queueId, minOffsetInQueue, maxOffsetInQueue, currentOffset, maxOffsetInQueue - currentOffset);
} else {
log.info("MessageDispatcher#dispatch, topic={}, queueId={}, offset={}-{}, current={}, remain={}",
topic, queueId, minOffsetInQueue, maxOffsetInQueue, currentOffset, maxOffsetInQueue - currentOffset);
}
}
} finally {
if (message != null) {
message.release();
}
}
message.release();

long offset = currentOffset;
for (; offset < targetOffset; offset++) {
Expand All @@ -218,6 +226,7 @@ public CompletableFuture<Boolean> doScheduleDispatch(FlatFileInterface flatFile,
ByteBuffer byteBuffer = message.getByteBuffer();
AppendResult result = flatFile.appendCommitLog(message);
if (!AppendResult.SUCCESS.equals(result)) {
message.release();
break;
}

Expand All @@ -233,8 +242,10 @@ public CompletableFuture<Boolean> doScheduleDispatch(FlatFileInterface flatFile,

result = flatFile.appendConsumeQueue(dispatchRequest);
if (!AppendResult.SUCCESS.equals(result)) {
message.release();
break;
}
flatFile.addMessageToBufferList(message);
}

// If there are many messages waiting to be uploaded, call the upload logic immediately.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -80,6 +80,8 @@ public interface FlatFileInterface {

long getConsumeQueueCommitOffset();

void addMessageToBufferList(SelectMappedBufferResult bufferResult);

/**
* Persist commit log file and consume queue file
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -143,6 +143,11 @@ public void initOffset(long offset) {
}
}

@Override
public void addMessageToBufferList(SelectMappedBufferResult message) {
this.bufferResultList.add(message);
}

@Override
public AppendResult appendCommitLog(ByteBuffer message) {
if (closed) {
Expand All @@ -156,7 +161,6 @@ public AppendResult appendCommitLog(SelectMappedBufferResult message) {
if (closed) {
return AppendResult.FILE_CLOSED;
}
this.bufferResultList.add(message);
return this.appendCommitLog(message.getByteBuffer());
}

Expand Down
Loading