This is an automated email from the ASF dual-hosted git repository.

dongjoon pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new 7c6693ddec00 [SPARK-53031][CORE] Support `getFile` in `SparkFileUtils`
7c6693ddec00 is described below

commit 7c6693ddec00c32d50d254817a7f09a993d71017
Author: Dongjoon Hyun <dongj...@apache.org>
AuthorDate: Wed Jul 30 20:04:43 2025 -0700

    [SPARK-53031][CORE] Support `getFile` in `SparkFileUtils`
    
    ### What changes were proposed in this pull request?
    
    This PR aims to support `SparkFileUtils.getFile`
    
    ### Why are the changes needed?
    
    To improve Spark file utility functions.
    
    ### Does this PR introduce _any_ user-facing change?
    
    No behavior change.
    
    ### How was this patch tested?
    
    Pass the CIs.
    
    ### Was this patch authored or co-authored using generative AI tooling?
    
    No.
    
    Closes #51733 from dongjoon-hyun/SPARK-53031.
    
    Authored-by: Dongjoon Hyun <dongj...@apache.org>
    Signed-off-by: Dongjoon Hyun <dongj...@apache.org>
---
 .../scala/org/apache/spark/util/SparkFileUtils.scala     | 16 +++++++++++++++-
 .../org/apache/spark/util/logging/DriverLogger.scala     |  7 +++----
 .../spark/deploy/history/FsHistoryProviderSuite.scala    |  7 +++----
 .../apache/spark/util/logging/DriverLoggerSuite.scala    |  7 +++----
 scalastyle-config.xml                                    |  5 +++++
 5 files changed, 29 insertions(+), 13 deletions(-)

diff --git 
a/common/utils/src/main/scala/org/apache/spark/util/SparkFileUtils.scala 
b/common/utils/src/main/scala/org/apache/spark/util/SparkFileUtils.scala
index 22f03df1b269..0c89aa3e1444 100644
--- a/common/utils/src/main/scala/org/apache/spark/util/SparkFileUtils.scala
+++ b/common/utils/src/main/scala/org/apache/spark/util/SparkFileUtils.scala
@@ -18,7 +18,7 @@ package org.apache.spark.util
 
 import java.io.File
 import java.net.{URI, URISyntaxException}
-import java.nio.file.Files
+import java.nio.file.{Files, Path}
 
 import org.apache.spark.internal.{Logging, LogKeys, MDC}
 import org.apache.spark.network.util.JavaUtils
@@ -120,6 +120,20 @@ private[spark] trait SparkFileUtils extends Logging {
   def deleteRecursively(file: File): Unit = {
     JavaUtils.deleteRecursively(file)
   }
+
+  def getFile(names: String*): File = {
+    require(names != null && names.forall(_ != null))
+    names.tail.foldLeft(Path.of(names.head)) { (path, part) =>
+      path.resolve(part)
+    }.toFile
+  }
+
+  def getFile(parent: File, names: String*): File = {
+    require(parent != null && names != null && names.forall(_ != null))
+    names.foldLeft(parent.toPath) { (path, part) =>
+      path.resolve(part)
+    }.toFile
+  }
 }
 
 private[spark] object SparkFileUtils extends SparkFileUtils
diff --git 
a/core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala 
b/core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala
index 840fb59d410a..392d3ba7a980 100644
--- a/core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala
+++ b/core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala
@@ -21,7 +21,6 @@ import java.io._
 import java.util.EnumSet
 import java.util.concurrent.{ScheduledExecutorService, TimeUnit}
 
-import org.apache.commons.io.FileUtils
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
 import org.apache.hadoop.fs.permission.FsPermission
@@ -47,8 +46,8 @@ private[spark] class DriverLogger(conf: SparkConf) extends 
Logging {
   private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 
8).toShort)
 
   private val localLogFile: String = conf.get(DRIVER_LOG_LOCAL_DIR).map {
-    FileUtils.getFile(_, DriverLogger.DRIVER_LOG_FILE).getAbsolutePath()
-  }.getOrElse(FileUtils.getFile(
+    Utils.getFile(_, DriverLogger.DRIVER_LOG_FILE).getAbsolutePath()
+  }.getOrElse(Utils.getFile(
     Utils.getLocalDir(conf),
     DriverLogger.DRIVER_LOG_DIR,
     DriverLogger.DRIVER_LOG_FILE).getAbsolutePath())
@@ -106,7 +105,7 @@ private[spark] class DriverLogger(conf: SparkConf) extends 
Logging {
         logError(s"Error in persisting driver logs", e)
     } finally {
       Utils.tryLogNonFatalError {
-        
JavaUtils.deleteRecursively(FileUtils.getFile(localLogFile).getParentFile())
+        
JavaUtils.deleteRecursively(Utils.getFile(localLogFile).getParentFile())
       }
     }
   }
diff --git 
a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
 
b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
index 69d531c2fdac..687e778702a7 100644
--- 
a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
+++ 
b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
@@ -26,7 +26,6 @@ import java.util.zip.{ZipInputStream, ZipOutputStream}
 import scala.concurrent.duration._
 
 import com.google.common.io.{ByteStreams, Files}
-import org.apache.commons.io.FileUtils
 import org.apache.hadoop.fs.{FileStatus, FileSystem, FSDataInputStream, Path}
 import org.apache.hadoop.hdfs.{DFSInputStream, DistributedFileSystem}
 import org.apache.hadoop.ipc.{CallerContext => HadoopCallerContext}
@@ -733,15 +732,15 @@ abstract class FsHistoryProviderSuite extends 
SparkFunSuite with Matchers with P
     testConf.set(MAX_DRIVER_LOG_AGE_S, maxAge)
     val provider = new FsHistoryProvider(testConf, clock)
 
-    val log1 = FileUtils.getFile(testDir, "1" + 
DriverLogger.DRIVER_LOG_FILE_SUFFIX)
+    val log1 = Utils.getFile(testDir, "1" + 
DriverLogger.DRIVER_LOG_FILE_SUFFIX)
     createEmptyFile(log1)
     clock.setTime(firstFileModifiedTime)
     log1.setLastModified(clock.getTimeMillis())
     provider.cleanDriverLogs()
 
-    val log2 = FileUtils.getFile(testDir, "2" + 
DriverLogger.DRIVER_LOG_FILE_SUFFIX)
+    val log2 = Utils.getFile(testDir, "2" + 
DriverLogger.DRIVER_LOG_FILE_SUFFIX)
     createEmptyFile(log2)
-    val log3 = FileUtils.getFile(testDir, "3" + 
DriverLogger.DRIVER_LOG_FILE_SUFFIX)
+    val log3 = Utils.getFile(testDir, "3" + 
DriverLogger.DRIVER_LOG_FILE_SUFFIX)
     createEmptyFile(log3)
     clock.setTime(secondFileModifiedTime)
     log2.setLastModified(clock.getTimeMillis())
diff --git 
a/core/src/test/scala/org/apache/spark/util/logging/DriverLoggerSuite.scala 
b/core/src/test/scala/org/apache/spark/util/logging/DriverLoggerSuite.scala
index 97cd5caaac90..223f5e17731d 100644
--- a/core/src/test/scala/org/apache/spark/util/logging/DriverLoggerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/logging/DriverLoggerSuite.scala
@@ -19,7 +19,6 @@ package org.apache.spark.util.logging
 
 import java.io.File
 
-import org.apache.commons.io.FileUtils
 import org.apache.hadoop.fs.Path
 
 import org.apache.spark._
@@ -52,7 +51,7 @@ class DriverLoggerSuite extends SparkFunSuite with 
LocalSparkContext {
 
     // Assert driver log file exists
     val rootDir = Utils.getLocalDir(sc.getConf)
-    val driverLogsDir = FileUtils.getFile(rootDir, DriverLogger.DRIVER_LOG_DIR)
+    val driverLogsDir = Utils.getFile(rootDir, DriverLogger.DRIVER_LOG_DIR)
     assert(driverLogsDir.exists())
     val files = driverLogsDir.listFiles()
     assert(files.length === 1)
@@ -60,7 +59,7 @@ class DriverLoggerSuite extends SparkFunSuite with 
LocalSparkContext {
 
     sc.stop()
     assert(!driverLogsDir.exists())
-    val dfsFile = FileUtils.getFile(sc.getConf.get(DRIVER_LOG_DFS_DIR).get,
+    val dfsFile = Utils.getFile(sc.getConf.get(DRIVER_LOG_DFS_DIR).get,
       app_id + DriverLogger.DRIVER_LOG_FILE_SUFFIX)
     assert(dfsFile.exists())
     assert(dfsFile.length() > 0)
@@ -77,7 +76,7 @@ class DriverLoggerSuite extends SparkFunSuite with 
LocalSparkContext {
 
     // Assert driver log file exists
     val rootDir = Utils.getLocalDir(sc.getConf)
-    val driverLogsDir = FileUtils.getFile(rootDir, DriverLogger.DRIVER_LOG_DIR)
+    val driverLogsDir = Utils.getFile(rootDir, DriverLogger.DRIVER_LOG_DIR)
     assert(driverLogsDir.exists())
     val files = driverLogsDir.listFiles()
     assert(files.length === 1)
diff --git a/scalastyle-config.xml b/scalastyle-config.xml
index f8844d1eb0f5..658e0acd36d1 100644
--- a/scalastyle-config.xml
+++ b/scalastyle-config.xml
@@ -293,6 +293,11 @@ This file is divided into 3 sections:
     of Commons Lang 2 (package org.apache.commons.lang.*)</customMessage>
   </check>
 
+  <check customId="getFile" level="error" 
class="org.scalastyle.file.RegexChecker" enabled="true">
+    <parameters><parameter 
name="regex">FileUtils\.getFile</parameter></parameters>
+    <customMessage>Use getFile of SparkFileUtil or Utils 
instead.</customMessage>
+  </check>
+
   <check customId="writeStringToFile" level="error" 
class="org.scalastyle.file.RegexChecker" enabled="true">
     <parameters><parameter 
name="regex">FileUtils\.writeStringToFile</parameter></parameters>
     <customMessage>Use java.nio.file.Files.writeString instead.</customMessage>


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to