Skip to content

[SPARK-52615][CORE] Replace File.mkdirs with Utils.createDirectory #51322

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 2 commits into
base: master
Choose a base branch
from

Conversation

pan3793
Copy link
Member

@pan3793 pan3793 commented Jun 30, 2025

What changes were proposed in this pull request?

We hit an issue that File.mkdirs() may occasionally fail with no error during the submission phase(we didn't configure spark.yarn.archive in that cluster so each submission requires packaging and uploading spark client jars, which cosumes a lot of disk IO), which was also mentioned in Utils.createDirectory

// SPARK-35907: The check was required by File.mkdirs() because it could sporadically
// fail silently. ...

25/06/27 19:12:17 WARN Client: Neither spark.yarn.jars nor spark.yarn.archive is set, falling back to uploading libraries under SPARK_HOME.
25/06/27 19:12:17 ERROR Utils: Failed to create dir in ./local. Ignoring this directory.
25/06/27 19:12:17 INFO Client: Deleted staging directory hdfs://<cluster>/user/<user>/.sparkStaging/application_1747844918192_28291290
Exception in thread "main" java.io.IOException: Failed to get a temp directory under [./local].
	at org.apache.spark.util.Utils$.getLocalDir(Utils.scala:896)
	at org.apache.spark.deploy.yarn.Client.prepareLocalResources(Client.scala:672)
	at org.apache.spark.deploy.yarn.Client.createContainerLaunchContext(Client.scala:1005)
	at org.apache.spark.deploy.yarn.Client.submitApplication(Client.scala:231)
	at org.apache.spark.deploy.yarn.Client.run(Client.scala:1352)
	at org.apache.spark.deploy.yarn.YarnClusterApplication.start(Client.scala:1800)
	at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:1019)
	at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:192)
	at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:215)
	at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:91)
	at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1107)
	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1116)
	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)

So I replaced File.mkdirs with Utils.createDirectory and deployed it into our internal cluster, no similar failures happens after then ... (not sure why, maybe the replaced NIO method is more robust?)

Additional context:
JDK-4227544 "design bug: File.mkdir(), etc. don't provide reason for failure" get closed with "Won't Fix"

It is too late to change this now. The new io framework will handle this situation better.

Why are the changes needed?

To achieve better error message reporting when creating a directory fails.

Does this PR introduce any user-facing change?

No.

How was this patch tested?

I made such a change in internal Spark, and deployed it to a busy YARN cluster, the submit process has been stable so far.

Was this patch authored or co-authored using generative AI tooling?

No.

@pan3793
Copy link
Member Author

pan3793 commented Jun 30, 2025

* Create a directory given the abstract pathname
* @return true, if the directory is successfully created; otherwise, return false.
*/
public static boolean createDirectory(File dir) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this created because public class JavaUtils is better than private[spark] trait SparkFileUtils?

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

To ensure that we fix every instances and to prevent a future regression, could you a Scalastyle and Checkstyle rule to ban .mkdirs pattern, @pan3793 ?

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1, LGTM (except the above Scalastyle and Checkstyle rule comment).

@LuciferYang
Copy link
Contributor

To ensure that we fix every instances and to prevent a future regression, could you a Scalastyle and Checkstyle rule to ban .mkdirs pattern, @pan3793 ?

+1

@pan3793
Copy link
Member Author

pan3793 commented Jun 30, 2025

could you a Scalastyle and Checkstyle rule to ban .mkdirs pattern

Thanks for your advice, I thought it, but I am afraid we can not do that because there are many places call org.apache.hadoop.fs.FileSystem.mkdirs

@@ -54,7 +54,7 @@ public void create() throws IOException {
localDirs[i] = JavaUtils.createDirectory(root, "spark").getAbsolutePath();

for (int p = 0; p < subDirsPerLocalDir; p ++) {
new File(localDirs[i], String.format("%02x", p)).mkdirs();
JavaUtils.createDirectory(new File(localDirs[i], String.format("%02x", p)));
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@dongjoon-hyun I moved the createDirectory to JavaUtils because of this place, since it's a test code, I can simply use java.nio.Files.createDirectory then we don't need to expose the method in JavaUtils publicly, please let me know which one you prefer.

Note: there are many places in Spark codebase directly call java.nio.Files.createDirectory

@dongjoon-hyun
Copy link
Member

dongjoon-hyun commented Jun 30, 2025

  1. Got it for the difficulty on banning.
  2. I prefer to use the existing SparkFileUtils instead of creating a new one. (But I'm fine with the new one too)

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants