Skip to content

Conversation

@gengliangwang
Copy link
Member

@gengliangwang gengliangwang commented Aug 5, 2020

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.

@SparkQA
Copy link

SparkQA commented Aug 5, 2020

Test build #127089 has finished for PR 29357 at commit 06e8f8f.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

sparkConf: SparkConf,
hadoopConf: Configuration): Set[FileSystem] = {
// scalastyle:off FileSystemGet
val defaultFS = FileSystem.get(hadoopConf)
Copy link
Member

Choose a reason for hiding this comment

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

I think it's a bit arguable to disallow it in style. If you want to just get the default file system (when path is not available), FileSystem.get is a valid usage and a good fallback to use.

Copy link
Member Author

Choose a reason for hiding this comment

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

I agree this is arguable. The rule is motivated by a regression of another project. I think it is worth doing so if we treat it as a warning for potential mistakes.

Copy link
Member

Choose a reason for hiding this comment

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

Okay, I am fine with it.

<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. Please use
"FileSystem.get(URI uri, Configuration conf)" or "Path.getFileSystem()" instead.
Copy link
Contributor

Choose a reason for hiding this comment

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

Path.getFileSystem() also takes a conf parameter?

Copy link
Member Author

Choose a reason for hiding this comment

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

Right, thank you.

@cloud-fan
Copy link
Contributor

@gengliangwang can you put a real example in the PR description? e.g. Path#getFileSystem will respect the default file scheme of the path and create a file system for s3, while FileSystem.get(Configuration conf) may return a file system for local files.

@gengliangwang
Copy link
Member Author

@gengliangwang can you put a real example in the PR description? e.g. Path#getFileSystem will respect the default file scheme of the path and create a file system for s3, while FileSystem.get(Configuration conf) may return a file system for local files.

Thanks, I have updated the description.

Copy link
Contributor

@xkrogen xkrogen left a comment

Choose a reason for hiding this comment

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

I worked on Hadoop (as a committer on HDFS) for a few years before starting my work on Spark, and I can attest that this is a common anti-pattern that can have severe implications. Even within the Hadoop codebase itself, I've had to go through and rip out old code which leveraged FileSystem.get(Configuration conf) because it breaks under some environments. Many developers don't realize it is almost always not the correct API to use.

So, strong +1 from me on this effort.

Comment on lines 270 to 272
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. Please use
"FileSystem.get(URI uri, Configuration conf)" or "Path.getFileSystem(Configuration conf)" instead.
Copy link
Contributor

Choose a reason for hiding this comment

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

Using FileSystem.get(Configuration conf) is dangerous even outside of the misconfiguration scenario you describe. It's entirely possible for Spark applications to be working with multiple FileSystem instances (say, local HDFS and a cloud blob store, or two HDFS clusters). Using FileSystem.get(Configuration conf) in this instance can also result in nasty errors.

Not sure if it's worth mentioning here, will leave that up to your judgement.

Copy link
Member Author

Choose a reason for hiding this comment

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

@xkrogen Thanks for the +1! I just update the message in style check. I think mentioning "reading a target path of non-default file system" is good enough.

Copy link
Contributor

Choose a reason for hiding this comment

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

how about

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
also lead to nasty errors when you deal with multiple file systems. Please use ....

Copy link
Member Author

Choose a reason for hiding this comment

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

@cloud-fan ok, updated.

@srowen
Copy link
Member

srowen commented Aug 5, 2020

Seems reasonable to guard against. I know we have fixed this a few times over the years.

@SparkQA
Copy link

SparkQA commented Aug 5, 2020

Test build #127097 has finished for PR 29357 at commit edb1114.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.


val hadoopConf = spark.sessionState.newHadoopConf()
val fs = FileSystem.get(hadoopConf)
val fs = new Path(tableDir.getAbsolutePath).getFileSystem(hadoopConf)
Copy link
Member

Choose a reason for hiding this comment

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

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 Do you mean making it

// scalastyle:off FileSystemGet
val fs = FileSystem.get(hadoopConf)
// scalastyle:on FileSystemGet

here?

Copy link
Member Author

Choose a reason for hiding this comment

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

Copy link
Contributor

Choose a reason for hiding this comment

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

Ideally we should not change the actual code when we just want to add a style check. But this is test and I think it's safe.

Copy link
Member

Choose a reason for hiding this comment

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

You are right, @gengliangwang ~

@SparkQA
Copy link

SparkQA commented Aug 5, 2020

Test build #127098 has finished for PR 29357 at commit b08aee6.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Aug 5, 2020

Test build #127100 has finished for PR 29357 at commit 92f3d6f.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@cloud-fan
Copy link
Contributor

thanks, merging to master!

@cloud-fan cloud-fan closed this in e93b8f0 Aug 6, 2020
@dongjoon-hyun
Copy link
Member

+1, late LGTM.

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.

7 participants