-
Notifications
You must be signed in to change notification settings - Fork 28.5k
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
[SPARK-23249] [SQL] Improved block merging logic for partitions #20372
Closed
glentakahashi
wants to merge
4
commits into
apache:master
from
glentakahashi:feature/improved-block-merging
Closed
Changes from 1 commit
Commits
Show all changes
4 commits
Select commit
Hold shift + click to select a range
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 |
---|---|---|
|
@@ -142,15 +142,16 @@ class FileSourceStrategySuite extends QueryTest with SharedSQLContext with Predi | |
SQLConf.FILES_OPEN_COST_IN_BYTES.key -> "1") { | ||
checkScan(table.select('c1)) { partitions => | ||
// Files should be laid out [(file1), (file2, file3), (file4, file5), (file6)] | ||
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. comment is stale now |
||
assert(partitions.size == 4, "when checking partitions") | ||
assert(partitions(0).files.size == 1, "when checking partition 1") | ||
assert(partitions.size == 3, "when checking partitions") | ||
assert(partitions(0).files.size == 2, "when checking partition 1") | ||
assert(partitions(1).files.size == 2, "when checking partition 2") | ||
assert(partitions(2).files.size == 2, "when checking partition 3") | ||
assert(partitions(3).files.size == 1, "when checking partition 4") | ||
|
||
// First partition reads (file1) | ||
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. comment is stale now |
||
assert(partitions(0).files(0).start == 0) | ||
assert(partitions(0).files(0).length == 2) | ||
assert(partitions(0).files(1).start == 0) | ||
assert(partitions(0).files(1).length == 1) | ||
|
||
// Second partition reads (file2, file3) | ||
assert(partitions(1).files(0).start == 0) | ||
|
@@ -163,10 +164,6 @@ class FileSourceStrategySuite extends QueryTest with SharedSQLContext with Predi | |
assert(partitions(2).files(0).length == 1) | ||
assert(partitions(2).files(1).start == 0) | ||
assert(partitions(2).files(1).length == 1) | ||
|
||
// Final partition reads (file6) | ||
assert(partitions(3).files(0).start == 0) | ||
assert(partitions(3).files(0).length == 1) | ||
} | ||
|
||
checkPartitionSchema(StructType(Nil)) | ||
|
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 handles files that are larger than the maxSplitBytes, which I think isn't done in the new algorithm. Need to make sure those don't form an infinite loop
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.
should add a test for this too
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 test still passes https://github.com/glentakahashi/spark/blob/c575977a5952bf50b605be8079c9be1e30f3bd36/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala#L101, and the splitFiles should have been already split by
filesMaxPartitionBytes
I originally had similar functionality in, but I /think/ its redundant
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.
Oh, only if the file
isSplittable
, so yea, will fail for non-splittable files. Will update codeThere 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.
saw you added a commit to handle the large non-splittable files case -- can you please add a test for that also?
ef04de9
want to make sure this while loop never becomes an infinite loop!