diff --git a/core/src/test/java/test/org/apache/spark/JavaAPISuite.java b/core/src/test/java/test/org/apache/spark/JavaAPISuite.java index 802cb2667cc8..57eafc38d3a6 100644 --- a/core/src/test/java/test/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/test/org/apache/spark/JavaAPISuite.java @@ -960,7 +960,7 @@ public void textFiles() throws IOException { rdd.saveAsTextFile(outputDir); // Read the plain text file and check it's OK File outputFile = new File(outputDir, "part-00000"); - String content = Files.asCharSource(outputFile, StandardCharsets.UTF_8).read(); + String content = java.nio.file.Files.readString(outputFile.toPath()); assertEquals("1\n2\n3\n4\n", content); // Also try reading it in as a text file RDD List expected = Arrays.asList("1", "2", "3", "4"); diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala index bc0e2bbee9c3..cd8c876c5017 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala @@ -19,10 +19,10 @@ package org.apache.spark.deploy.history import java.io.{ByteArrayInputStream, ByteArrayOutputStream, File} import java.net.URI -import java.nio.charset.StandardCharsets +import java.nio.file.Files import java.util.zip.{ZipInputStream, ZipOutputStream} -import com.google.common.io.{ByteStreams, Files} +import com.google.common.io.ByteStreams import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.scalatest.BeforeAndAfter @@ -220,8 +220,8 @@ class SingleFileEventLogFileReaderSuite extends EventLogFileReadersSuite { val entry = is.getNextEntry assert(entry != null) - val actual = new String(ByteStreams.toByteArray(is), StandardCharsets.UTF_8) - val expected = Files.asCharSource(new File(logPath.toString), StandardCharsets.UTF_8).read() + val actual = ByteStreams.toByteArray(is) + val expected = Files.readAllBytes(new File(logPath.toString).toPath) assert(actual === expected) assert(is.getNextEntry === null) } @@ -367,9 +367,8 @@ class RollingEventLogFilesReaderSuite extends EventLogFileReadersSuite { val fileName = entry.getName.stripPrefix(logPath.getName + "/") assert(allFileNames.contains(fileName)) - val actual = new String(ByteStreams.toByteArray(is), StandardCharsets.UTF_8) - val expected = Files.asCharSource( - new File(logPath.toString, fileName), StandardCharsets.UTF_8).read() + val actual = ByteStreams.toByteArray(is) + val expected = Files.readAllBytes(new File(logPath.toString, fileName).toPath) assert(actual === expected) } } 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 687e778702a7..6c2a926a2d42 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 @@ -708,8 +708,7 @@ abstract class FsHistoryProviderSuite extends SparkFunSuite with Matchers with P while (entry != null) { val actual = new String(ByteStreams.toByteArray(inputStream), StandardCharsets.UTF_8) val expected = - Files.asCharSource(logs.find(_.getName == entry.getName).get, StandardCharsets.UTF_8) - .read() + java.nio.file.Files.readString(logs.find(_.getName == entry.getName).get.toPath) actual should be (expected) totalEntries += 1 entry = inputStream.getNextEntry diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 6131adcacf83..9d064d31672f 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -19,12 +19,13 @@ package org.apache.spark.deploy.history import java.io.{File, FileInputStream, FileWriter, InputStream, IOException} import java.net.{HttpURLConnection, URI, URL} import java.nio.charset.StandardCharsets +import java.nio.file.Files import java.util.zip.ZipInputStream import scala.concurrent.duration._ import scala.jdk.CollectionConverters._ -import com.google.common.io.{ByteStreams, Files} +import com.google.common.io.ByteStreams import jakarta.servlet._ import jakarta.servlet.http.{HttpServletRequest, HttpServletRequestWrapper, HttpServletResponse} import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} @@ -308,7 +309,7 @@ abstract class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with val expectedFile = { new File(logDir, entry.getName) } - val expected = Files.asCharSource(expectedFile, StandardCharsets.UTF_8).read() + val expected = Files.readString(expectedFile.toPath) val actual = new String(ByteStreams.toByteArray(zipStream), StandardCharsets.UTF_8) actual should be (expected) filesCompared += 1 diff --git a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala index 6cfefa795883..6b5bee1dfc85 100644 --- a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala @@ -58,7 +58,7 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter { val appender = new FileAppender(inputStream, testFile) inputStream.close() appender.awaitTermination() - assert(Files.asCharSource(testFile, StandardCharsets.UTF_8).read() === header + testString) + assert(java.nio.file.Files.readString(testFile.toPath) === header + testString) } test("SPARK-35027: basic file appender - close stream") { @@ -392,7 +392,7 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter { Utils.closeQuietly(inputStream) } } else { - Files.asCharSource(file, StandardCharsets.UTF_8).read() + java.nio.file.Files.readString(file.toPath) } }.mkString("") assert(allText === expectedText) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala index e1b4969bfe79..345d61aaf7cc 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala @@ -17,10 +17,9 @@ package org.apache.spark.deploy.k8s import java.io.File -import java.nio.charset.StandardCharsets +import java.nio.file.Files import com.fasterxml.jackson.databind.ObjectMapper -import com.google.common.io.Files import io.fabric8.kubernetes.client.{ConfigBuilder, KubernetesClient, KubernetesClientBuilder} import io.fabric8.kubernetes.client.Config.KUBERNETES_REQUEST_RETRY_BACKOFFLIMIT_SYSTEM_PROPERTY import io.fabric8.kubernetes.client.Config.autoConfigure @@ -98,8 +97,7 @@ object SparkKubernetesClientFactory extends Logging { .withOption(oauthTokenValue) { (token, configBuilder) => configBuilder.withOauthToken(token) }.withOption(oauthTokenFile) { - (file, configBuilder) => - configBuilder.withOauthToken(Files.asCharSource(file, StandardCharsets.UTF_8).read()) + (file, configBuilder) => configBuilder.withOauthToken(Files.readString(file.toPath)) }.withOption(caCertFile) { (file, configBuilder) => configBuilder.withCaCertFile(file) }.withOption(clientKeyFile) { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStep.scala index d64378a65d66..290f6d377aee 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStep.scala @@ -17,11 +17,10 @@ package org.apache.spark.deploy.k8s.features import java.io.File -import java.nio.charset.StandardCharsets +import java.nio.file.Files import scala.jdk.CollectionConverters._ -import com.google.common.io.Files import io.fabric8.kubernetes.api.model._ import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesUtils, SparkPod} @@ -116,7 +115,7 @@ private[spark] class HadoopConfDriverFeatureStep(conf: KubernetesConf) override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { if (confDir.isDefined) { val fileMap = confFiles.map { file => - (file.getName(), Files.asCharSource(file, StandardCharsets.UTF_8).read()) + (file.getName(), Files.readString(file.toPath)) }.toMap.asJava Seq(new ConfigMapBuilder() diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala index bd591b39de01..032a5266b0f2 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala @@ -17,7 +17,6 @@ package org.apache.spark.deploy.k8s.features import java.io.File -import java.nio.charset.StandardCharsets import scala.jdk.CollectionConverters._ import scala.util.control.NonFatal @@ -236,7 +235,7 @@ private[spark] class KerberosConfDriverFeatureStep(kubernetesConf: KubernetesDri .endMetadata() .withImmutable(true) .addToData( - Map(file.getName() -> Files.asCharSource(file, StandardCharsets.UTF_8).read()).asJava) + Map(file.getName() -> java.nio.file.Files.readString(file.toPath)).asJava) .build() } } ++ { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStep.scala index f94dad2d15dc..3d0828044dc0 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStep.scala @@ -17,9 +17,8 @@ package org.apache.spark.deploy.k8s.features import java.io.File -import java.nio.charset.StandardCharsets +import java.nio.file.Files -import com.google.common.io.Files import io.fabric8.kubernetes.api.model.{ConfigMapBuilder, ContainerBuilder, HasMetadata, PodBuilder} import org.apache.spark.deploy.SparkHadoopUtil @@ -81,7 +80,7 @@ private[spark] class PodTemplateConfigMapStep(conf: KubernetesConf) val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf.sparkConf) val uri = downloadFile(podTemplateFile, Utils.createTempDir(), conf.sparkConf, hadoopConf) val file = new java.net.URI(uri).getPath - val podTemplateString = Files.asCharSource(new File(file), StandardCharsets.UTF_8).read() + val podTemplateString = Files.readString(new File(file).toPath) Seq(new ConfigMapBuilder() .withNewMetadata() .withName(configmapName) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index 9f2b6cdbcb3f..2450d50366a2 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -17,13 +17,11 @@ package org.apache.spark.deploy.k8s.integrationtest import java.io.File -import java.nio.charset.StandardCharsets -import java.nio.file.{Path, Paths} +import java.nio.file.{Files, Path, Paths} import java.util.UUID import scala.jdk.CollectionConverters._ -import com.google.common.io.Files import io.fabric8.kubernetes.api.model.Pod import io.fabric8.kubernetes.client.{Watcher, WatcherException} import io.fabric8.kubernetes.client.KubernetesClientException @@ -129,7 +127,7 @@ class KubernetesSuite extends SparkFunSuite val tagFile = new File(path) require(tagFile.isFile, s"No file found for image tag at ${tagFile.getAbsolutePath}.") - Files.asCharSource(tagFile, StandardCharsets.UTF_8).read().trim + Files.readString(tagFile.toPath).trim } .orElse(sys.props.get(CONFIG_KEY_IMAGE_TAG)) .getOrElse { diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala index 3bf6a6e84a88..f21928ca865b 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala @@ -236,11 +236,11 @@ abstract class BaseYarnClusterSuite extends SparkFunSuite with Matchers { // an error message val output = new Object() { override def toString: String = outFile - .map(Files.asCharSource(_, StandardCharsets.UTF_8).read()) + .map((f: File) => java.nio.file.Files.readString(f.toPath)) .getOrElse("(stdout/stderr was not captured)") } assert(finalState === SparkAppHandle.State.FINISHED, output) - val resultString = Files.asCharSource(result, StandardCharsets.UTF_8).read() + val resultString = java.nio.file.Files.readString(result.toPath) assert(resultString === expected, output) } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 3280b80b50f4..1c490e0d60f9 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -377,7 +377,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite { extraEnv = Map("SPARK_CONF_DIR" -> confDir.getAbsolutePath), extraConf = Map(CLIENT_INCLUDE_DRIVER_LOGS_LINK.key -> true.toString)) checkResult(finalState, result) - val logOutput = Files.asCharSource(logOutFile, StandardCharsets.UTF_8).read() + val logOutput = java.nio.file.Files.readString(logOutFile.toPath) val logFilePattern = raw"""(?s).+\sDriver Logs \(\): https?://.+/(\?\S+)?\s.+""" logOutput should fullyMatch regex logFilePattern.replace("", "stdout") logOutput should fullyMatch regex logFilePattern.replace("", "stderr") diff --git a/scalastyle-config.xml b/scalastyle-config.xml index 7ee04019e219..4cff6c18d73c 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -297,6 +297,11 @@ This file is divided into 3 sections: Use Files.readString instead. + + \bFiles\.asCharSource\b + Use Files.readString instead. + + \bFileUtils\.write\b Use Files.writeString instead.