-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-22233] [core] Allow user to filter out empty split in HadoopRDD #19464
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
Changes from 2 commits
cf0c350
31a5d30
4dcfd83
527b367
25f98d0
534d8fb
a6818b6
9f42f9f
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -270,6 +270,15 @@ package object config { | |
| .longConf | ||
| .createWithDefault(4 * 1024 * 1024) | ||
|
|
||
| private [spark] val FILTER_OUT_EMPTY_SPLIT = ConfigBuilder("spark.files.filterOutEmptySplit") | ||
|
||
| .doc("If set to true, HadoopRDD/NewHadoopRDD will not handle the split which its length is 0." + | ||
| "Maybe you will read an empty hive table but has many empty files. If set to false, Spark " + | ||
| "generates many tasks to handle these empty files. Sometimes, users maybe want to use " + | ||
| "SparkContext#textFile to handle a file stored in hadoop, and they don't want to generate " + | ||
| "any task when this file is empty, they can set this configuration to true.") | ||
| .booleanConf | ||
| .createWithDefault(false) | ||
|
|
||
| private[spark] val SECRET_REDACTION_PATTERN = | ||
| ConfigBuilder("spark.redaction.regex") | ||
| .doc("Regex to decide which Spark configuration properties and environment variables in " + | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -35,7 +35,7 @@ import org.apache.spark.annotation.DeveloperApi | |
| import org.apache.spark.broadcast.Broadcast | ||
| import org.apache.spark.deploy.SparkHadoopUtil | ||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.internal.config.IGNORE_CORRUPT_FILES | ||
| import org.apache.spark.internal.config.{FILTER_OUT_EMPTY_SPLIT, IGNORE_CORRUPT_FILES} | ||
| import org.apache.spark.rdd.HadoopRDD.HadoopMapPartitionsWithSplitRDD | ||
| import org.apache.spark.scheduler.{HDFSCacheTaskLocation, HostTaskLocation} | ||
| import org.apache.spark.storage.StorageLevel | ||
|
|
@@ -196,7 +196,11 @@ class HadoopRDD[K, V]( | |
| // add the credentials here as this can be called before SparkContext initialized | ||
| SparkHadoopUtil.get.addCredentials(jobConf) | ||
| val inputFormat = getInputFormat(jobConf) | ||
| val inputSplits = inputFormat.getSplits(jobConf, minPartitions) | ||
| val inputSplits = if (sparkContext.getConf.get(FILTER_OUT_EMPTY_SPLIT)) { | ||
|
||
| inputFormat.getSplits(jobConf, minPartitions).filter(_.getLength > 0) | ||
| } else { | ||
| inputFormat.getSplits(jobConf, minPartitions) | ||
| } | ||
| val array = new Array[Partition](inputSplits.size) | ||
| for (i <- 0 until inputSplits.size) { | ||
| array(i) = new HadoopPartition(id, i, inputSplits(i)) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -31,7 +31,7 @@ import org.apache.hadoop.mapreduce.Job | |
| import org.apache.hadoop.mapreduce.lib.input.{FileSplit => NewFileSplit, TextInputFormat => NewTextInputFormat} | ||
| import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutputFormat} | ||
|
|
||
| import org.apache.spark.internal.config.IGNORE_CORRUPT_FILES | ||
| import org.apache.spark.internal.config.{FILTER_OUT_EMPTY_SPLIT, IGNORE_CORRUPT_FILES} | ||
| import org.apache.spark.rdd.{HadoopRDD, NewHadoopRDD} | ||
| import org.apache.spark.storage.StorageLevel | ||
| import org.apache.spark.util.Utils | ||
|
|
@@ -347,7 +347,7 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { | |
| } | ||
| } | ||
|
|
||
| test ("allow user to disable the output directory existence checking (old Hadoop API") { | ||
| test ("allow user to disable the output directory existence checking (old Hadoop API)") { | ||
| val sf = new SparkConf() | ||
| sf.setAppName("test").setMaster("local").set("spark.hadoop.validateOutputSpecs", "false") | ||
| sc = new SparkContext(sf) | ||
|
|
@@ -510,4 +510,65 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { | |
| } | ||
| } | ||
|
|
||
| test("allow user to filter out empty split (old Hadoop API)") { | ||
| val sf = new SparkConf() | ||
|
||
| sf.setAppName("test").setMaster("local").set(FILTER_OUT_EMPTY_SPLIT, true) | ||
| sc = new SparkContext(sf) | ||
|
|
||
| // Ensure that if all of the splits are empty, we remove the splits correctly | ||
| val emptyRDD = sc.parallelize(Array.empty[Tuple2[String, String]], 1) | ||
| emptyRDD.saveAsHadoopFile[TextOutputFormat[String, String]](tempDir.getPath + "/output") | ||
|
||
| assert(new File(tempDir.getPath + "/output/part-00000").exists() === true) | ||
| val hadoopRDD = sc.textFile(tempDir.getPath + "/output/part-*") | ||
| assert(hadoopRDD.partitions.length === 0) | ||
|
|
||
| // Ensure that if no split is empty, we don't lose any splits | ||
| val randomRDD = sc.parallelize( | ||
| Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 2) | ||
| randomRDD.saveAsHadoopFile[TextOutputFormat[String, String]](tempDir.getPath + "/output1") | ||
| assert(new File(tempDir.getPath + "/output1/part-00001").exists() === true) | ||
| val hadoopRDD1 = sc.textFile(tempDir.getPath + "/output1/part-*") | ||
| assert(hadoopRDD1.partitions.length === 2) | ||
|
|
||
| // Ensure that if part of the splits are empty, we remove the splits correctly | ||
| val randomRDD2 = sc.parallelize( | ||
| Array(("key1", "a"), ("key2", "a")), 5) | ||
| randomRDD2.saveAsHadoopFile[TextOutputFormat[String, String]]( | ||
| tempDir.getPath + "/output2") | ||
| assert(new File(tempDir.getPath + "/output2/part-00004").exists() === true) | ||
| val hadoopRDD2 = sc.textFile(tempDir.getPath + "/output2/part-*") | ||
| assert(hadoopRDD2.partitions.length === 2) | ||
| } | ||
|
|
||
| test("allow user to filter out empty split (new Hadoop API)") { | ||
| val sf = new SparkConf() | ||
| sf.setAppName("test").setMaster("local").set(FILTER_OUT_EMPTY_SPLIT, true) | ||
| sc = new SparkContext(sf) | ||
|
|
||
| // Ensure that if all of the splits are empty, we remove the splits correctly | ||
| val emptyRDD = sc.parallelize(Array.empty[Tuple2[String, String]], 1) | ||
| emptyRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]]( | ||
| tempDir.getPath + "/output") | ||
| assert(new File(tempDir.getPath + "/output/part-r-00000").exists() === true) | ||
| val hadoopRDD = sc.textFile(tempDir.getPath + "/output/part-r-*") | ||
| assert(hadoopRDD.partitions.length === 0) | ||
|
||
|
|
||
| // Ensure that if no split is empty, we don't lose any splits | ||
| val randomRDD1 = sc.parallelize( | ||
| Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 2) | ||
| randomRDD1.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]]( | ||
| tempDir.getPath + "/output1") | ||
| assert(new File(tempDir.getPath + "/output1/part-r-00001").exists() === true) | ||
| val hadoopRDD1 = sc.textFile(tempDir.getPath + "/output1/part-r-*") | ||
| assert(hadoopRDD1.partitions.length === 2) | ||
|
|
||
| // Ensure that if part of the splits are empty, we remove the splits correctly | ||
| val randomRDD2 = sc.parallelize( | ||
| Array(("key1", "a"), ("key2", "a")), 5) | ||
| randomRDD2.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]]( | ||
| tempDir.getPath + "/output2") | ||
| assert(new File(tempDir.getPath + "/output2/part-r-00004").exists() === true) | ||
| val hadoopRDD2 = sc.textFile(tempDir.getPath + "/output2/part-r-*") | ||
| assert(hadoopRDD2.partitions.length === 2) | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -1191,6 +1191,14 @@ Apart from these, the following properties are also available, and may be useful | |
| then the partitions with small files will be faster than partitions with bigger files. | ||
| </td> | ||
| </tr> | ||
| <tr> | ||
| <td><code>spark.files.filterOutEmptySplit</code></td> | ||
|
||
| <td>false</td> | ||
| <td>If set to true, HadoopRDD/NewHadoopRDD will not handle the split which its length is 0. Maybe you will read an empty | ||
| hive table but has many empty files. If set to false, Spark generates many tasks to handle these empty files. | ||
| Sometimes, users maybe want to use SparkContext#textFile to handle a file stored in hadoop, and they don't | ||
| want to generate any task when this file is empty, they can set this configuration to true.</td> | ||
| </tr> | ||
| <tr> | ||
| <td><code>spark.hadoop.cloneConf</code></td> | ||
| <td>false</td> | ||
|
|
||
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: no space after private
This doc is much too verbose for a flag. Just say, "If true, methods like that use HadoopRDD and NewHadoopRDD such as SparkContext.textFiles will not create a partition for input splits that are empty."