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



##########
File path: core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.spark._
+import org.apache.spark.annotation.DeveloperApi
+import org.apache.spark.internal.Logging
+import org.apache.spark.metrics.source.HiveCatalogMetrics
+
+/**
+ * :: DeveloperApi ::
+ * Utility functions to simplify and speed-up file listing.
+ */
+@DeveloperApi
+object HadoopFSUtils extends Logging {
+  /**
+   * :: DeveloperApi ::
+   * 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
+   */
+  @DeveloperApi
+  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])] 
= {
+    HiveCatalogMetrics.incrementParallelListingJobCount(1)

Review comment:
       We could pass in a callback for listing metrics?

##########
File path: core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.spark._
+import org.apache.spark.annotation.Private
+import org.apache.spark.internal.Logging
+import org.apache.spark.metrics.source.HiveCatalogMetrics
+
+/**
+ * Utility functions to simplify and speed-up file listing.
+ */
+@Private

Review comment:
       Private maybe seems a bit too restrictive in scope, what about 
DeveloperAPI?

##########
File path: core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.spark._
+import org.apache.spark.annotation.DeveloperApi
+import org.apache.spark.internal.Logging
+import org.apache.spark.metrics.source.HiveCatalogMetrics
+
+/**
+ * :: DeveloperApi ::
+ * Utility functions to simplify and speed-up file listing.
+ */
+@DeveloperApi
+object HadoopFSUtils extends Logging {
+  /**
+   * :: DeveloperApi ::
+   * 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
+   */
+  @DeveloperApi
+  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])] 
= {
+    HiveCatalogMetrics.incrementParallelListingJobCount(1)
+
+    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) // TODO(holden): should we use 
jobgroup?
+      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,

Review comment:
       So the is/are is because listLeafFiles takes a single name and this 
method takes in a list of files.

##########
File path: core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.spark._
+import org.apache.spark.annotation.Private
+import org.apache.spark.internal.Logging
+import org.apache.spark.metrics.source.HiveCatalogMetrics
+
+/**
+ * Utility functions to simplify and speed-up file listing.
+ */
+@Private
+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.
+   *
+   * @param sc Spark context used to run parallel listing.
+   * @param paths Input paths to list
+   * @param hadoopConf Hadoop configuration
+   * @param filter Path filter used to exclude leaf files from result
+   * @param areSQLRootPaths Whether the input paths are SQL root paths
+   * @param ignoreMissingFiles Ignore missing files that happen during 
recursive listing
+   *                           (e.g., due to race conditions)
+   * @param ignoreLocality Whether to fetch data locality info when listing 
leaf files. If false,
+   *                       this will return `FileStatus` without 
`BlockLocation` info.
+   * @param parallelismThreshold The threshold to enable parallelism. If the 
number of input paths
+   *                             is smaller than this value, this will 
fallback to use
+   *                             sequential listing.
+   * @param parallelismMax The maximum parallelism for listing. If the number 
of input paths is
+   *                           larger than this value, parallelism will be 
throttled to this value
+   *                           to avoid generating too many tasks.
+   * @param filterFun Optional predicate on the leaf files. Files who failed 
the check will be
+   *                  excluded from the results
+   * @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,
+      parallelismThreshold: Int,
+      parallelismMax: Int,
+      filterFun: Option[String => Boolean] = None): Seq[(Path, 
Seq[FileStatus])] = {
+
+    // Short-circuits parallel listing when serial listing is likely to be 
faster.
+    if (paths.size <= parallelismThreshold) {
+      return paths.map { path =>
+        val leafFiles = listLeafFiles(
+          path,
+          hadoopConf,
+          filter,
+          Some(sc),
+          ignoreMissingFiles = ignoreMissingFiles,
+          ignoreLocality = ignoreLocality,
+          isSQLRootPath = areSQLRootPaths,
+          parallelismThreshold = parallelismThreshold,
+          parallelismDefault = parallelismMax,
+          filterFun = filterFun)
+        (path, leafFiles)
+      }
+    }
+
+    logInfo(s"Listing leaf files and directories in parallel under 
${paths.length} paths." +
+      s" The first several paths are: ${paths.take(10).mkString(", ")}.")
+    HiveCatalogMetrics.incrementParallelListingJobCount(1)
+
+    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, parallelismMax)
+
+    val previousJobDescription = 
sc.getLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION)
+    val statusMap = try {
+      val description = paths.size match {
+        case 0 =>
+          "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(
+              path = path,
+              hadoopConf = hadoopConf,
+              filter = filter,
+              contextOpt = None, // Can't execute parallel scans on workers
+              ignoreMissingFiles = ignoreMissingFiles,
+              ignoreLocality = ignoreLocality,
+              isSQLRootPath = areSQLRootPaths,
+              filterFun = filterFun,
+              parallelismThreshold = Int.MaxValue,
+              parallelismDefault = 0)
+            (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)
+    }
+  }
+
+  // scalastyle:off argcount
+  /**
+   * 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.
+   */
+  private def listLeafFiles(
+      path: Path,
+      hadoopConf: Configuration,
+      filter: PathFilter,
+      contextOpt: Option[SparkContext],
+      ignoreMissingFiles: Boolean,
+      ignoreLocality: Boolean,
+      isSQLRootPath: Boolean,
+      filterFun: Option[String => Boolean],
+      parallelismThreshold: Int,
+      parallelismDefault: Int): Seq[FileStatus] = {
+
+    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

Review comment:
       I think changing this in a follow up PR sounds fine to me. I'd like to 
us to use the faster method by default and fall back on exception, but that 
could be a follow on. Want to file a JIRA for broadening the scope of using the 
faster method?

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala
##########
@@ -147,32 +142,6 @@ class InMemoryFileIndex(
 
 object InMemoryFileIndex extends Logging {
 
-  /** A serializable variant of HDFS's BlockLocation. */
-  private case class SerializableBlockLocation(
-      names: Array[String],
-      hosts: Array[String],
-      offset: Long,
-      length: Long)
-
-  /** A serializable variant of HDFS's FileStatus. */
-  private case class SerializableFileStatus(

Review comment:
       Yeah a seperate JIRA is best.




----------------------------------------------------------------
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