Skip to content

Conversation

@gengliangwang
Copy link
Member

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

…ata source options

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`.

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

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

Unit test

Closes apache#28760 from gengliangwang/ds_conf.

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

This PR backports #28760 to branch-3.0

@gengliangwang gengliangwang changed the title [SPARK-31935][SQL] Hadoop file system config should be effective in data source options [3.0][SPARK-31935][SQL] Hadoop file system config should be effective in data source options Jun 10, 2020
@gengliangwang gengliangwang requested a review from cloud-fan June 10, 2020 04:49
@SparkQA
Copy link

SparkQA commented Jun 10, 2020

Test build #123723 has finished for PR 28776 at commit f6cca6b.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@gengliangwang
Copy link
Member Author

retest this please

Copy link
Member

@HyukjinKwon HyukjinKwon left a comment

Choose a reason for hiding this comment

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

LGTM if tests pass

@HyukjinKwon HyukjinKwon changed the title [3.0][SPARK-31935][SQL] Hadoop file system config should be effective in data source options [SPARK-31935][SQL][3.0] Hadoop file system config should be effective in data source options Jun 10, 2020
@SparkQA
Copy link

SparkQA commented Jun 10, 2020

Test build #123734 has finished for PR 28776 at commit f6cca6b.

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

@cloud-fan
Copy link
Contributor

retest this please

@SparkQA
Copy link

SparkQA commented Jun 10, 2020

Test build #123754 has finished for PR 28776 at commit f6cca6b.

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

@dongjoon-hyun
Copy link
Member

Please hold on this PR because this will break Hadoop 3.2.

@dongjoon-hyun dongjoon-hyun changed the title [SPARK-31935][SQL][3.0] Hadoop file system config should be effective in data source options [SPARK-31935][SQL][3.0][test-hadoop3.2] Hadoop file system config should be effective in data source options Jun 10, 2020
@dongjoon-hyun
Copy link
Member

Retest this please.

@dongjoon-hyun
Copy link
Member

@gengliangwang . Please include my follow-PR here.

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

This PR updates the test case to accept Hadoop 2/3 error message correctly.

### Why are the changes needed?

SPARK-31935(apache#28760) breaks Hadoop 3.2 UT because Hadoop 2 and Hadoop 3 have different exception messages.

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

No.

### How was this patch tested?

Pass the Jenkins with both Hadoop 2/3 or do the following manually.

**Hadoop 2.7**
```
$ build/sbt "sql/testOnly *.FileBasedDataSourceSuite -- -z SPARK-31935"
...
[info] All tests passed.
```

**Hadoop 3.2**
```
$ build/sbt "sql/testOnly *.FileBasedDataSourceSuite -- -z SPARK-31935" -Phadoop-3.2
...
[info] All tests passed.
```

Closes apache#28791 from dongjoon-hyun/SPARK-31935.

Authored-by: Dongjoon Hyun <[email protected]>
Signed-off-by: Dongjoon Hyun <[email protected]>
@gengliangwang
Copy link
Member Author

@dongjoon-hyun sure, I have included it in this PR. Thanks.

@dongjoon-hyun
Copy link
Member

Thanks~

@SparkQA
Copy link

SparkQA commented Jun 11, 2020

Test build #123794 has finished for PR 28776 at commit f6cca6b.

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

val message = intercept[java.io.IOException] {
spark.readStream.option("fs.defaultFS", defaultFs).text(path)
}.getMessage
assert(message == expectMessage)
Copy link
Member

Choose a reason for hiding this comment

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

Oops. It seems that I missed here.

Copy link
Member

Choose a reason for hiding this comment

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

Could you fix this place and forward-port to master, too?

Copy link
Member Author

Choose a reason for hiding this comment

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

ok, let me do it now

Copy link
Member

Choose a reason for hiding this comment

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

Thanks again!

@SparkQA
Copy link

SparkQA commented Jun 11, 2020

Test build #123802 has finished for PR 28776 at commit 5623228.

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

This PR updates the test case to accept Hadoop 2/3 error message correctly.

SPARK-31935(apache#28760) breaks Hadoop 3.2 UT because Hadoop 2 and Hadoop 3 have different exception messages.
In apache#28791, there are two test suites missed the fix

No

Unit test

Closes apache#28796 from gengliangwang/SPARK-31926-followup.

Authored-by: Gengliang Wang <[email protected]>
Signed-off-by: Dongjoon Hyun <[email protected]>
@SparkQA
Copy link

SparkQA commented Jun 11, 2020

Test build #123821 has finished for PR 28776 at commit da8d48d.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HyukjinKwon
Copy link
Member

retest this please

@SparkQA
Copy link

SparkQA commented Jun 11, 2020

Test build #123839 has finished for PR 28776 at commit da8d48d.

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

@HyukjinKwon
Copy link
Member

retest this please

@HyukjinKwon
Copy link
Member

I reverted dad163f from branch-3.0.

@SparkQA
Copy link

SparkQA commented Jun 11, 2020

Test build #123849 has finished for PR 28776 at commit da8d48d.

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

@cloud-fan
Copy link
Contributor

retest this please

@SparkQA
Copy link

SparkQA commented Jun 11, 2020

Test build #123857 has finished for PR 28776 at commit da8d48d.

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

@dongjoon-hyun
Copy link
Member

Thank you, @gengliangwang and all.
Merged to branch-3.0.

@dongjoon-hyun dongjoon-hyun changed the title [SPARK-31935][SQL][3.0][test-hadoop3.2] Hadoop file system config should be effective in data source options [SPARK-31935][SQL][3.0] Hadoop file system config should be effective in data source options Jun 11, 2020
dongjoon-hyun added a commit that referenced this pull request Jun 11, 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 #28776 from gengliangwang/SPARK-31935-3.0.

Lead-authored-by: Gengliang Wang <[email protected]>
Co-authored-by: Dongjoon Hyun <[email protected]>
Signed-off-by: Dongjoon Hyun <[email protected]>
holdenk pushed a commit to holdenk/spark that referenced this pull request Jun 25, 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 apache#28776 from gengliangwang/SPARK-31935-3.0.

Lead-authored-by: Gengliang Wang <[email protected]>
Co-authored-by: Dongjoon Hyun <[email protected]>
Signed-off-by: Dongjoon Hyun <[email protected]>
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.

5 participants