holdenk commented on a change in pull request #29179:
URL: https://github.com/apache/spark/pull/29179#discussion_r459756204



##########
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:
       Makes sense. So maybe exposing this as a util function + trait + sample 
allowing people to selictively override parts would be better then yeah?




----------------------------------------------------------------
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:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to