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

[SPARK-23249] [SQL] Improved block merging logic for partitions #20372

Closed
Show file tree
Hide file tree
Changes from 1 commit
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 @@ -445,16 +445,25 @@ case class FileSourceScanExec(
currentSize = 0
}

// Assign files to partitions using "Next Fit Decreasing"
splitFiles.foreach { file =>
if (currentSize + file.length > maxSplitBytes) {
def addFile(file: PartitionedFile): Unit = {
currentFiles += file
currentSize += file.length + openCostInBytes
}

var frontIndex = 0
var backIndex = splitFiles.length - 1

while (frontIndex <= backIndex) {
while (frontIndex <= backIndex && currentSize + splitFiles(frontIndex).length <= maxSplitBytes) {
addFile(splitFiles(frontIndex))
frontIndex += 1
}
while (backIndex > frontIndex && currentSize + splitFiles(backIndex).length <= maxSplitBytes) {
addFile(splitFiles(backIndex))
backIndex -= 1
}
closePartition()
}
// Add the given file to the current partition.
currentSize += file.length + openCostInBytes
Copy link
Contributor

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

Copy link
Contributor

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

Copy link
Author

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

Copy link
Author

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 code

Copy link
Contributor

@ash211 ash211 Jan 24, 2018

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!

currentFiles += file
}
closePartition()

new FileScanRDD(fsRelation.sparkSession, readFile, partitions)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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)]
Copy link
Contributor

Choose a reason for hiding this comment

The 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)
Copy link
Contributor

Choose a reason for hiding this comment

The 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)
Expand All @@ -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))
Expand Down