-
Notifications
You must be signed in to change notification settings - Fork 539
[client] Refactor RemoteLogDownloader to use chunked file append instead of downloading whole log file #3263
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
Open
swuferhong
wants to merge
1
commit into
apache:main
Choose a base branch
from
swuferhong:remote-log-download-slice
base: main
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,47 +17,69 @@ | |
|
|
||
| package org.apache.fluss.client.table.scanner.log; | ||
|
|
||
| import org.apache.fluss.exception.FlussRuntimeException; | ||
| import org.apache.fluss.record.FileLogRecords; | ||
| import org.apache.fluss.record.LogRecords; | ||
|
|
||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| import java.io.File; | ||
| import java.io.IOException; | ||
| import java.util.concurrent.CompletableFuture; | ||
| import java.util.function.Consumer; | ||
|
|
||
| /** Represents the future of a remote log download request. */ | ||
| /** | ||
| * Represents the future of a single chunk read from a remote log segment. Each chunk is delivered | ||
| * as a {@link LogRecords} via a {@link CompletableFuture}. | ||
| */ | ||
| public class RemoteLogDownloadFuture { | ||
|
|
||
| private final CompletableFuture<File> logFileFuture; | ||
| private static final Logger LOG = LoggerFactory.getLogger(RemoteLogDownloadFuture.class); | ||
|
|
||
| private final CompletableFuture<LogRecords> chunkFuture; | ||
| private final Runnable recycleCallback; | ||
| private Consumer<RemoteLogDownloadFuture> nextChunkCallback; | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. volatile? |
||
|
|
||
| public RemoteLogDownloadFuture( | ||
| CompletableFuture<File> logFileFuture, Runnable recycleCallback) { | ||
| this.logFileFuture = logFileFuture; | ||
| CompletableFuture<LogRecords> chunkFuture, Runnable recycleCallback) { | ||
| this.chunkFuture = chunkFuture; | ||
| this.recycleCallback = recycleCallback; | ||
| } | ||
|
|
||
| public boolean isDone() { | ||
| return logFileFuture.isDone(); | ||
| return chunkFuture.isDone(); | ||
| } | ||
|
|
||
| public FileLogRecords getFileLogRecords(int startPosition) { | ||
| try { | ||
| FileLogRecords fileLogRecords = FileLogRecords.open(logFileFuture.join(), false); | ||
| if (startPosition > 0) { | ||
| return fileLogRecords.slice(startPosition, Integer.MAX_VALUE); | ||
| } else { | ||
| return fileLogRecords; | ||
| } | ||
| } catch (IOException e) { | ||
| throw new FlussRuntimeException(e); | ||
| } | ||
| /** Returns the chunk data. Blocks until the chunk is ready. */ | ||
| public LogRecords getLogRecords() { | ||
| return chunkFuture.join(); | ||
| } | ||
|
|
||
| public Runnable getRecycleCallback() { | ||
| return recycleCallback; | ||
| } | ||
|
|
||
| public void onComplete(Runnable callback) { | ||
| logFileFuture.thenRun(callback); | ||
| // Use whenComplete (instead of thenRun) so the callback fires regardless of whether | ||
| // the chunkFuture completed successfully or exceptionally. This is critical: if a chunk | ||
| // read fails, the LogFetchBuffer still needs tryComplete() to be called so the failed | ||
| // PendingFetch can be drained (otherwise the bucket would be stuck permanently). | ||
| chunkFuture.whenComplete( | ||
| (result, throwable) -> { | ||
| try { | ||
| callback.run(); | ||
| } catch (Throwable t) { | ||
| LOG.error("Exception in chunk completion callback", t); | ||
| } | ||
| }); | ||
| } | ||
|
|
||
| /** | ||
| * Sets a callback that will be invoked when the next chunk's future is created. This allows the | ||
| * {@link LogFetcher} to register a new {@link RemotePendingFetch} for each subsequent chunk. | ||
| */ | ||
| public void setNextChunkCallback(Consumer<RemoteLogDownloadFuture> callback) { | ||
| this.nextChunkCallback = callback; | ||
| } | ||
|
|
||
| public Consumer<RemoteLogDownloadFuture> getNextChunkCallback() { | ||
| return nextChunkCallback; | ||
| } | ||
| } | ||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: I'm not sure: what if the issue is permanent? It would be a infitite loop with only warning. Should we have retry with backoff and then surface an error in the scanner?