steveloughran commented on a change in pull request #29179: URL: https://github.com/apache/spark/pull/29179#discussion_r458974067
########## File path: core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala ########## @@ -0,0 +1,314 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.io.FileNotFoundException + +import scala.collection.mutable + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs._ +import org.apache.hadoop.fs.viewfs.ViewFileSystem +import org.apache.hadoop.hdfs.DistributedFileSystem +import org.apache.hadoop.mapred.{FileInputFormat, JobConf} + +import org.apache.spark.SparkContext +import org.apache.spark.internal.Logging + +private[spark] object HadoopFSUtils extends Logging { + /** + * Lists a collection of paths recursively. Picks the listing strategy adaptively depending + * on the number of paths to list. + * + * This may only be called on the driver. + * + * @return for each input path, the set of discovered files for the path + */ + + def parallelListLeafFiles(sc: SparkContext, paths: Seq[Path], hadoopConf: Configuration, + filter: PathFilter, areSQLRootPaths: Boolean, ignoreMissingFiles: Boolean, + ignoreLocality: Boolean, maxParallelism: Int, + filterFun: Option[String => Boolean] = None): Seq[(Path, Seq[FileStatus])] = { + + val serializableConfiguration = new SerializableConfiguration(hadoopConf) + val serializedPaths = paths.map(_.toString) + + // Set the number of parallelism to prevent following file listing from generating many tasks + // in case of large #defaultParallelism. + val numParallelism = Math.min(paths.size, maxParallelism) + + val previousJobDescription = sc.getLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION) + val statusMap = try { + val description = paths.size match { + case 0 => + s"Listing leaf files and directories 0 paths" + case 1 => + s"Listing leaf files and directories for 1 path:<br/>${paths(0)}" + case s => + s"Listing leaf files and directories for $s paths:<br/>${paths(0)}, ..." + } + sc.setJobDescription(description) + sc + .parallelize(serializedPaths, numParallelism) + .mapPartitions { pathStrings => + val hadoopConf = serializableConfiguration.value + pathStrings.map(new Path(_)).toSeq.map { path => + val leafFiles = listLeafFiles( + contextOpt = None, // Can't execute parallel scans on workers + path = path, + hadoopConf = hadoopConf, + filter = filter, + ignoreMissingFiles = ignoreMissingFiles, + ignoreLocality = ignoreLocality, + isSQLRootPath = areSQLRootPaths, + filterFun = filterFun, + parallelScanCallBack = None // Can't execute parallel scans on workers + ) + (path, leafFiles) + }.iterator + }.map { case (path, statuses) => + val serializableStatuses = statuses.map { status => + // Turn FileStatus into SerializableFileStatus so we can send it back to the driver + val blockLocations = status match { + case f: LocatedFileStatus => + f.getBlockLocations.map { loc => + SerializableBlockLocation( + loc.getNames, + loc.getHosts, + loc.getOffset, + loc.getLength) + } + + case _ => + Array.empty[SerializableBlockLocation] + } + + SerializableFileStatus( Review comment: been serializable for a while (HADOOP-13895), but this should be less brittle. ########## File path: core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala ########## @@ -0,0 +1,314 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.io.FileNotFoundException + +import scala.collection.mutable + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs._ +import org.apache.hadoop.fs.viewfs.ViewFileSystem +import org.apache.hadoop.hdfs.DistributedFileSystem +import org.apache.hadoop.mapred.{FileInputFormat, JobConf} + +import org.apache.spark.SparkContext +import org.apache.spark.internal.Logging + +private[spark] object HadoopFSUtils extends Logging { + /** + * Lists a collection of paths recursively. Picks the listing strategy adaptively depending + * on the number of paths to list. + * + * This may only be called on the driver. + * + * @return for each input path, the set of discovered files for the path + */ + + def parallelListLeafFiles(sc: SparkContext, paths: Seq[Path], hadoopConf: Configuration, + filter: PathFilter, areSQLRootPaths: Boolean, ignoreMissingFiles: Boolean, + ignoreLocality: Boolean, maxParallelism: Int, + filterFun: Option[String => Boolean] = None): Seq[(Path, Seq[FileStatus])] = { + + val serializableConfiguration = new SerializableConfiguration(hadoopConf) + val serializedPaths = paths.map(_.toString) + + // Set the number of parallelism to prevent following file listing from generating many tasks + // in case of large #defaultParallelism. + val numParallelism = Math.min(paths.size, maxParallelism) + + val previousJobDescription = sc.getLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION) + val statusMap = try { + val description = paths.size match { + case 0 => + s"Listing leaf files and directories 0 paths" + case 1 => + s"Listing leaf files and directories for 1 path:<br/>${paths(0)}" + case s => + s"Listing leaf files and directories for $s paths:<br/>${paths(0)}, ..." + } + sc.setJobDescription(description) + sc + .parallelize(serializedPaths, numParallelism) + .mapPartitions { pathStrings => + val hadoopConf = serializableConfiguration.value + pathStrings.map(new Path(_)).toSeq.map { path => + val leafFiles = listLeafFiles( + contextOpt = None, // Can't execute parallel scans on workers + path = path, + hadoopConf = hadoopConf, + filter = filter, + ignoreMissingFiles = ignoreMissingFiles, + ignoreLocality = ignoreLocality, + isSQLRootPath = areSQLRootPaths, + filterFun = filterFun, + parallelScanCallBack = None // Can't execute parallel scans on workers + ) + (path, leafFiles) + }.iterator + }.map { case (path, statuses) => + val serializableStatuses = statuses.map { status => + // Turn FileStatus into SerializableFileStatus so we can send it back to the driver + val blockLocations = status match { + case f: LocatedFileStatus => + f.getBlockLocations.map { loc => + SerializableBlockLocation( + loc.getNames, + loc.getHosts, + loc.getOffset, + loc.getLength) + } + + case _ => + Array.empty[SerializableBlockLocation] + } + + SerializableFileStatus( + status.getPath.toString, Review comment: Again, serializable (HADOOP-13519). Otherwise, use a URI rather than string as you get round trip guarantees that toString doesn't make ########## File path: core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala ########## @@ -0,0 +1,314 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.io.FileNotFoundException + +import scala.collection.mutable + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs._ +import org.apache.hadoop.fs.viewfs.ViewFileSystem +import org.apache.hadoop.hdfs.DistributedFileSystem Review comment: referencing this in the code means it may not work on installations without hadoop-hdfs on the classpath, which may include HD/I as well as spark standalone ########## File path: core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala ########## @@ -0,0 +1,314 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.io.FileNotFoundException + +import scala.collection.mutable + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs._ +import org.apache.hadoop.fs.viewfs.ViewFileSystem +import org.apache.hadoop.hdfs.DistributedFileSystem +import org.apache.hadoop.mapred.{FileInputFormat, JobConf} + +import org.apache.spark.SparkContext +import org.apache.spark.internal.Logging + +private[spark] object HadoopFSUtils extends Logging { + /** + * Lists a collection of paths recursively. Picks the listing strategy adaptively depending + * on the number of paths to list. + * + * This may only be called on the driver. + * + * @return for each input path, the set of discovered files for the path + */ + + def parallelListLeafFiles(sc: SparkContext, paths: Seq[Path], hadoopConf: Configuration, + filter: PathFilter, areSQLRootPaths: Boolean, ignoreMissingFiles: Boolean, + ignoreLocality: Boolean, maxParallelism: Int, + filterFun: Option[String => Boolean] = None): Seq[(Path, Seq[FileStatus])] = { + + val serializableConfiguration = new SerializableConfiguration(hadoopConf) + val serializedPaths = paths.map(_.toString) + + // Set the number of parallelism to prevent following file listing from generating many tasks + // in case of large #defaultParallelism. + val numParallelism = Math.min(paths.size, maxParallelism) + + val previousJobDescription = sc.getLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION) + val statusMap = try { + val description = paths.size match { + case 0 => + s"Listing leaf files and directories 0 paths" + case 1 => + s"Listing leaf files and directories for 1 path:<br/>${paths(0)}" + case s => + s"Listing leaf files and directories for $s paths:<br/>${paths(0)}, ..." + } + sc.setJobDescription(description) + sc + .parallelize(serializedPaths, numParallelism) + .mapPartitions { pathStrings => + val hadoopConf = serializableConfiguration.value + pathStrings.map(new Path(_)).toSeq.map { path => + val leafFiles = listLeafFiles( + contextOpt = None, // Can't execute parallel scans on workers + path = path, + hadoopConf = hadoopConf, + filter = filter, + ignoreMissingFiles = ignoreMissingFiles, + ignoreLocality = ignoreLocality, + isSQLRootPath = areSQLRootPaths, + filterFun = filterFun, + parallelScanCallBack = None // Can't execute parallel scans on workers + ) + (path, leafFiles) + }.iterator + }.map { case (path, statuses) => + val serializableStatuses = statuses.map { status => + // Turn FileStatus into SerializableFileStatus so we can send it back to the driver + val blockLocations = status match { + case f: LocatedFileStatus => + f.getBlockLocations.map { loc => + SerializableBlockLocation( + loc.getNames, + loc.getHosts, + loc.getOffset, + loc.getLength) + } + + case _ => + Array.empty[SerializableBlockLocation] + } + + SerializableFileStatus( + status.getPath.toString, + status.getLen, + status.isDirectory, + status.getReplication, + status.getBlockSize, + status.getModificationTime, + status.getAccessTime, + blockLocations) + } + (path.toString, serializableStatuses) + }.collect() + } finally { + sc.setJobDescription(previousJobDescription) + } + + // turn SerializableFileStatus back to Status + statusMap.map { case (path, serializableStatuses) => + val statuses = serializableStatuses.map { f => + val blockLocations = f.blockLocations.map { loc => + new BlockLocation(loc.names, loc.hosts, loc.offset, loc.length) + } + new LocatedFileStatus( + new FileStatus( + f.length, f.isDir, f.blockReplication, f.blockSize, f.modificationTime, + new Path(f.path)), + blockLocations) + } + (new Path(path), statuses) + } + } + /** + * Lists a single filesystem path recursively. If a Sparkcontext object is specified, this + * function may launch Spark jobs to parallelize listing based on parallelismThreshold. + * + * If sessionOpt is None, this may be called on executors. + * + * @return all children of path that match the specified filter. + */ + // scalastyle:off argcount + def listLeafFiles( + path: Path, + hadoopConf: Configuration, + filter: PathFilter, + contextOpt: Option[SparkContext], + ignoreMissingFiles: Boolean, + ignoreLocality: Boolean, + isSQLRootPath: Boolean, + parallelScanCallBack: Option[() => Unit], + filterFun: Option[String => Boolean], + parallelismThreshold: Int = 1, + maxParallelism: Int = 1): Seq[FileStatus] = { + // scalastyle:on argcount + + logTrace(s"Listing $path") + val fs = path.getFileSystem(hadoopConf) + + // Note that statuses only include FileStatus for the files and dirs directly under path, + // and does not include anything else recursively. + val statuses: Array[FileStatus] = try { + fs match { + // DistributedFileSystem overrides listLocatedStatus to make 1 single call to namenode + // to retrieve the file status with the file block location. The reason to still fallback + // to listStatus is because the default implementation would potentially throw a + // FileNotFoundException which is better handled by doing the lookups manually below. + case (_: DistributedFileSystem | _: ViewFileSystem) if !ignoreLocality => + val remoteIter = fs.listLocatedStatus(path) + new Iterator[LocatedFileStatus]() { + def next(): LocatedFileStatus = remoteIter.next + def hasNext(): Boolean = remoteIter.hasNext + }.toArray + case _ => fs.listStatus(path) + } + } catch { + // If we are listing a root path for SQL (e.g. a top level directory of a table), we need to + // ignore FileNotFoundExceptions during this root level of the listing because + // + // (a) certain code paths might construct an InMemoryFileIndex with root paths that + // might not exist (i.e. not all callers are guaranteed to have checked + // path existence prior to constructing InMemoryFileIndex) and, + // (b) we need to ignore deleted root paths during REFRESH TABLE, otherwise we break + // existing behavior and break the ability drop SessionCatalog tables when tables' + // root directories have been deleted (which breaks a number of Spark's own tests). + // + // If we are NOT listing a root path then a FileNotFoundException here means that the + // directory was present in a previous level of file listing but is absent in this + // listing, likely indicating a race condition (e.g. concurrent table overwrite or S3 + // list inconsistency). + // + // The trade-off in supporting existing behaviors / use-cases is that we won't be + // able to detect race conditions involving root paths being deleted during + // InMemoryFileIndex construction. However, it's still a net improvement to detect and + // fail-fast on the non-root cases. For more info see the SPARK-27676 review discussion. + case _: FileNotFoundException if isSQLRootPath || ignoreMissingFiles => + logWarning(s"The directory $path was not found. Was it deleted very recently?") + Array.empty[FileStatus] + } + + val filteredStatuses = filterFun match { + case Some(shouldFilterOut) => + statuses.filterNot(status => shouldFilterOut(status.getPath.getName)) + case None => + statuses + } + + val allLeafStatuses = { + val (dirs, topLevelFiles) = filteredStatuses.partition(_.isDirectory) + val nestedFiles: Seq[FileStatus] = contextOpt match { + case Some(context) if dirs.size > parallelismThreshold => + parallelScanCallBack.foreach(f => f()) + parallelListLeafFiles( + context, + dirs.map(_.getPath), + hadoopConf = hadoopConf, + filter = filter, + areSQLRootPaths = false, + ignoreMissingFiles = ignoreMissingFiles, + ignoreLocality = ignoreLocality, + filterFun = filterFun, + maxParallelism = maxParallelism + ).flatMap(_._2) + case _ => + dirs.flatMap { dir => + listLeafFiles( + path = dir.getPath, + hadoopConf = hadoopConf, + filter = filter, + contextOpt = contextOpt, + ignoreMissingFiles = ignoreMissingFiles, + ignoreLocality = ignoreLocality, + isSQLRootPath = false, + parallelScanCallBack = parallelScanCallBack, + filterFun = filterFun) + } + } + val allFiles = topLevelFiles ++ nestedFiles + if (filter != null) allFiles.filter(f => filter.accept(f.getPath)) else allFiles + } + + val missingFiles = mutable.ArrayBuffer.empty[String] + val filteredLeafStatuses = filterFun match { + case Some(shouldFilterOut) => + allLeafStatuses.filterNot( + status => shouldFilterOut(status.getPath.getName)) + case None => + allLeafStatuses + } + val resolvedLeafStatuses = filteredLeafStatuses.flatMap { + case f: LocatedFileStatus => + Some(f) + + // NOTE: + // + // - Although S3/S3A/S3N file system can be quite slow for remote file metadata Review comment: yeah. getFileStatus is the expensive one; list calls more efficient as you can get lower latency as well as >1 response back ########## File path: core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala ########## @@ -0,0 +1,314 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.io.FileNotFoundException + +import scala.collection.mutable + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs._ +import org.apache.hadoop.fs.viewfs.ViewFileSystem +import org.apache.hadoop.hdfs.DistributedFileSystem +import org.apache.hadoop.mapred.{FileInputFormat, JobConf} + +import org.apache.spark.SparkContext +import org.apache.spark.internal.Logging + +private[spark] object HadoopFSUtils extends Logging { + /** + * Lists a collection of paths recursively. Picks the listing strategy adaptively depending + * on the number of paths to list. + * + * This may only be called on the driver. + * + * @return for each input path, the set of discovered files for the path + */ + + def parallelListLeafFiles(sc: SparkContext, paths: Seq[Path], hadoopConf: Configuration, + filter: PathFilter, areSQLRootPaths: Boolean, ignoreMissingFiles: Boolean, + ignoreLocality: Boolean, maxParallelism: Int, + filterFun: Option[String => Boolean] = None): Seq[(Path, Seq[FileStatus])] = { + + val serializableConfiguration = new SerializableConfiguration(hadoopConf) + val serializedPaths = paths.map(_.toString) + + // Set the number of parallelism to prevent following file listing from generating many tasks + // in case of large #defaultParallelism. + val numParallelism = Math.min(paths.size, maxParallelism) + + val previousJobDescription = sc.getLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION) + val statusMap = try { + val description = paths.size match { + case 0 => + s"Listing leaf files and directories 0 paths" + case 1 => + s"Listing leaf files and directories for 1 path:<br/>${paths(0)}" + case s => + s"Listing leaf files and directories for $s paths:<br/>${paths(0)}, ..." + } + sc.setJobDescription(description) + sc + .parallelize(serializedPaths, numParallelism) + .mapPartitions { pathStrings => + val hadoopConf = serializableConfiguration.value + pathStrings.map(new Path(_)).toSeq.map { path => + val leafFiles = listLeafFiles( + contextOpt = None, // Can't execute parallel scans on workers + path = path, + hadoopConf = hadoopConf, + filter = filter, + ignoreMissingFiles = ignoreMissingFiles, + ignoreLocality = ignoreLocality, + isSQLRootPath = areSQLRootPaths, + filterFun = filterFun, + parallelScanCallBack = None // Can't execute parallel scans on workers + ) + (path, leafFiles) + }.iterator + }.map { case (path, statuses) => + val serializableStatuses = statuses.map { status => + // Turn FileStatus into SerializableFileStatus so we can send it back to the driver + val blockLocations = status match { + case f: LocatedFileStatus => + f.getBlockLocations.map { loc => + SerializableBlockLocation( + loc.getNames, + loc.getHosts, + loc.getOffset, + loc.getLength) + } + + case _ => + Array.empty[SerializableBlockLocation] + } + + SerializableFileStatus( + status.getPath.toString, + status.getLen, + status.isDirectory, + status.getReplication, + status.getBlockSize, + status.getModificationTime, + status.getAccessTime, + blockLocations) + } + (path.toString, serializableStatuses) + }.collect() + } finally { + sc.setJobDescription(previousJobDescription) + } + + // turn SerializableFileStatus back to Status + statusMap.map { case (path, serializableStatuses) => + val statuses = serializableStatuses.map { f => + val blockLocations = f.blockLocations.map { loc => + new BlockLocation(loc.names, loc.hosts, loc.offset, loc.length) + } + new LocatedFileStatus( + new FileStatus( + f.length, f.isDir, f.blockReplication, f.blockSize, f.modificationTime, + new Path(f.path)), + blockLocations) + } + (new Path(path), statuses) + } + } + /** + * Lists a single filesystem path recursively. If a Sparkcontext object is specified, this + * function may launch Spark jobs to parallelize listing based on parallelismThreshold. + * + * If sessionOpt is None, this may be called on executors. + * + * @return all children of path that match the specified filter. + */ + // scalastyle:off argcount + def listLeafFiles( + path: Path, + hadoopConf: Configuration, + filter: PathFilter, + contextOpt: Option[SparkContext], + ignoreMissingFiles: Boolean, + ignoreLocality: Boolean, + isSQLRootPath: Boolean, + parallelScanCallBack: Option[() => Unit], + filterFun: Option[String => Boolean], + parallelismThreshold: Int = 1, + maxParallelism: Int = 1): Seq[FileStatus] = { + // scalastyle:on argcount + + logTrace(s"Listing $path") + val fs = path.getFileSystem(hadoopConf) + + // Note that statuses only include FileStatus for the files and dirs directly under path, + // and does not include anything else recursively. + val statuses: Array[FileStatus] = try { + fs match { + // DistributedFileSystem overrides listLocatedStatus to make 1 single call to namenode + // to retrieve the file status with the file block location. The reason to still fallback + // to listStatus is because the default implementation would potentially throw a + // FileNotFoundException which is better handled by doing the lookups manually below. + case (_: DistributedFileSystem | _: ViewFileSystem) if !ignoreLocality => + val remoteIter = fs.listLocatedStatus(path) Review comment: this is already optimised for in S3A with WiP on overlapping page fetch with returning current results, ABFS could do the same. Looking at the main filesystems, it's only local FS and the azure stores which don't override it ########## File path: core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala ########## @@ -0,0 +1,314 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.io.FileNotFoundException + +import scala.collection.mutable + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs._ +import org.apache.hadoop.fs.viewfs.ViewFileSystem +import org.apache.hadoop.hdfs.DistributedFileSystem +import org.apache.hadoop.mapred.{FileInputFormat, JobConf} + +import org.apache.spark.SparkContext +import org.apache.spark.internal.Logging + +private[spark] object HadoopFSUtils extends Logging { + /** + * Lists a collection of paths recursively. Picks the listing strategy adaptively depending + * on the number of paths to list. + * + * This may only be called on the driver. + * + * @return for each input path, the set of discovered files for the path + */ + + def parallelListLeafFiles(sc: SparkContext, paths: Seq[Path], hadoopConf: Configuration, + filter: PathFilter, areSQLRootPaths: Boolean, ignoreMissingFiles: Boolean, + ignoreLocality: Boolean, maxParallelism: Int, + filterFun: Option[String => Boolean] = None): Seq[(Path, Seq[FileStatus])] = { + + val serializableConfiguration = new SerializableConfiguration(hadoopConf) + val serializedPaths = paths.map(_.toString) + + // Set the number of parallelism to prevent following file listing from generating many tasks + // in case of large #defaultParallelism. + val numParallelism = Math.min(paths.size, maxParallelism) + + val previousJobDescription = sc.getLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION) + val statusMap = try { + val description = paths.size match { + case 0 => + s"Listing leaf files and directories 0 paths" + case 1 => + s"Listing leaf files and directories for 1 path:<br/>${paths(0)}" + case s => + s"Listing leaf files and directories for $s paths:<br/>${paths(0)}, ..." + } + sc.setJobDescription(description) + sc + .parallelize(serializedPaths, numParallelism) + .mapPartitions { pathStrings => + val hadoopConf = serializableConfiguration.value + pathStrings.map(new Path(_)).toSeq.map { path => + val leafFiles = listLeafFiles( + contextOpt = None, // Can't execute parallel scans on workers + path = path, + hadoopConf = hadoopConf, + filter = filter, + ignoreMissingFiles = ignoreMissingFiles, + ignoreLocality = ignoreLocality, + isSQLRootPath = areSQLRootPaths, + filterFun = filterFun, + parallelScanCallBack = None // Can't execute parallel scans on workers + ) + (path, leafFiles) Review comment: be interesting to include the stats on how long the listing took (overall, #of files) so you could easily get some aggregate values to look @ when tuning performance here/in the FS ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org