diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala
index 4e91e72361488..a46864e2d3c9c 100644
--- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala
@@ -139,7 +139,9 @@ private[deploy] object HadoopFSDelegationTokenProvider {
def hadoopFSsToAccess(
sparkConf: SparkConf,
hadoopConf: Configuration): Set[FileSystem] = {
+ // scalastyle:off FileSystemGet
val defaultFS = FileSystem.get(hadoopConf)
+ // scalastyle:on FileSystemGet
val filesystemsToAccess = sparkConf.get(KERBEROS_FILESYSTEMS_TO_ACCESS)
.map(new Path(_).getFileSystem(hadoopConf))
diff --git a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala
index a619f10bbf064..6d174b5e0f81b 100644
--- a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala
+++ b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala
@@ -222,7 +222,9 @@ class HadoopMapRedWriteConfigUtil[K, V: ClassTag](conf: SerializableJobConf)
if (path != null) {
path.getFileSystem(getConf)
} else {
+ // scalastyle:off FileSystemGet
FileSystem.get(getConf)
+ // scalastyle:on FileSystemGet
}
}
@@ -285,7 +287,9 @@ class HadoopMapRedWriteConfigUtil[K, V: ClassTag](conf: SerializableJobConf)
if (SparkHadoopWriterUtils.isOutputSpecValidationEnabled(conf)) {
// FileOutputFormat ignores the filesystem parameter
+ // scalastyle:off FileSystemGet
val ignoredFs = FileSystem.get(getConf)
+ // scalastyle:on FileSystemGet
getOutputFormat().checkOutputSpecs(ignoredFs, getConf)
}
}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index 1045fb089c017..517a4af2e4b02 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -181,10 +181,12 @@ private[spark] class Client(
// The app staging dir based on the STAGING_DIR configuration if configured
// otherwise based on the users home directory.
+ // scalastyle:off FileSystemGet
val appStagingBaseDir = sparkConf.get(STAGING_DIR)
.map { new Path(_, UserGroupInformation.getCurrentUser.getShortUserName) }
.getOrElse(FileSystem.get(hadoopConf).getHomeDirectory())
stagingDirPath = new Path(appStagingBaseDir, getAppStagingDir(appId))
+ // scalastyle:on FileSystemGet
new CallerContext("CLIENT", sparkConf.get(APP_CALLER_CONTEXT),
Option(appId.toString)).setCurrentContext()
diff --git a/scalastyle-config.xml b/scalastyle-config.xml
index 73ac14fdba1cf..c1dc57be56dff 100644
--- a/scalastyle-config.xml
+++ b/scalastyle-config.xml
@@ -264,6 +264,20 @@ This file is divided into 3 sections:
of Commons Lang 2 (package org.apache.commons.lang.*)
+
+ FileSystem.get\([a-zA-Z_$][a-zA-Z_$0-9]*\)
+
+
+
extractOpt
Use jsonOption(x).map(.extract[T]) instead of .extractOpt[T], as the latter
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala
index a14f6416199a1..8c4eedfde76cd 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala
@@ -165,7 +165,7 @@ class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedS
// the assumption on column stats, and also the end-to-end behavior.
val hadoopConf = spark.sessionState.newHadoopConf()
- val fs = FileSystem.get(hadoopConf)
+ val fs = new Path(tableDir.getAbsolutePath).getFileSystem(hadoopConf)
val parts = fs.listStatus(new Path(tableDir.getAbsolutePath), new PathFilter {
override def accept(path: Path): Boolean = !path.getName.startsWith("_")
})