[SPARK-20894][SS]Resolve the checkpoint location in driver and use the resolved path in state store#18149
Closed
zsxwing wants to merge 2 commits intoapache:masterfrom
Closed
[SPARK-20894][SS]Resolve the checkpoint location in driver and use the resolved path in state store#18149zsxwing wants to merge 2 commits intoapache:masterfrom
zsxwing wants to merge 2 commits intoapache:masterfrom
Conversation
|
Test build #77558 has finished for PR 18149 at commit
|
tdas
reviewed
May 31, 2017
|
|
||
| /** | ||
| * Processes any data available between `availableOffsets` and `committedOffsets`. | ||
| * |
tdas
reviewed
May 31, 2017
| val query = MemoryStream[Int].toDF | ||
| .writeStream | ||
| .option("checkpointLocation", checkpointLocation) | ||
| .format("console").start() |
Contributor
There was a problem hiding this comment.
NIT: .start() on the next line.
Contributor
|
roughly LGTM, as long as you resolve conflicts and tests pass. |
|
Test build #77605 has finished for PR 18149 at commit
|
Member
Author
|
Thanks! Merging to master and |
zsxwing
added a commit
to zsxwing/spark
that referenced
this pull request
Jun 1, 2017
…he resolved path in state store When the user runs a Structured Streaming query in a cluster, if the driver uses the local file system, StateStore running in executors will throw a file-not-found exception. However, the current error is not obvious. This PR makes StreamExecution resolve the path in driver and uses the full path including the scheme part (such as `hdfs:/`, `file:/`) in StateStore. Then if the above error happens, StateStore will throw an error with this full path which starts with `file:/`, and it makes this error obvious: the checkpoint location is on the local file system. One potential minor issue is that the user cannot use different default file system settings in driver and executors (e.g., use a public HDFS address in driver and a private HDFS address in executors) after this change. However, since the batch query also has this issue (See https://github.com/apache/spark/blob/4bb6a53ebd06de3de97139a2dbc7c85fc3aa3e66/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala#L402), it doesn't make things worse. The new added test. Author: Shixiong Zhu <shixiong@databricks.com> Closes apache#18149 from zsxwing/SPARK-20894.
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?
When the user runs a Structured Streaming query in a cluster, if the driver uses the local file system, StateStore running in executors will throw a file-not-found exception. However, the current error is not obvious.
This PR makes StreamExecution resolve the path in driver and uses the full path including the scheme part (such as
hdfs:/,file:/) in StateStore.Then if the above error happens, StateStore will throw an error with this full path which starts with
file:/, and it makes this error obvious: the checkpoint location is on the local file system.One potential minor issue is that the user cannot use different default file system settings in driver and executors (e.g., use a public HDFS address in driver and a private HDFS address in executors) after this change. However, since the batch query also has this issue (See
spark/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
Line 402 in 4bb6a53
How was this patch tested?
The new added test.