Skip to content

Commit 96898a2

Browse files
committed
[SPARK-19761][SQL]create InMemoryFileIndex with an empty rootPaths when set PARALLEL_PARTITION_DISCOVERY_THRESHOLD to zero failed
1 parent 9f8e392 commit 96898a2

File tree

2 files changed

+8
-1
lines changed

2 files changed

+8
-1
lines changed

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -300,7 +300,7 @@ object PartitioningAwareFileIndex extends Logging {
300300
sparkSession: SparkSession): Seq[(Path, Seq[FileStatus])] = {
301301

302302
// Short-circuits parallel listing when serial listing is likely to be faster.
303-
if (paths.size < sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) {
303+
if (paths.size <= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) {
304304
return paths.map { path =>
305305
(path, listLeafFiles(path, hadoopConf, filter, Some(sparkSession)))
306306
}

sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@ import org.apache.hadoop.fs.{FileStatus, Path, RawLocalFileSystem}
2727

2828
import org.apache.spark.metrics.source.HiveCatalogMetrics
2929
import org.apache.spark.sql.catalyst.util._
30+
import org.apache.spark.sql.internal.SQLConf
3031
import org.apache.spark.sql.test.SharedSQLContext
3132

3233
class FileIndexSuite extends SharedSQLContext {
@@ -178,6 +179,12 @@ class FileIndexSuite extends SharedSQLContext {
178179
assert(catalog2.allFiles().nonEmpty)
179180
}
180181
}
182+
183+
test("InMemoryFileIndex with empty rootPaths when PARALLEL_PARTITION_DISCOVERY_THRESHOLD is 0") {
184+
withSQLConf(SQLConf.PARALLEL_PARTITION_DISCOVERY_THRESHOLD.key -> "0") {
185+
new InMemoryFileIndex(spark, Seq.empty, Map.empty, None)
186+
}
187+
}
181188
}
182189

183190
class FakeParentPathFileSystem extends RawLocalFileSystem {

0 commit comments

Comments
 (0)