mridulm commented on code in PR #42357:
URL: https://github.com/apache/spark/pull/42357#discussion_r1320755309


##########
resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala:
##########
@@ -485,7 +530,16 @@ private[spark] class Client(
     val localResources = HashMap[String, LocalResource]()
     FileSystem.mkdirs(fs, destDir, new FsPermission(STAGING_DIR_PERMISSION))
 
-    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
+    // If preload is enabled, preload the statCache with the files in the 
directories
+    val statCache = if (statCachePreloadEnabled) {
+      // Consider only following configurations, as they involve the 
distribution of multiple files
+      val files = sparkConf.get(SPARK_JARS).orNull ++ 
sparkConf.get(JARS_TO_DISTRIBUTE) ++

Review Comment:
   `sparkConf.get(SPARK_JARS).orNull` -> 
`sparkConf.get(SPARK_JARS).getOrElse(Nil)`
   Can you add a condition to the test where `SPARK_JARS` is empty to make sure 
this is caught ?
   
   



##########
resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala:
##########
@@ -462,6 +462,30 @@ package object config extends Logging {
     .stringConf
     .createWithDefault("yarn.io/fpga")
 
+  private[spark] val YARN_CLIENT_STAT_CACHE_PRELOAD_ENABLED =
+    ConfigBuilder("spark.yarn.client.statCache.preload.enabled")
+    .doc("This configuration enables statCache to be preloaded at YARN client 
side. This feature " +
+      "analyzes the pattern of resources paths, and if multiple resources 
shared the same parent " +
+      "directory, a single <code>listStatus</code> will be invoked on the 
parent directory " +
+      "instead of multiple <code>getFileStatus</code> performed on each 
individual resources. " +
+      "If most resources from a small set of directories, this can 
substantially improve job " +

Review Comment:
   ```suggestion
         "If most are resources from a small set of directories, this can 
improve job " +
   ```



##########
resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala:
##########
@@ -462,6 +462,30 @@ package object config extends Logging {
     .stringConf
     .createWithDefault("yarn.io/fpga")
 
+  private[spark] val YARN_CLIENT_STAT_CACHE_PRELOAD_ENABLED =
+    ConfigBuilder("spark.yarn.client.statCache.preload.enabled")
+    .doc("This configuration enables statCache to be preloaded at YARN client 
side. This feature " +

Review Comment:
   ```suggestion
       .doc("Enables statCache to be preloaded at YARN client side. This 
feature " +
   ```



##########
resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala:
##########
@@ -458,6 +461,48 @@ private[spark] class Client(
     new Path(resolvedDestDir, qualifiedDestPath.getName())
   }
 
+  /**
+   * For each non-local and non-glob resource, we will count its parent 
directory. If its
+   * frequency is larger than the threshold specified by
+   * spark.yarn.client.statCache.preloaded.perDirectoryThreshold, the 
corresponding file status
+   * from the directory will be preloaded.
+   *
+   * @param jars : the list of jars to upload
+   * @return a hashmap contains directories to be preloaded and all file names 
in that directory
+   */
+  private[yarn] def directoriesToBePreloaded(jars: Seq[String]): HashMap[URI, 
HashSet[String]] = {

Review Comment:
   `jars` -> `files`



##########
resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala:
##########
@@ -462,6 +462,30 @@ package object config extends Logging {
     .stringConf
     .createWithDefault("yarn.io/fpga")
 
+  private[spark] val YARN_CLIENT_STAT_CACHE_PRELOAD_ENABLED =
+    ConfigBuilder("spark.yarn.client.statCache.preload.enabled")
+    .doc("This configuration enables statCache to be preloaded at YARN client 
side. This feature " +
+      "analyzes the pattern of resources paths, and if multiple resources 
shared the same parent " +
+      "directory, a single <code>listStatus</code> will be invoked on the 
parent directory " +
+      "instead of multiple <code>getFileStatus</code> performed on each 
individual resources. " +

Review Comment:
   nit:
   ```suggestion
         "instead of multiple <code>getFileStatus</code> on individual 
resources. " +
   ```



##########
resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala:
##########
@@ -462,6 +462,30 @@ package object config extends Logging {
     .stringConf
     .createWithDefault("yarn.io/fpga")
 
+  private[spark] val YARN_CLIENT_STAT_CACHE_PRELOAD_ENABLED =
+    ConfigBuilder("spark.yarn.client.statCache.preload.enabled")
+    .doc("This configuration enables statCache to be preloaded at YARN client 
side. This feature " +
+      "analyzes the pattern of resources paths, and if multiple resources 
shared the same parent " +
+      "directory, a single <code>listStatus</code> will be invoked on the 
parent directory " +
+      "instead of multiple <code>getFileStatus</code> performed on each 
individual resources. " +
+      "If most resources from a small set of directories, this can 
substantially improve job " +
+      "submission time. Enabling this feature may potentially increase client 
memory overhead.")
+    .version("4.0.0")
+    .booleanConf
+    .createWithDefault(false)
+
+  private[spark] val YARN_CLIENT_STAT_CACHE_PRELOAD_PER_DIRECTORY_THRESHOLD =
+    ConfigBuilder("spark.yarn.client.statCache.preload.perDirectoryThreshold")
+      .doc("This configuration defines the threshold for the number of 
resources in a directory " +
+        "that triggers the activation of the statCache preloading. When the 
count of individual " +
+        "resources specified by <code>spark.yarn.jars</code> within a 
directory is no less than " +
+        "this threshold, the statCache preloading for that directory will be 
activated. It's " +
+        "important to note that this configuration will only take effect when 
the " +
+        "<code>spark.yarn.client.statCache.preloaded.enabled</code> option is 
enabled.")

Review Comment:
   ```suggestion
         .doc("Minimum resource count in a directory to trigger statCache 
preloading when " +
           "submitting an application. If the number of resources in a 
directory, without " +
           "any wildcards, equals or exceeds this threshold, the statCache for 
that directory " +
           "will be preloaded. This configuration will only take effect when " +
           "<code>spark.yarn.client.statCache.preloaded.enabled</code> option 
is enabled.")
   ```



##########
resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala:
##########
@@ -462,6 +462,30 @@ package object config extends Logging {
     .stringConf
     .createWithDefault("yarn.io/fpga")
 
+  private[spark] val YARN_CLIENT_STAT_CACHE_PRELOAD_ENABLED =
+    ConfigBuilder("spark.yarn.client.statCache.preload.enabled")
+    .doc("This configuration enables statCache to be preloaded at YARN client 
side. This feature " +
+      "analyzes the pattern of resources paths, and if multiple resources 
shared the same parent " +

Review Comment:
   ```suggestion
         "analyzes the pattern of resources paths, and if multiple resources 
share the same parent " +
   ```



##########
resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala:
##########
@@ -462,6 +462,30 @@ package object config extends Logging {
     .stringConf
     .createWithDefault("yarn.io/fpga")
 
+  private[spark] val YARN_CLIENT_STAT_CACHE_PRELOAD_ENABLED =
+    ConfigBuilder("spark.yarn.client.statCache.preload.enabled")
+    .doc("This configuration enables statCache to be preloaded at YARN client 
side. This feature " +
+      "analyzes the pattern of resources paths, and if multiple resources 
shared the same parent " +
+      "directory, a single <code>listStatus</code> will be invoked on the 
parent directory " +
+      "instead of multiple <code>getFileStatus</code> performed on each 
individual resources. " +
+      "If most resources from a small set of directories, this can 
substantially improve job " +
+      "submission time. Enabling this feature may potentially increase client 
memory overhead.")

Review Comment:
   How substantial is the impact on memory overhead ?
   I am assuming this is applicable when an extremely large number of files are 
present in the parent directory ?
   If yes, did you expose using alternatives `listStatus` which might help ?
   



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

To unsubscribe, e-mail: [email protected]

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