-
Notifications
You must be signed in to change notification settings - Fork 1.7k
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
[Kernel][LogReplay] Make a single read request for all checkpoint files #2701
Merged
Merged
Changes from all commits
Commits
Show all changes
5 commits
Select commit
Hold shift + click to select a range
83c9e69
[Kernel][LogReplay] Make a single read request for all checkpoint files
vkorukanti e586ef3
review
vkorukanti e7c82aa
update/fix comments
vkorukanti 93682e5
review
vkorukanti 121a928
Merge branch 'master' into multiFileReadRequest
vkorukanti 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 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 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 |
---|---|---|
|
@@ -18,20 +18,21 @@ | |
|
||
import java.io.IOException; | ||
import java.io.UncheckedIOException; | ||
import java.util.Iterator; | ||
import java.util.List; | ||
import java.util.NoSuchElementException; | ||
import java.util.Optional; | ||
import java.util.*; | ||
|
||
import io.delta.kernel.client.TableClient; | ||
import io.delta.kernel.data.ColumnarBatch; | ||
import io.delta.kernel.types.StructType; | ||
import io.delta.kernel.utils.CloseableIterator; | ||
import io.delta.kernel.utils.FileStatus; | ||
|
||
import io.delta.kernel.internal.fs.Path; | ||
import io.delta.kernel.internal.util.FileNames; | ||
import io.delta.kernel.internal.util.Utils; | ||
import static io.delta.kernel.internal.fs.Path.getName; | ||
import static io.delta.kernel.internal.util.FileNames.*; | ||
import static io.delta.kernel.internal.util.Utils.singletonCloseableIterator; | ||
import static io.delta.kernel.internal.util.Utils.toCloseableIterator; | ||
|
||
/** | ||
* This class takes as input a list of delta files (.json, .checkpoint.parquet) and produces an | ||
|
@@ -45,17 +46,19 @@ class ActionsIterator implements CloseableIterator<ActionWrapper> { | |
private final TableClient tableClient; | ||
|
||
/** | ||
* Iterator over the files. | ||
* Linked list of iterator files (commit files and/or checkpoint files) | ||
* {@link LinkedList} to allow removing the head of the list and also to peek at the head | ||
* of the list. The {@link Iterator} doesn't provide a way to peek. | ||
* <p> | ||
* Each file will be split (by 1, or more) to yield an iterator of FileDataReadResults. | ||
* Each of these files return an iterator of {@link ColumnarBatch} containing the actions | ||
*/ | ||
private final Iterator<FileStatus> filesIter; | ||
private final LinkedList<FileStatus> filesList; | ||
tdas marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
||
private final StructType readSchema; | ||
|
||
/** | ||
* The current (ColumnarBatch, isFromCheckpoint) tuple. Whenever this iterator | ||
* is exhausted, we will try and fetch the next one from the `filesIter`. | ||
* is exhausted, we will try and fetch the next one from the `filesList`. | ||
* <p> | ||
* If it is ever empty, that means there are no more batches to produce. | ||
*/ | ||
|
@@ -68,7 +71,8 @@ class ActionsIterator implements CloseableIterator<ActionWrapper> { | |
List<FileStatus> files, | ||
StructType readSchema) { | ||
this.tableClient = tableClient; | ||
this.filesIter = files.iterator(); | ||
this.filesList = new LinkedList<>(); | ||
this.filesList.addAll(files); | ||
this.readSchema = readSchema; | ||
this.actionsIter = Optional.empty(); | ||
} | ||
|
@@ -115,7 +119,7 @@ public void close() throws IOException { | |
|
||
/** | ||
* If the current `actionsIter` has no more elements, this function finds the next | ||
* non-empty file in `filesIter` and uses it to set `actionsIter`. | ||
* non-empty file in `filesList` and uses it to set `actionsIter`. | ||
*/ | ||
private void tryEnsureNextActionsIterIsReady() { | ||
if (actionsIter.isPresent()) { | ||
|
@@ -132,7 +136,7 @@ private void tryEnsureNextActionsIterIsReady() { | |
} | ||
|
||
// Search for the next non-empty file and use that iter | ||
while (filesIter.hasNext()) { | ||
while (!filesList.isEmpty()) { | ||
actionsIter = Optional.of(getNextActionsIter()); | ||
|
||
if (actionsIter.get().hasNext()) { | ||
|
@@ -149,37 +153,43 @@ private void tryEnsureNextActionsIterIsReady() { | |
} | ||
|
||
/** | ||
* Get the next file from `filesIter` (.json or .checkpoint.parquet), contextualize it | ||
* (allow the connector to split it), and then read it + inject the `isFromCheckpoint` | ||
* information. | ||
* Get the next file from `filesList` (.json or .checkpoint.parquet) | ||
* read it + inject the `isFromCheckpoint` information. | ||
* <p> | ||
* Requires that `filesIter.hasNext` is true. | ||
* Requires that `filesList.isEmpty` is false. | ||
*/ | ||
private CloseableIterator<ActionWrapper> getNextActionsIter() { | ||
final FileStatus nextFile = filesIter.next(); | ||
|
||
// TODO: [#1965] It should be possible to read our JSON and parquet files | ||
// many-at-once instead of one at a time. | ||
|
||
final FileStatus nextFile = filesList.pop(); | ||
final Path nextFilePath = new Path(nextFile.getPath()); | ||
try { | ||
if (nextFile.getPath().endsWith(".json")) { | ||
final long fileVersion = FileNames.deltaVersion(nextFile.getPath()); | ||
if (isCommitFile(nextFilePath.getName())) { | ||
final long fileVersion = FileNames.deltaVersion(nextFilePath); | ||
|
||
// We can not read multiple JSON files in parallel (like the checkpoint files), | ||
// because each one has a different version, and we need to associate the version | ||
// with actions read from the JSON file for further optimizations later on (faster | ||
// metadata & protocol loading in subsequent runs by remembering the version of | ||
// the last version where the metadata and protocol are found). | ||
|
||
// Read that file | ||
final CloseableIterator<ColumnarBatch> dataIter = | ||
tableClient.getJsonHandler().readJsonFiles( | ||
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. is this an optimization that we should consider (not in this PR)? that the returned columnar batches has the file status of the file from which it was read? that would allow parallel json reads. |
||
singletonCloseableIterator(nextFile), | ||
readSchema, | ||
Optional.empty()); | ||
|
||
return combine(dataIter, false /* isFromCheckpoint */, fileVersion); | ||
} else if (nextFile.getPath().endsWith(".parquet")) { | ||
final long fileVersion = FileNames.checkpointVersion(nextFile.getPath()); | ||
} else if (isCheckpointFile(nextFilePath.getName())) { | ||
final long fileVersion = checkpointVersion(nextFilePath); | ||
|
||
// Try to retrieve the remaining checkpoint files (if there are any) and issue | ||
// read request for all in one go, so that the {@link ParquetHandler} can do | ||
// optimizations like reading multiple files in parallel. | ||
CloseableIterator<FileStatus> checkpointFilesIter = | ||
retrieveRemainingCheckpointFiles(nextFile, fileVersion); | ||
tdas marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
||
// Read that file | ||
final CloseableIterator<ColumnarBatch> dataIter = | ||
tableClient.getParquetHandler().readParquetFiles( | ||
singletonCloseableIterator(nextFile), | ||
checkpointFilesIter, | ||
readSchema, | ||
Optional.empty()); | ||
|
||
|
@@ -218,4 +228,31 @@ public void close() throws IOException { | |
} | ||
}; | ||
} | ||
|
||
/** | ||
* Given a checkpoint file, retrieve all the files that are part of the same checkpoint. | ||
* <p> | ||
* This is done by looking at the file name and finding all the files that have the same | ||
* version number. | ||
*/ | ||
private CloseableIterator<FileStatus> retrieveRemainingCheckpointFiles( | ||
FileStatus checkpointFile, | ||
long version) { | ||
|
||
// Find the contiguous parquet files that are part of the same checkpoint | ||
final List<FileStatus> checkpointFiles = new ArrayList<>(); | ||
|
||
// Add the already retrieved checkpoint file to the list. | ||
checkpointFiles.add(checkpointFile); | ||
|
||
FileStatus peek = filesList.peek(); | ||
while (peek != null && | ||
isCheckpointFile(getName(peek.getPath())) && | ||
checkpointVersion(peek.getPath()) == version) { | ||
checkpointFiles.add(filesList.pop()); | ||
peek = filesList.peek(); | ||
} | ||
|
||
return toCloseableIterator(checkpointFiles.iterator()); | ||
} | ||
} |
This file contains 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 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
Oops, something went wrong.
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.
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.
This is broken on current master.
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.
what does this mean?
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.
there was a commit in
delta-spark
that updated the argument list. Our current CI setup just runs thedelta-spark
tests and not its dependencies. We need to revisit our CI trigger job to make sure all run tests for all dependent modules.I can make a separate PR for this change if you would like.