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

Reply via email to