Skip to content

Conversation

@gengliangwang
Copy link
Member

@gengliangwang gengliangwang commented Jun 9, 2020

What changes were proposed in this pull request?

Make Hadoop file system config effective in data source options.

From org.apache.hadoop.fs.FileSystem.java:

  public static FileSystem get(URI uri, Configuration conf) throws IOException {
    String scheme = uri.getScheme();
    String authority = uri.getAuthority();

    if (scheme == null && authority == null) {     // use default FS
      return get(conf);
    }

    if (scheme != null && authority == null) {     // no authority
      URI defaultUri = getDefaultUri(conf);
      if (scheme.equals(defaultUri.getScheme())    // if scheme matches default
          && defaultUri.getAuthority() != null) {  // & default has authority
        return get(defaultUri, conf);              // return default
      }
    }
    
    String disableCacheName = String.format("fs.%s.impl.disable.cache", scheme);
    if (conf.getBoolean(disableCacheName, false)) {
      return createFileSystem(uri, conf);
    }

    return CACHE.get(uri, conf);
  }

Before changes, the file system configurations in data source options are not propagated in DataSource.scala.
After changes, we can specify authority and URI schema related configurations for scanning file systems.

This problem only exists in data source V1. In V2, we already use sparkSession.sessionState.newHadoopConfWithOptions(options) in FileTable.

Why are the changes needed?

Allow users to specify authority and URI schema related Hadoop configurations for file source reading.

Does this PR introduce any user-facing change?

Yes, the file system related Hadoop configuration in data source option will be effective on reading.

How was this patch tested?

Unit test

@gengliangwang
Copy link
Member Author

This PR is to backport #28760 to branch-2.4

@gengliangwang
Copy link
Member Author

cc @liancheng

@SparkQA
Copy link

SparkQA commented Jun 9, 2020

Test build #123703 has finished for PR 28771 at commit 6984453.

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

@gengliangwang
Copy link
Member Author

merging to branch 2.4

gengliangwang added a commit that referenced this pull request Jun 10, 2020
… in data source options

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

Mkae Hadoop file system config effective in data source options.

From `org.apache.hadoop.fs.FileSystem.java`:
```
  public static FileSystem get(URI uri, Configuration conf) throws IOException {
    String scheme = uri.getScheme();
    String authority = uri.getAuthority();

    if (scheme == null && authority == null) {     // use default FS
      return get(conf);
    }

    if (scheme != null && authority == null) {     // no authority
      URI defaultUri = getDefaultUri(conf);
      if (scheme.equals(defaultUri.getScheme())    // if scheme matches default
          && defaultUri.getAuthority() != null) {  // & default has authority
        return get(defaultUri, conf);              // return default
      }
    }

    String disableCacheName = String.format("fs.%s.impl.disable.cache", scheme);
    if (conf.getBoolean(disableCacheName, false)) {
      return createFileSystem(uri, conf);
    }

    return CACHE.get(uri, conf);
  }
```
Before changes, the file system configurations in data source options are not propagated in `DataSource.scala`.
After changes, we can specify authority and URI schema related configurations for scanning file systems.

This problem only exists in data source V1. In V2, we already use `sparkSession.sessionState.newHadoopConfWithOptions(options)` in `FileTable`.
### Why are the changes needed?

Allow users to specify authority and URI schema related Hadoop configurations for file source reading.

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

Yes, the file system related Hadoop configuration in data source option will be effective on reading.

### How was this patch tested?

Unit test

Closes #28771 from gengliangwang/SPARK-31935-2.4.

Authored-by: Gengliang Wang <[email protected]>
Signed-off-by: Gengliang Wang <[email protected]>
@dongjoon-hyun
Copy link
Member

Hi, @gengliangwang . Did you merge this without any LGTM from other committers?

cc @gatorsmile , @cloud-fan , @HyukjinKwon

@dongjoon-hyun
Copy link
Member

dongjoon-hyun commented Jul 1, 2020

Note that I'm fine with that backporting because it seems that you were confident with this and it looked urgent to you.

@HyukjinKwon
Copy link
Member

Yeah, it should be best to have LGTM or at least some positive comments.

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.

4 participants