Skip to content

Commit 25e9156

Browse files
cozossrowen
authored andcommitted
[SPARK-29089][SQL] Parallelize blocking FileSystem calls in DataSource#checkAndGlobPathIfNecessary
### What changes were proposed in this pull request? See JIRA: https://issues.apache.org/jira/browse/SPARK-29089 Mailing List: http://apache-spark-developers-list.1001551.n3.nabble.com/DataFrameReader-bottleneck-in-DataSource-checkAndGlobPathIfNecessary-when-reading-S3-files-td27828.html When using DataFrameReader#csv to read many files on S3, globbing and fs.exists on DataSource#checkAndGlobPathIfNecessary becomes a bottleneck. From the mailing list discussions, an improvement that can be made is to parallelize the blocking FS calls: > - have SparkHadoopUtils differentiate between files returned by globStatus(), and which therefore exist, and those which it didn't glob for -it will only need to check those. > - add parallel execution to the glob and existence checks ### Why are the changes needed? Verifying/globbing files happens on the driver, and if this operations take a long time (for example against S3), then the entire cluster has to wait, potentially sitting idle. This change hopes to make this process faster. ### Does this PR introduce any user-facing change? No ### How was this patch tested? I added a test suite `DataSourceSuite` - open to suggestions for better naming. See [here](#25899 (comment)) and [here](#25899 (comment)) for some measurements Closes #25899 from cozos/master. Lead-authored-by: Arwin Tio <Arwin.tio@adroll.com> Co-authored-by: Arwin Tio <arwin.tio@hotmail.com> Co-authored-by: Arwin Tio <arwin.tio@adroll.com> Signed-off-by: Sean Owen <srowen@gmail.com>
1 parent 06217cf commit 25e9156

File tree

2 files changed

+235
-18
lines changed

2 files changed

+235
-18
lines changed

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

Lines changed: 42 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@ import scala.util.{Failure, Success, Try}
2626
import org.apache.hadoop.conf.Configuration
2727
import org.apache.hadoop.fs.Path
2828

29+
import org.apache.spark.SparkException
2930
import org.apache.spark.deploy.SparkHadoopUtil
3031
import org.apache.spark.internal.Logging
3132
import org.apache.spark.sql._
@@ -50,7 +51,7 @@ import org.apache.spark.sql.sources._
5051
import org.apache.spark.sql.streaming.OutputMode
5152
import org.apache.spark.sql.types.{CalendarIntervalType, StructField, StructType}
5253
import org.apache.spark.sql.util.SchemaUtils
53-
import org.apache.spark.util.Utils
54+
import org.apache.spark.util.{ThreadUtils, Utils}
5455

5556
/**
5657
* The main class responsible for representing a pluggable Data Source in Spark SQL. In addition to
@@ -739,30 +740,53 @@ object DataSource extends Logging {
739740
* Checks and returns files in all the paths.
740741
*/
741742
private[sql] def checkAndGlobPathIfNecessary(
742-
paths: Seq[String],
743+
pathStrings: Seq[String],
743744
hadoopConf: Configuration,
744745
checkEmptyGlobPath: Boolean,
745-
checkFilesExist: Boolean): Seq[Path] = {
746-
val allGlobPath = paths.flatMap { path =>
747-
val hdfsPath = new Path(path)
748-
val fs = hdfsPath.getFileSystem(hadoopConf)
749-
val qualified = hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory)
750-
val globPath = SparkHadoopUtil.get.globPathIfNecessary(fs, qualified)
751-
752-
if (checkEmptyGlobPath && globPath.isEmpty) {
753-
throw new AnalysisException(s"Path does not exist: $qualified")
746+
checkFilesExist: Boolean,
747+
numThreads: Integer = 40): Seq[Path] = {
748+
val qualifiedPaths = pathStrings.map { pathString =>
749+
val path = new Path(pathString)
750+
val fs = path.getFileSystem(hadoopConf)
751+
path.makeQualified(fs.getUri, fs.getWorkingDirectory)
752+
}
753+
754+
// Split the paths into glob and non glob paths, because we don't need to do an existence check
755+
// for globbed paths.
756+
val (globPaths, nonGlobPaths) = qualifiedPaths.partition(SparkHadoopUtil.get.isGlobPath)
757+
758+
val globbedPaths =
759+
try {
760+
ThreadUtils.parmap(globPaths, "globPath", numThreads) { globPath =>
761+
val fs = globPath.getFileSystem(hadoopConf)
762+
val globResult = SparkHadoopUtil.get.globPath(fs, globPath)
763+
764+
if (checkEmptyGlobPath && globResult.isEmpty) {
765+
throw new AnalysisException(s"Path does not exist: $globPath")
766+
}
767+
768+
globResult
769+
}.flatten
770+
} catch {
771+
case e: SparkException => throw e.getCause
754772
}
755773

756-
// Sufficient to check head of the globPath seq for non-glob scenario
757-
// Don't need to check once again if files exist in streaming mode
758-
if (checkFilesExist && !fs.exists(globPath.head)) {
759-
throw new AnalysisException(s"Path does not exist: ${globPath.head}")
774+
if (checkFilesExist) {
775+
try {
776+
ThreadUtils.parmap(nonGlobPaths, "checkPathsExist", numThreads) { path =>
777+
val fs = path.getFileSystem(hadoopConf)
778+
if (!fs.exists(path)) {
779+
throw new AnalysisException(s"Path does not exist: $path")
780+
}
781+
}
782+
} catch {
783+
case e: SparkException => throw e.getCause
760784
}
761-
globPath
762785
}
763786

787+
val allPaths = globbedPaths ++ nonGlobPaths
764788
if (checkFilesExist) {
765-
val (filteredOut, filteredIn) = allGlobPath.partition { path =>
789+
val (filteredOut, filteredIn) = allPaths.partition { path =>
766790
InMemoryFileIndex.shouldFilterOut(path.getName)
767791
}
768792
if (filteredIn.isEmpty) {
@@ -774,7 +798,7 @@ object DataSource extends Logging {
774798
}
775799
}
776800

777-
allGlobPath
801+
allPaths.toSeq
778802
}
779803

780804
/**
Lines changed: 193 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,193 @@
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+
18+
package org.apache.spark.sql.execution.datasources
19+
20+
import org.apache.hadoop.conf.Configuration
21+
import org.apache.hadoop.fs.{FileStatus, Path, RawLocalFileSystem}
22+
23+
import org.apache.spark.sql.AnalysisException
24+
import org.apache.spark.sql.test.SharedSparkSession
25+
26+
class DataSourceSuite extends SharedSparkSession {
27+
import TestPaths._
28+
29+
test("test glob and non glob paths") {
30+
val resultPaths = DataSource.checkAndGlobPathIfNecessary(
31+
Seq(
32+
path1.toString,
33+
path2.toString,
34+
globPath1.toString,
35+
globPath2.toString
36+
),
37+
hadoopConf,
38+
checkEmptyGlobPath = true,
39+
checkFilesExist = true
40+
)
41+
42+
assert(resultPaths.toSet === allPathsInFs.toSet)
43+
}
44+
45+
test("test glob paths") {
46+
val resultPaths = DataSource.checkAndGlobPathIfNecessary(
47+
Seq(
48+
globPath1.toString,
49+
globPath2.toString
50+
),
51+
hadoopConf,
52+
checkEmptyGlobPath = true,
53+
checkFilesExist = true
54+
)
55+
56+
assert(
57+
resultPaths.toSet === Set(
58+
globPath1Result1,
59+
globPath1Result2,
60+
globPath2Result1,
61+
globPath2Result2
62+
)
63+
)
64+
}
65+
66+
test("test non glob paths") {
67+
val resultPaths = DataSource.checkAndGlobPathIfNecessary(
68+
Seq(
69+
path1.toString,
70+
path2.toString
71+
),
72+
hadoopConf,
73+
checkEmptyGlobPath = true,
74+
checkFilesExist = true
75+
)
76+
77+
assert(
78+
resultPaths.toSet === Set(
79+
path1,
80+
path2
81+
)
82+
)
83+
}
84+
85+
test("test non glob paths checkFilesExist=false") {
86+
val resultPaths = DataSource.checkAndGlobPathIfNecessary(
87+
Seq(
88+
path1.toString,
89+
path2.toString,
90+
nonExistentPath.toString
91+
),
92+
hadoopConf,
93+
checkEmptyGlobPath = true,
94+
checkFilesExist = false
95+
)
96+
97+
assert(
98+
resultPaths.toSet === Set(
99+
path1,
100+
path2,
101+
nonExistentPath
102+
)
103+
)
104+
}
105+
106+
test("test non existent paths") {
107+
assertThrows[AnalysisException](
108+
DataSource.checkAndGlobPathIfNecessary(
109+
Seq(
110+
path1.toString,
111+
path2.toString,
112+
nonExistentPath.toString
113+
),
114+
hadoopConf,
115+
checkEmptyGlobPath = true,
116+
checkFilesExist = true
117+
)
118+
)
119+
}
120+
121+
test("test non existent glob paths") {
122+
assertThrows[AnalysisException](
123+
DataSource.checkAndGlobPathIfNecessary(
124+
Seq(
125+
globPath1.toString,
126+
globPath2.toString,
127+
nonExistentGlobPath.toString
128+
),
129+
hadoopConf,
130+
checkEmptyGlobPath = true,
131+
checkFilesExist = true
132+
)
133+
)
134+
}
135+
}
136+
137+
object TestPaths {
138+
val hadoopConf = new Configuration()
139+
hadoopConf.set("fs.mockFs.impl", classOf[MockFileSystem].getName)
140+
141+
val path1 = new Path("mockFs://mockFs/somepath1")
142+
val path2 = new Path("mockFs://mockFs/somepath2")
143+
val globPath1 = new Path("mockFs://mockFs/globpath1*")
144+
val globPath2 = new Path("mockFs://mockFs/globpath2*")
145+
146+
val nonExistentPath = new Path("mockFs://mockFs/nonexistentpath")
147+
val nonExistentGlobPath = new Path("mockFs://mockFs/nonexistentpath*")
148+
149+
val globPath1Result1 = new Path("mockFs://mockFs/globpath1/path1")
150+
val globPath1Result2 = new Path("mockFs://mockFs/globpath1/path2")
151+
val globPath2Result1 = new Path("mockFs://mockFs/globpath2/path1")
152+
val globPath2Result2 = new Path("mockFs://mockFs/globpath2/path2")
153+
154+
val allPathsInFs = Seq(
155+
path1,
156+
path2,
157+
globPath1Result1,
158+
globPath1Result2,
159+
globPath2Result1,
160+
globPath2Result2
161+
)
162+
163+
val mockGlobResults: Map[Path, Array[FileStatus]] = Map(
164+
globPath1 ->
165+
Array(
166+
createMockFileStatus(globPath1Result1.toString),
167+
createMockFileStatus(globPath1Result2.toString)
168+
),
169+
globPath2 ->
170+
Array(
171+
createMockFileStatus(globPath2Result1.toString),
172+
createMockFileStatus(globPath2Result2.toString)
173+
)
174+
)
175+
176+
def createMockFileStatus(path: String): FileStatus = {
177+
val fileStatus = new FileStatus()
178+
fileStatus.setPath(new Path(path))
179+
fileStatus
180+
}
181+
}
182+
183+
class MockFileSystem extends RawLocalFileSystem {
184+
import TestPaths._
185+
186+
override def exists(f: Path): Boolean = {
187+
allPathsInFs.contains(f)
188+
}
189+
190+
override def globStatus(pathPattern: Path): Array[FileStatus] = {
191+
mockGlobResults.getOrElse(pathPattern, Array())
192+
}
193+
}

0 commit comments

Comments
 (0)