Skip to content

Commit 8ccfbc1

Browse files
sunchaoholdenk
andcommitted
[SPARK-32381][CORE][SQL] Move and refactor parallel listing & non-location sensitive listing to core
<!-- Thanks for sending a pull request! Here are some tips for you: 1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html 2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html 3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'. 4. Be sure to keep the PR description updated to reflect all changes. 5. Please write your PR title to summarize what this PR proposes. 6. If possible, provide a concise example to reproduce the issue for a faster review. 7. If you want to add a new configuration, please read the guideline first for naming configurations in 'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'. --> ### What changes were proposed in this pull request? <!-- Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below. 1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers. 2. If you fix some SQL features, you can provide some references of other DBMSes. 3. If there is design documentation, please add the link. 4. If there is a discussion in the mailing list, please add the link. --> This moves and refactors the parallel listing utilities from `InMemoryFileIndex` to Spark core so it can be reused by modules beside SQL. Along the process this also did some cleanups/refactorings: - Created a `HadoopFSUtils` class under core - Moved `InMemoryFileIndex.bulkListLeafFiles` into `HadoopFSUtils.parallelListLeafFiles`. It now depends on a `SparkContext` instead of `SparkSession` in SQL. Also added a few parameters which used to be read from `SparkSession.conf`: `ignoreMissingFiles`, `ignoreLocality`, `parallelismThreshold`, `parallelismMax ` and `filterFun` (for additional filtering support but we may be able to merge this with `filter` parameter in future). - Moved `InMemoryFileIndex.listLeafFiles` into `HadoopFSUtils.listLeafFiles` with similar changes above. ### Why are the changes needed? <!-- Please clarify why the changes are needed. For instance, 1. If you propose a new API, clarify the use case for a new API. 2. If you fix a bug, you can clarify why it is a bug. --> Currently the locality-aware parallel listing mechanism only applies to `InMemoryFileIndex`. By moving this to core, we can potentially reuse the same mechanism for other code paths as well. ### Does this PR introduce _any_ user-facing change? <!-- Note that it means *any* user-facing change including all aspects such as the documentation fix. If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible. If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master. If no, write 'No'. --> No. ### How was this patch tested? <!-- If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible. If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future. If tests were not added, please describe why they were not added and/or why it was difficult to add. --> Since this is mostly a refactoring, it relies on existing unit tests such as those for `InMemoryFileIndex`. Closes #29471 from sunchao/SPARK-32381. Lead-authored-by: Chao Sun <[email protected]> Co-authored-by: Holden Karau <[email protected]> Co-authored-by: Chao Sun <[email protected]> Signed-off-by: Holden Karau <[email protected]>
1 parent 4ae0f70 commit 8ccfbc1

File tree

3 files changed

+376
-283
lines changed

3 files changed

+376
-283
lines changed
Lines changed: 360 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,360 @@
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.util
19+
20+
import java.io.FileNotFoundException
21+
22+
import scala.collection.mutable
23+
24+
import org.apache.hadoop.conf.Configuration
25+
import org.apache.hadoop.fs._
26+
import org.apache.hadoop.fs.viewfs.ViewFileSystem
27+
import org.apache.hadoop.hdfs.DistributedFileSystem
28+
29+
import org.apache.spark._
30+
import org.apache.spark.annotation.Private
31+
import org.apache.spark.internal.Logging
32+
import org.apache.spark.metrics.source.HiveCatalogMetrics
33+
34+
/**
35+
* Utility functions to simplify and speed-up file listing.
36+
*/
37+
private[spark] object HadoopFSUtils extends Logging {
38+
/**
39+
* Lists a collection of paths recursively. Picks the listing strategy adaptively depending
40+
* on the number of paths to list.
41+
*
42+
* This may only be called on the driver.
43+
*
44+
* @param sc Spark context used to run parallel listing.
45+
* @param paths Input paths to list
46+
* @param hadoopConf Hadoop configuration
47+
* @param filter Path filter used to exclude leaf files from result
48+
* @param isRootLevel Whether the input paths are at the root level, i.e., they are the root
49+
* paths as opposed to nested paths encountered during recursive calls of this.
50+
* @param ignoreMissingFiles Ignore missing files that happen during recursive listing
51+
* (e.g., due to race conditions)
52+
* @param ignoreLocality Whether to fetch data locality info when listing leaf files. If false,
53+
* this will return `FileStatus` without `BlockLocation` info.
54+
* @param parallelismThreshold The threshold to enable parallelism. If the number of input paths
55+
* is smaller than this value, this will fallback to use
56+
* sequential listing.
57+
* @param parallelismMax The maximum parallelism for listing. If the number of input paths is
58+
* larger than this value, parallelism will be throttled to this value
59+
* to avoid generating too many tasks.
60+
* @param filterFun Optional predicate on the leaf files. Files who failed the check will be
61+
* excluded from the results
62+
* @return for each input path, the set of discovered files for the path
63+
*/
64+
def parallelListLeafFiles(
65+
sc: SparkContext,
66+
paths: Seq[Path],
67+
hadoopConf: Configuration,
68+
filter: PathFilter,
69+
isRootLevel: Boolean,
70+
ignoreMissingFiles: Boolean,
71+
ignoreLocality: Boolean,
72+
parallelismThreshold: Int,
73+
parallelismMax: Int,
74+
filterFun: Option[String => Boolean] = None): Seq[(Path, Seq[FileStatus])] = {
75+
76+
// Short-circuits parallel listing when serial listing is likely to be faster.
77+
if (paths.size <= parallelismThreshold) {
78+
return paths.map { path =>
79+
val leafFiles = listLeafFiles(
80+
path,
81+
hadoopConf,
82+
filter,
83+
Some(sc),
84+
ignoreMissingFiles = ignoreMissingFiles,
85+
ignoreLocality = ignoreLocality,
86+
isRootPath = isRootLevel,
87+
parallelismThreshold = parallelismThreshold,
88+
parallelismMax = parallelismMax,
89+
filterFun = filterFun)
90+
(path, leafFiles)
91+
}
92+
}
93+
94+
logInfo(s"Listing leaf files and directories in parallel under ${paths.length} paths." +
95+
s" The first several paths are: ${paths.take(10).mkString(", ")}.")
96+
HiveCatalogMetrics.incrementParallelListingJobCount(1)
97+
98+
val serializableConfiguration = new SerializableConfiguration(hadoopConf)
99+
val serializedPaths = paths.map(_.toString)
100+
101+
// Set the number of parallelism to prevent following file listing from generating many tasks
102+
// in case of large #defaultParallelism.
103+
val numParallelism = Math.min(paths.size, parallelismMax)
104+
105+
val previousJobDescription = sc.getLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION)
106+
val statusMap = try {
107+
val description = paths.size match {
108+
case 0 =>
109+
"Listing leaf files and directories 0 paths"
110+
case 1 =>
111+
s"Listing leaf files and directories for 1 path:<br/>${paths(0)}"
112+
case s =>
113+
s"Listing leaf files and directories for $s paths:<br/>${paths(0)}, ..."
114+
}
115+
sc.setJobDescription(description)
116+
sc
117+
.parallelize(serializedPaths, numParallelism)
118+
.mapPartitions { pathStrings =>
119+
val hadoopConf = serializableConfiguration.value
120+
pathStrings.map(new Path(_)).toSeq.map { path =>
121+
val leafFiles = listLeafFiles(
122+
path = path,
123+
hadoopConf = hadoopConf,
124+
filter = filter,
125+
contextOpt = None, // Can't execute parallel scans on workers
126+
ignoreMissingFiles = ignoreMissingFiles,
127+
ignoreLocality = ignoreLocality,
128+
isRootPath = isRootLevel,
129+
filterFun = filterFun,
130+
parallelismThreshold = Int.MaxValue,
131+
parallelismMax = 0)
132+
(path, leafFiles)
133+
}.iterator
134+
}.map { case (path, statuses) =>
135+
val serializableStatuses = statuses.map { status =>
136+
// Turn FileStatus into SerializableFileStatus so we can send it back to the driver
137+
val blockLocations = status match {
138+
case f: LocatedFileStatus =>
139+
f.getBlockLocations.map { loc =>
140+
SerializableBlockLocation(
141+
loc.getNames,
142+
loc.getHosts,
143+
loc.getOffset,
144+
loc.getLength)
145+
}
146+
147+
case _ =>
148+
Array.empty[SerializableBlockLocation]
149+
}
150+
151+
SerializableFileStatus(
152+
status.getPath.toString,
153+
status.getLen,
154+
status.isDirectory,
155+
status.getReplication,
156+
status.getBlockSize,
157+
status.getModificationTime,
158+
status.getAccessTime,
159+
blockLocations)
160+
}
161+
(path.toString, serializableStatuses)
162+
}.collect()
163+
} finally {
164+
sc.setJobDescription(previousJobDescription)
165+
}
166+
167+
// turn SerializableFileStatus back to Status
168+
statusMap.map { case (path, serializableStatuses) =>
169+
val statuses = serializableStatuses.map { f =>
170+
val blockLocations = f.blockLocations.map { loc =>
171+
new BlockLocation(loc.names, loc.hosts, loc.offset, loc.length)
172+
}
173+
new LocatedFileStatus(
174+
new FileStatus(
175+
f.length, f.isDir, f.blockReplication, f.blockSize, f.modificationTime,
176+
new Path(f.path)),
177+
blockLocations)
178+
}
179+
(new Path(path), statuses)
180+
}
181+
}
182+
183+
// scalastyle:off argcount
184+
/**
185+
* Lists a single filesystem path recursively. If a `SparkContext` object is specified, this
186+
* function may launch Spark jobs to parallelize listing based on `parallelismThreshold`.
187+
*
188+
* If sessionOpt is None, this may be called on executors.
189+
*
190+
* @return all children of path that match the specified filter.
191+
*/
192+
private def listLeafFiles(
193+
path: Path,
194+
hadoopConf: Configuration,
195+
filter: PathFilter,
196+
contextOpt: Option[SparkContext],
197+
ignoreMissingFiles: Boolean,
198+
ignoreLocality: Boolean,
199+
isRootPath: Boolean,
200+
filterFun: Option[String => Boolean],
201+
parallelismThreshold: Int,
202+
parallelismMax: Int): Seq[FileStatus] = {
203+
204+
logTrace(s"Listing $path")
205+
val fs = path.getFileSystem(hadoopConf)
206+
207+
// Note that statuses only include FileStatus for the files and dirs directly under path,
208+
// and does not include anything else recursively.
209+
val statuses: Array[FileStatus] = try {
210+
fs match {
211+
// DistributedFileSystem overrides listLocatedStatus to make 1 single call to namenode
212+
// to retrieve the file status with the file block location. The reason to still fallback
213+
// to listStatus is because the default implementation would potentially throw a
214+
// FileNotFoundException which is better handled by doing the lookups manually below.
215+
case (_: DistributedFileSystem | _: ViewFileSystem) if !ignoreLocality =>
216+
val remoteIter = fs.listLocatedStatus(path)
217+
new Iterator[LocatedFileStatus]() {
218+
def next(): LocatedFileStatus = remoteIter.next
219+
def hasNext(): Boolean = remoteIter.hasNext
220+
}.toArray
221+
case _ => fs.listStatus(path)
222+
}
223+
} catch {
224+
// If we are listing a root path for SQL (e.g. a top level directory of a table), we need to
225+
// ignore FileNotFoundExceptions during this root level of the listing because
226+
//
227+
// (a) certain code paths might construct an InMemoryFileIndex with root paths that
228+
// might not exist (i.e. not all callers are guaranteed to have checked
229+
// path existence prior to constructing InMemoryFileIndex) and,
230+
// (b) we need to ignore deleted root paths during REFRESH TABLE, otherwise we break
231+
// existing behavior and break the ability drop SessionCatalog tables when tables'
232+
// root directories have been deleted (which breaks a number of Spark's own tests).
233+
//
234+
// If we are NOT listing a root path then a FileNotFoundException here means that the
235+
// directory was present in a previous level of file listing but is absent in this
236+
// listing, likely indicating a race condition (e.g. concurrent table overwrite or S3
237+
// list inconsistency).
238+
//
239+
// The trade-off in supporting existing behaviors / use-cases is that we won't be
240+
// able to detect race conditions involving root paths being deleted during
241+
// InMemoryFileIndex construction. However, it's still a net improvement to detect and
242+
// fail-fast on the non-root cases. For more info see the SPARK-27676 review discussion.
243+
case _: FileNotFoundException if isRootPath || ignoreMissingFiles =>
244+
logWarning(s"The directory $path was not found. Was it deleted very recently?")
245+
Array.empty[FileStatus]
246+
}
247+
248+
def doFilter(statuses: Array[FileStatus]) = filterFun match {
249+
case Some(shouldFilterOut) =>
250+
statuses.filterNot(status => shouldFilterOut(status.getPath.getName))
251+
case None =>
252+
statuses
253+
}
254+
255+
val filteredStatuses = doFilter(statuses)
256+
val allLeafStatuses = {
257+
val (dirs, topLevelFiles) = filteredStatuses.partition(_.isDirectory)
258+
val nestedFiles: Seq[FileStatus] = contextOpt match {
259+
case Some(context) if dirs.size > parallelismThreshold =>
260+
parallelListLeafFiles(
261+
context,
262+
dirs.map(_.getPath),
263+
hadoopConf = hadoopConf,
264+
filter = filter,
265+
isRootLevel = false,
266+
ignoreMissingFiles = ignoreMissingFiles,
267+
ignoreLocality = ignoreLocality,
268+
filterFun = filterFun,
269+
parallelismThreshold = parallelismThreshold,
270+
parallelismMax = parallelismMax
271+
).flatMap(_._2)
272+
case _ =>
273+
dirs.flatMap { dir =>
274+
listLeafFiles(
275+
path = dir.getPath,
276+
hadoopConf = hadoopConf,
277+
filter = filter,
278+
contextOpt = contextOpt,
279+
ignoreMissingFiles = ignoreMissingFiles,
280+
ignoreLocality = ignoreLocality,
281+
isRootPath = false,
282+
filterFun = filterFun,
283+
parallelismThreshold = parallelismThreshold,
284+
parallelismMax = parallelismMax)
285+
}
286+
}
287+
val allFiles = topLevelFiles ++ nestedFiles
288+
if (filter != null) allFiles.filter(f => filter.accept(f.getPath)) else allFiles
289+
}
290+
291+
val missingFiles = mutable.ArrayBuffer.empty[String]
292+
val filteredLeafStatuses = doFilter(allLeafStatuses)
293+
val resolvedLeafStatuses = filteredLeafStatuses.flatMap {
294+
case f: LocatedFileStatus =>
295+
Some(f)
296+
297+
// NOTE:
298+
//
299+
// - Although S3/S3A/S3N file system can be quite slow for remote file metadata
300+
// operations, calling `getFileBlockLocations` does no harm here since these file system
301+
// implementations don't actually issue RPC for this method.
302+
//
303+
// - Here we are calling `getFileBlockLocations` in a sequential manner, but it should not
304+
// be a big deal since we always use to `parallelListLeafFiles` when the number of
305+
// paths exceeds threshold.
306+
case f if !ignoreLocality =>
307+
// The other constructor of LocatedFileStatus will call FileStatus.getPermission(),
308+
// which is very slow on some file system (RawLocalFileSystem, which is launch a
309+
// subprocess and parse the stdout).
310+
try {
311+
val locations = fs.getFileBlockLocations(f, 0, f.getLen).map { loc =>
312+
// Store BlockLocation objects to consume less memory
313+
if (loc.getClass == classOf[BlockLocation]) {
314+
loc
315+
} else {
316+
new BlockLocation(loc.getNames, loc.getHosts, loc.getOffset, loc.getLength)
317+
}
318+
}
319+
val lfs = new LocatedFileStatus(f.getLen, f.isDirectory, f.getReplication, f.getBlockSize,
320+
f.getModificationTime, 0, null, null, null, null, f.getPath, locations)
321+
if (f.isSymlink) {
322+
lfs.setSymlink(f.getSymlink)
323+
}
324+
Some(lfs)
325+
} catch {
326+
case _: FileNotFoundException if ignoreMissingFiles =>
327+
missingFiles += f.getPath.toString
328+
None
329+
}
330+
331+
case f => Some(f)
332+
}
333+
334+
if (missingFiles.nonEmpty) {
335+
logWarning(
336+
s"the following files were missing during file scan:\n ${missingFiles.mkString("\n ")}")
337+
}
338+
339+
resolvedLeafStatuses
340+
}
341+
// scalastyle:on argcount
342+
343+
/** A serializable variant of HDFS's BlockLocation. */
344+
private case class SerializableBlockLocation(
345+
names: Array[String],
346+
hosts: Array[String],
347+
offset: Long,
348+
length: Long)
349+
350+
/** A serializable variant of HDFS's FileStatus. */
351+
private case class SerializableFileStatus(
352+
path: String,
353+
length: Long,
354+
isDir: Boolean,
355+
blockReplication: Short,
356+
blockSize: Long,
357+
modificationTime: Long,
358+
accessTime: Long,
359+
blockLocations: Array[SerializableBlockLocation])
360+
}

sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -163,7 +163,7 @@ object CommandUtils extends Logging {
163163
.getConfString("hive.exec.stagingdir", ".hive-staging")
164164
val filter = new PathFilterIgnoreNonData(stagingDir)
165165
val sizes = InMemoryFileIndex.bulkListLeafFiles(paths.flatten,
166-
sparkSession.sessionState.newHadoopConf(), filter, sparkSession, areRootPaths = true).map {
166+
sparkSession.sessionState.newHadoopConf(), filter, sparkSession, isRootLevel = true).map {
167167
case (_, files) => files.map(_.getLen).sum
168168
}
169169
// the size is 0 where paths(i) is not defined and sizes(i) where it is defined

0 commit comments

Comments
 (0)