Skip to content

Commit

Permalink
[SPARK-32539][INFRA] Disallow FileSystem.get(Configuration conf) in…
Browse files Browse the repository at this point in the history
… style check by default

### What changes were proposed in this pull request?

Disallow `FileSystem.get(Configuration conf)` in Scala style check by default and suggest developers use `FileSystem.get(URI uri, Configuration conf)` or `Path.getFileSystem()` instead.

### Why are the changes needed?

The method `FileSystem.get(Configuration conf)` will return a default FileSystem instance if the conf `fs.file.impl` is not set. This can cause file not found exception on reading a target path of non-default file system, e.g. S3. It is hard to discover such a mistake via unit tests.
If we disallow it in Scala style check by default and suggest developers use `FileSystem.get(URI uri, Configuration conf)` or `Path.getFileSystem(Configuration conf)`, we can reduce potential regression and PR review effort.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Manually run scala style check and test.

Closes apache#29357 from gengliangwang/newStyleRule.

Authored-by: Gengliang Wang <[email protected]>
Signed-off-by: Wenchen Fan <[email protected]>
  • Loading branch information
gengliangwang authored and cloud-fan committed Aug 6, 2020
1 parent 7f275ee commit e93b8f0
Show file tree
Hide file tree
Showing 5 changed files with 23 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -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))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
}

Expand Down Expand Up @@ -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)
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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()
Expand Down
14 changes: 14 additions & 0 deletions scalastyle-config.xml
Original file line number Diff line number Diff line change
Expand Up @@ -264,6 +264,20 @@ This file is divided into 3 sections:
of Commons Lang 2 (package org.apache.commons.lang.*)</customMessage>
</check>

<check customId="FileSystemGet" level="error" class="org.scalastyle.file.RegexChecker" enabled="true">
<parameters><parameter name="regex">FileSystem.get\([a-zA-Z_$][a-zA-Z_$0-9]*\)</parameter></parameters>
<customMessage><![CDATA[
Are you sure that you want to use "FileSystem.get(Configuration conf)"? If the input
configuration is not set properly, a default FileSystem instance will be returned. It can
lead to errors when you deal with multiple file systems. Please consider using
"FileSystem.get(URI uri, Configuration conf)" or "Path.getFileSystem(Configuration conf)" instead.
If you must use the method "FileSystem.get(Configuration conf)", wrap the code block with
// scalastyle:off FileSystemGet
FileSystem.get(...)
// scalastyle:on FileSystemGet
]]></customMessage>
</check>

<check customId="extractopt" level="error" class="org.scalastyle.scalariform.TokenChecker" enabled="true">
<parameters><parameter name="regex">extractOpt</parameter></parameters>
<customMessage>Use jsonOption(x).map(.extract[T]) instead of .extractOpt[T], as the latter
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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("_")
})
Expand Down

0 comments on commit e93b8f0

Please sign in to comment.