[SPARK-19407][SS] defaultFS is used FileSystem.get instead of getting it from uri scheme#16815
Closed
uncleGen wants to merge 2 commits intoapache:masterfrom
Closed
[SPARK-19407][SS] defaultFS is used FileSystem.get instead of getting it from uri scheme#16815uncleGen wants to merge 2 commits intoapache:masterfrom
uncleGen wants to merge 2 commits intoapache:masterfrom
Conversation
|
Test build #72420 has finished for PR 16815 at commit
|
Contributor
Author
srowen
reviewed
Feb 6, 2017
| /** Read the metadata from file if it exists */ | ||
| def read(metadataFile: Path, hadoopConf: Configuration): Option[StreamMetadata] = { | ||
| val fs = FileSystem.get(hadoopConf) | ||
| val fs = FileSystem.get(metadataFile.toUri, hadoopConf) |
Member
There was a problem hiding this comment.
I think this should be metadataFile.getFileSystem(hadoopConf)
Contributor
Author
|
retest this please. |
|
Test build #72436 has finished for PR 16815 at commit
|
|
Test build #3558 has finished for PR 16815 at commit
|
|
Test build #72435 has finished for PR 16815 at commit
|
Contributor
|
LGTM, though checkpointing to S3 has its own separate issues related to rename performance and listing inconsistency. While this fix lets people request different filesystems for the data, checkpointing is still at risk of not working on s3 or swift, probably not GCS. It will work on Azure though. |
Member
|
yea! - I found this earlier but forgot to track it. |
Member
|
LGTM. Merging to master and 2.1. |
asfgit
pushed a commit
that referenced
this pull request
Feb 7, 2017
… it from uri scheme ## What changes were proposed in this pull request? ``` Caused by: java.lang.IllegalArgumentException: Wrong FS: s3a://**************/checkpoint/7b2231a3-d845-4740-bfa3-681850e5987f/metadata, expected: file:/// at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:649) at org.apache.hadoop.fs.RawLocalFileSystem.pathToFile(RawLocalFileSystem.java:82) at org.apache.hadoop.fs.RawLocalFileSystem.deprecatedGetFileStatus(RawLocalFileSystem.java:606) at org.apache.hadoop.fs.RawLocalFileSystem.getFileLinkStatusInternal(RawLocalFileSystem.java:824) at org.apache.hadoop.fs.RawLocalFileSystem.getFileStatus(RawLocalFileSystem.java:601) at org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:421) at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:1426) at org.apache.spark.sql.execution.streaming.StreamMetadata$.read(StreamMetadata.scala:51) at org.apache.spark.sql.execution.streaming.StreamExecution.<init>(StreamExecution.scala:100) at org.apache.spark.sql.streaming.StreamingQueryManager.createQuery(StreamingQueryManager.scala:232) at org.apache.spark.sql.streaming.StreamingQueryManager.startQuery(StreamingQueryManager.scala:269) at org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:262) ``` Can easily replicate on spark standalone cluster by providing checkpoint location uri scheme anything other than "file://" and not overriding in config. WorkAround --conf spark.hadoop.fs.defaultFS=s3a://somebucket or set it in sparkConf or spark-default.conf ## How was this patch tested? existing ut Author: uncleGen <hustyugm@gmail.com> Closes #16815 from uncleGen/SPARK-19407. (cherry picked from commit 7a0a630) Signed-off-by: Shixiong Zhu <shixiong@databricks.com>
cmonkey
pushed a commit
to cmonkey/spark
that referenced
this pull request
Feb 15, 2017
… it from uri scheme ## What changes were proposed in this pull request? ``` Caused by: java.lang.IllegalArgumentException: Wrong FS: s3a://**************/checkpoint/7b2231a3-d845-4740-bfa3-681850e5987f/metadata, expected: file:/// at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:649) at org.apache.hadoop.fs.RawLocalFileSystem.pathToFile(RawLocalFileSystem.java:82) at org.apache.hadoop.fs.RawLocalFileSystem.deprecatedGetFileStatus(RawLocalFileSystem.java:606) at org.apache.hadoop.fs.RawLocalFileSystem.getFileLinkStatusInternal(RawLocalFileSystem.java:824) at org.apache.hadoop.fs.RawLocalFileSystem.getFileStatus(RawLocalFileSystem.java:601) at org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:421) at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:1426) at org.apache.spark.sql.execution.streaming.StreamMetadata$.read(StreamMetadata.scala:51) at org.apache.spark.sql.execution.streaming.StreamExecution.<init>(StreamExecution.scala:100) at org.apache.spark.sql.streaming.StreamingQueryManager.createQuery(StreamingQueryManager.scala:232) at org.apache.spark.sql.streaming.StreamingQueryManager.startQuery(StreamingQueryManager.scala:269) at org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:262) ``` Can easily replicate on spark standalone cluster by providing checkpoint location uri scheme anything other than "file://" and not overriding in config. WorkAround --conf spark.hadoop.fs.defaultFS=s3a://somebucket or set it in sparkConf or spark-default.conf ## How was this patch tested? existing ut Author: uncleGen <hustyugm@gmail.com> Closes apache#16815 from uncleGen/SPARK-19407.
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.This suggestion is invalid because no changes were made to the code.Suggestions cannot be applied while the pull request is closed.Suggestions cannot be applied while viewing a subset of changes.Only one suggestion per line can be applied in a batch.Add this suggestion to a batch that can be applied as a single commit.Applying suggestions on deleted lines is not supported.You must change the existing code in this line in order to create a valid suggestion.Outdated suggestions cannot be applied.This suggestion has been applied or marked resolved.Suggestions cannot be applied from pending reviews.Suggestions cannot be applied on multi-line comments.Suggestions cannot be applied while the pull request is queued to merge.Suggestion cannot be applied right now. Please check back later.
What changes were proposed in this pull request?
Can easily replicate on spark standalone cluster by providing checkpoint location uri scheme anything other than "file://" and not overriding in config.
WorkAround --conf spark.hadoop.fs.defaultFS=s3a://somebucket or set it in sparkConf or spark-default.conf
How was this patch tested?
existing ut