Skip to content

Commit ea02219

Browse files
committed
regenerate input partitions
1 parent 5950550 commit ea02219

13 files changed

Lines changed: 304 additions & 2 deletions

File tree

docs/Configuration.md

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -102,6 +102,7 @@ nav_order: 15
102102
| spark.gluten.sql.columnar.shuffle.sort.partitions.threshold | 4000 | The threshold to determine whether to use sort-based columnar shuffle. Sort-based shuffle will be used if the number of partitions is greater than this threshold. |
103103
| spark.gluten.sql.columnar.shuffledHashJoin | true | Enable or disable columnar shuffledHashJoin. |
104104
| spark.gluten.sql.columnar.shuffledHashJoin.optimizeBuildSide | true | Whether to allow Gluten to choose an optimal build side for shuffled hash join. |
105+
| spark.gluten.sql.columnar.smallFileThreshold | 0.5 | The total size threshold of small files in table scan.To avoid small files being placed into the same partition, Gluten will try to distribute small files into different partitions when the total size of small files is below this threshold. |
105106
| spark.gluten.sql.columnar.sort | true | Enable or disable columnar sort. |
106107
| spark.gluten.sql.columnar.sortMergeJoin | true | Enable or disable columnar sortMergeJoin. This should be set with preferSortMergeJoin=false. |
107108
| spark.gluten.sql.columnar.tableCache | false | Enable or disable columnar table cache. |

gluten-substrait/src/main/scala/org/apache/gluten/config/GlutenConfig.scala

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -289,6 +289,8 @@ class GlutenConfig(conf: SQLConf) extends GlutenCoreConfig(conf) {
289289

290290
def extendedExpressionTransformer: String = getConf(EXTENDED_EXPRESSION_TRAN_CONF)
291291

292+
def smallFileThreshold: Double = getConf(SMALL_FILE_THRESHOLD)
293+
292294
def expressionBlacklist: Set[String] = {
293295
val blacklistSet = getConf(EXPRESSION_BLACK_LIST)
294296
.map(_.toLowerCase(Locale.ROOT).split(",").map(_.trim()).filter(_.nonEmpty).toSet)
@@ -1577,4 +1579,14 @@ object GlutenConfig extends ConfigRegistry {
15771579
.doc("Enable or disable columnar collectTail.")
15781580
.booleanConf
15791581
.createWithDefault(true)
1582+
1583+
val SMALL_FILE_THRESHOLD =
1584+
buildConf("spark.gluten.sql.columnar.smallFileThreshold")
1585+
.doc(
1586+
"The total size threshold of small files in table scan." +
1587+
"To avoid small files being placed into the same partition, " +
1588+
"Gluten will try to distribute small files into different partitions when the " +
1589+
"total size of small files is below this threshold.")
1590+
.doubleConf
1591+
.createWithDefault(0.5)
15801592
}

gluten-substrait/src/main/scala/org/apache/gluten/utils/PartitionsUtil.scala

Lines changed: 96 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -16,17 +16,21 @@
1616
*/
1717
package org.apache.gluten.utils
1818

19+
import org.apache.gluten.config.GlutenConfig
1920
import org.apache.gluten.sql.shims.SparkShimLoader
21+
import org.apache.gluten.utils.PartitionsUtil.regeneratePartition
2022

2123
import org.apache.spark.Partition
2224
import org.apache.spark.internal.Logging
2325
import org.apache.spark.sql.catalyst.expressions.Attribute
24-
import org.apache.spark.sql.execution.datasources.{BucketingUtils, FilePartition, HadoopFsRelation, PartitionDirectory}
26+
import org.apache.spark.sql.execution.datasources.{BucketingUtils, FilePartition, HadoopFsRelation, PartitionDirectory, PartitionedFile}
2527
import org.apache.spark.sql.types.StructType
2628
import org.apache.spark.util.collection.BitSet
2729

2830
import org.apache.hadoop.fs.Path
2931

32+
import scala.collection.mutable
33+
3034
case class PartitionsUtil(
3135
relation: HadoopFsRelation,
3236
requiredSchema: StructType,
@@ -96,7 +100,10 @@ case class PartitionsUtil(
96100
}
97101
.sortBy(_.length)(implicitly[Ordering[Long]].reverse)
98102

99-
FilePartition.getFilePartitions(relation.sparkSession, splitFiles, maxSplitBytes)
103+
val inputPartitions =
104+
FilePartition.getFilePartitions(relation.sparkSession, splitFiles, maxSplitBytes)
105+
106+
regeneratePartition(inputPartitions, GlutenConfig.get.smallFileThreshold)
100107
}
101108

102109
private def genBucketedPartitionSeq(): Seq[Partition] = {
@@ -140,3 +147,90 @@ case class PartitionsUtil(
140147
output.find(_.name == colName)
141148
}
142149
}
150+
151+
object PartitionsUtil {
152+
153+
/**
154+
* Regenerate the partitions by balancing the number of files per partition and total size per
155+
* partition.
156+
*/
157+
def regeneratePartition(
158+
inputPartitions: Seq[FilePartition],
159+
smallFileThreshold: Double): Seq[FilePartition] = {
160+
161+
// Flatten and sort descending by file size.
162+
val filesSorted: Seq[(PartitionedFile, Long)] =
163+
inputPartitions
164+
.flatMap(_.files)
165+
.map(f => (f, f.length))
166+
.sortBy(_._2)(Ordering.Long.reverse)
167+
168+
val partitions = Array.fill(inputPartitions.size)(mutable.ArrayBuffer.empty[PartitionedFile])
169+
170+
def addToBucket(
171+
heap: mutable.PriorityQueue[(Long, Int, Int)],
172+
file: PartitionedFile,
173+
sz: Long): Unit = {
174+
val (load, numFiles, idx) = heap.dequeue()
175+
partitions(idx) += file
176+
heap.enqueue((load + sz, numFiles + 1, idx))
177+
}
178+
179+
// First by load, then by numFiles.
180+
val heapByFileSize =
181+
mutable.PriorityQueue.empty[(Long, Int, Int)](
182+
Ordering
183+
.by[(Long, Int, Int), (Long, Int)] {
184+
case (load, numFiles, _) =>
185+
(load, numFiles)
186+
}
187+
.reverse
188+
)
189+
190+
if (smallFileThreshold > 0) {
191+
val smallFileTotalSize = filesSorted.map(_._2).sum * smallFileThreshold
192+
// First by numFiles, then by load.
193+
val heapByFileNum =
194+
mutable.PriorityQueue.empty[(Long, Int, Int)](
195+
Ordering
196+
.by[(Long, Int, Int), (Int, Long)] {
197+
case (load, numFiles, _) =>
198+
(numFiles, load)
199+
}
200+
.reverse
201+
)
202+
203+
inputPartitions.indices.foreach(i => heapByFileNum.enqueue((0L, 0, i)))
204+
205+
var numSmallFiles = 0
206+
var smallFileSize = 0L
207+
// Enqueue small files to the least number of files and the least load.
208+
filesSorted.reverse.takeWhile(f => f._2 + smallFileSize <= smallFileTotalSize).foreach {
209+
case (file, sz) =>
210+
addToBucket(heapByFileNum, file, sz)
211+
numSmallFiles += 1
212+
smallFileSize += sz
213+
}
214+
215+
// Move buckets from heapByFileNum to heapByFileSize.
216+
while (heapByFileNum.nonEmpty) {
217+
heapByFileSize.enqueue(heapByFileNum.dequeue())
218+
}
219+
220+
// Finally, enqueue remaining files.
221+
filesSorted.take(filesSorted.size - numSmallFiles).foreach {
222+
case (file, sz) =>
223+
addToBucket(heapByFileSize, file, sz)
224+
}
225+
} else {
226+
inputPartitions.indices.foreach(i => heapByFileSize.enqueue((0L, 0, i)))
227+
228+
filesSorted.foreach {
229+
case (file, sz) =>
230+
addToBucket(heapByFileSize, file, sz)
231+
}
232+
}
233+
234+
partitions.zipWithIndex.map { case (p, idx) => FilePartition(idx, p.toArray) }
235+
}
236+
}
Lines changed: 102 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,102 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
package org.apache.gluten.utils
18+
19+
import org.apache.gluten.execution.PartitionedFileUtilShim
20+
21+
import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile}
22+
23+
import org.scalatest.funsuite.AnyFunSuite
24+
25+
class PartitionsUtilSuite extends AnyFunSuite {
26+
27+
private def makePartitionedFile(path: String, length: Long): PartitionedFile =
28+
PartitionedFileUtilShim.makePartitionedFileFromPath(path, length)
29+
30+
private def makeFilePartitions(
31+
files: Seq[PartitionedFile],
32+
numPartitions: Int): Seq[FilePartition] = {
33+
val numGroups = files.size / numPartitions +
34+
(if (files.size % numPartitions == 0) 0 else 1)
35+
files.grouped(numGroups).toSeq.zipWithIndex.map {
36+
case (p, idx) => FilePartition(idx, p.toArray)
37+
}
38+
}
39+
40+
test("large files are distributed evenly by size") {
41+
val files = Seq(
42+
makePartitionedFile("f1", 100),
43+
makePartitionedFile("f2", 90),
44+
makePartitionedFile("f3", 80),
45+
makePartitionedFile("f4", 70)
46+
)
47+
val initialPartitions = makeFilePartitions(files, 2)
48+
49+
val result = PartitionsUtil.regeneratePartition(initialPartitions, 0.0)
50+
51+
assert(result.size === 2)
52+
53+
val sizes = result.map(_.files.map(_.length).sum)
54+
assert(sizes.forall(_ === 170))
55+
}
56+
57+
test("small files are distributed evenly by number of files") {
58+
val files = (1 to 10).map(i => makePartitionedFile(s"f$i", 10))
59+
val initialPartitions = makeFilePartitions(files, 5)
60+
61+
val result = PartitionsUtil.regeneratePartition(initialPartitions, 1.0)
62+
63+
assert(result.size === 5)
64+
val counts = result.map(_.files.length)
65+
assert(counts.forall(_ === 2))
66+
}
67+
68+
test("small files should not be placed into one partition") {
69+
val files = Seq(
70+
makePartitionedFile("f1", 10),
71+
makePartitionedFile("f2", 20),
72+
makePartitionedFile("f3", 30),
73+
makePartitionedFile("f4", 40),
74+
makePartitionedFile("f5", 100)
75+
)
76+
val initialPartitions = makeFilePartitions(files, 2)
77+
78+
// Only "f5" is not small file.
79+
val result = PartitionsUtil.regeneratePartition(initialPartitions, 0.5)
80+
81+
assert(result.size === 2)
82+
assert(result.forall(_.files.exists(_.length <= 40)))
83+
}
84+
85+
test("zero length files") {
86+
val files = Seq(
87+
makePartitionedFile("f1", 0),
88+
makePartitionedFile("f2", 0)
89+
)
90+
val initialPartitions = makeFilePartitions(files, 2)
91+
92+
val result = PartitionsUtil.regeneratePartition(initialPartitions, 0.0)
93+
94+
assert(result.size === 2)
95+
assert(result.count(_.files.nonEmpty) === 2)
96+
}
97+
98+
test("empty inputs") {
99+
val result = PartitionsUtil.regeneratePartition(Seq.empty, 0.5)
100+
assert(result.size === 0)
101+
}
102+
}

gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -189,6 +189,7 @@ class GlutenSQLQueryTestSuite
189189
.set("spark.memory.offHeap.size", "1024MB")
190190
.set("spark.plugins", "org.apache.gluten.GlutenPlugin")
191191
.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager")
192+
.set(GlutenConfig.SMALL_FILE_THRESHOLD.key, "0")
192193

193194
if (isCHBackend) {
194195
conf

gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -189,6 +189,7 @@ class GlutenSQLQueryTestSuite
189189
.set("spark.memory.offHeap.size", "1024MB")
190190
.set("spark.plugins", "org.apache.gluten.GlutenPlugin")
191191
.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager")
192+
.set(GlutenConfig.SMALL_FILE_THRESHOLD.key, "0")
192193

193194
if (isCHBackend) {
194195
conf

gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -193,6 +193,7 @@ class GlutenSQLQueryTestSuite
193193
.set("spark.memory.offHeap.size", "1024MB")
194194
.set("spark.plugins", "org.apache.gluten.GlutenPlugin")
195195
.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager")
196+
.set(GlutenConfig.SMALL_FILE_THRESHOLD.key, "0")
196197

197198
if (isCHBackend) {
198199
conf

gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -116,6 +116,7 @@ class GlutenSQLQueryTestSuite
116116
.set("spark.memory.offHeap.size", "1024MB")
117117
.set("spark.plugins", "org.apache.gluten.GlutenPlugin")
118118
.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager")
119+
.set(GlutenConfig.SMALL_FILE_THRESHOLD.key, "0")
119120

120121
if (isCHBackend) {
121122
conf
Lines changed: 26 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,26 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
package org.apache.gluten.execution
18+
19+
import org.apache.spark.sql.execution.datasources.PartitionedFile
20+
21+
object PartitionedFileUtilShim {
22+
// Helper method to create PartitionedFile from path and length.
23+
def makePartitionedFileFromPath(path: String, length: Long): PartitionedFile = {
24+
PartitionedFile(null, path, 0, length, Array.empty)
25+
}
26+
}
Lines changed: 26 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,26 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
package org.apache.gluten.execution
18+
19+
import org.apache.spark.sql.execution.datasources.PartitionedFile
20+
21+
object PartitionedFileUtilShim {
22+
// Helper method to create PartitionedFile from path and length.
23+
def makePartitionedFileFromPath(path: String, length: Long): PartitionedFile = {
24+
PartitionedFile(null, path, 0, length, Array.empty)
25+
}
26+
}

0 commit comments

Comments
 (0)