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


##########
resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala:
##########
@@ -471,38 +470,40 @@ private[spark] class Client(
    * @param jars : the list of jars to upload
    * @return a list of directories to be preloaded
    * */
-  def directoriesToBePreloaded(jars: Seq[String]): mutable.Iterable[URI] = {
+  private[yarn] def directoriesToBePreloaded(jars: Seq[String]): List[URI] = {
     val directoryCounter = new HashMap[URI, Int]()
     jars.foreach { jar =>
       if (!Utils.isLocalUri(jar) && !new GlobPattern(jar).hasWildcard) {
-        val path = getQualifiedLocalPath(Utils.resolveURI(jar), hadoopConf)
-        val parentUri = path.getParent.toUri
+        val parentUri = new Path (Utils.resolveURI(jar)).getParent.toUri
         directoryCounter.update(parentUri, 
directoryCounter.getOrElse(parentUri, 0) + 1)
       }
     }
-    directoryCounter.filter(_._2 >= perDirectoryThreshold).keys
+    directoryCounter.filter(_._2 >= 
statCachePreloadDirectoryCountThreshold).keys.toList
   }
 
   /**
    * Preload the statCache with file status. For each directory in the list, 
we will list all
    * files from that directory and add them to the statCache.
-   *
-   * @param fs : the target file system
-   * @param statCache : stat cache to be preloaded with fileStatus
+   * @param testFS : the file system to be used for testing only (optional)
+   * @return A preloaded statCache with fileStatus
    */
-  def statCachePreload(fs: FileSystem, statCache: Map[URI, FileStatus]): Unit 
= {
-    logDebug("Preload the following directories")
-    val directory = sparkConf.get(SPARK_JARS)
-      .map(directoriesToBePreloaded)
-      .getOrElse(ArrayBuffer.empty[URI])
-
-    directory.foreach { dir =>
-      fs.listStatus(new Path(dir)).filter(_.isFile()).foreach { fileStatus =>
-        val uri = fileStatus.getPath.toUri
-        logDebug(s"add ${uri} plan to added to stat cache.")
-        statCache.put(uri, fileStatus)
-      }
+  private[yarn] def getPreloadedStatCache(testFS: Option[FileSystem]):
+  HashMap[URI, FileStatus] = {
+    val statCache = HashMap[URI, FileStatus]()
+    val jars = sparkConf.get(SPARK_JARS)
+    val directories = 
jars.map(directoriesToBePreloaded).getOrElse(ArrayBuffer.empty[URI])
+
+    directories.foreach { dir =>
+      testFS.getOrElse(FileSystem.get(dir, hadoopConf)).listStatus(new 
Path(dir)).filter(_.isFile())

Review Comment:
   The `testFS` feels a little awkward to me. I didn't see any super clean way 
to do it without some major refactoring to `yarn.Client`, but how about this?
   ```suggestion
      * @param fsLookup: Function for looking up an FS based on a URI; override 
for testing
      * @return A preloaded statCache with fileStatus
      */
     private[yarn] def getPreloadedStatCache(
         fsLookup: URI => FileSystem = FileSystem.get(_, hadoopConf)): 
HashMap[URI, FileStatus] = {
       val statCache = HashMap[URI, FileStatus]()
       val jars = sparkConf.get(SPARK_JARS)
       val directories = 
jars.map(directoriesToBePreloaded).getOrElse(ArrayBuffer.empty[URI])
   
       directories.foreach { dir =>
         fsLookup(dir).listStatus(new Path(dir)).filter(_.isFile())
   ```



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