You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/09/14 02:35:46 UTC

[GitHub] [spark] HyukjinKwon commented on a change in pull request #29471: [SPARK-32381][CORE][SQL] Explore allowing parallel listing & non-location sensitive listing in core

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



##########
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[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.
+   *
+   * @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 isRootLevel Whether the input paths are at the root level, i.e., they are the root
+   *                    paths as opposed to nested paths encountered during recursive calls of this.
+   * @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,
+      isRootLevel: 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,
+          isRootPath = isRootLevel,
+          parallelismThreshold = parallelismThreshold,
+          parallelismMax = 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,
+              isRootPath = isRootLevel,
+              filterFun = filterFun,
+              parallelismThreshold = Int.MaxValue,
+              parallelismMax = 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,
+      isRootPath: Boolean,
+      filterFun: Option[String => Boolean],
+      parallelismThreshold: Int,

Review comment:
       This seems a new parameter that did not exist before. Why do we need this? If people want parallelized listing, people can invoke `parallelListLeafFiles` above.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org