-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-25778] WriteAheadLogBackedBlockRDD in YARN Cluster Mode Fails … #22867
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
|
@lresende any possibilities of getting this change committed or at least getting it on the radar for key changes for spark to work with YARN Cluster mode specifically since $PWD creates issues with Spark when a similar path is used within HDFS and users do not have access to it when using writeAheadLogs. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't think a new configuration is the right thing here.
Seems like the problem here is that this should really be pointing at the local disk, and not fs.defaultFS; in which case, instead of getAbsolutePath, it should be .toURI().toString(). Does that work?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@vanzin I wondered the same thing I figured for now I'd generate the patch to try to get some feedback. This only affects the YARN Cluster mode as it looks $PWD which resolves to the local disk where the job is executing. I can look at making the change to use localFS if it seems reasonable? does that work?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
does that work?
That's my question. I don't like a new configuration. So I'd only consider going down that path if there is no other alternative.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'll run some tests over here on my cluster with a few changes and see what happens
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Basically what I'm suggesting with my latest top-level comment, and what I first suggested here, is to replace getAbsolutePath in this line with .toURI().toString().
|
History from JIRA and error WriteAheadLogBackedBlockRDD in YARN Cluster Mode Fails due lack of access to HDFS path
|
|
@vanzin trying this.. I'll advise shortly private val tmpDir = "file:///" + System.getProperty("java.io.tmpdir") |
|
@vanzin I made the following change and it didn't work. How do you want to proceed? override def isValid(): Boolean = true
It did not work.. Looks like it needs an HDFS path.. 18/10/30 01:06:50 ERROR Executor: Exception in task 0.2 in stage 3.0 (TID 73) |
|
@vanzin this seems to work.. Not sure what your thoughts are on this private val tmpDir = broadcastedHadoopConf.value.get("hadoop.tmp.dir", |
|
@vanzin can you please review latest patch thanks! |
|
I think the reason your original attempt didn't work is because of how you wrote the code. That seems to have too many slashes. I suggested using "File.toURI" which results in a different URI: Could you try that instead? If that really doesn't work we can go with |
|
@vanzin you are right! I appreciate the help with this one. I will cut a patch in the AM after testing on a large scale cluster job that is taking from IBM MQ and ETLing data and shipping off to Kafka. But this looks to work: |
…due to lack of access to java.io.tmpdir in HDFS WriteAheadLogBackedBlockRDD usage of java.io.tmpdir will fail if $PWD resolves to a folder in HDFS and the Spark YARN Cluster job does not have the correct access to this folder in regards to the dummy folder. So the fix is to pass the locally resolved dummy folder to the createLogForReceiver with the correct nonExistantDirectory
|
@vanzin can you review the latest PR. This seems like it works without issue. I ran this on my home cluster and my larger clusters at work where the issue was first discovered and it seems to solve it. Thanks again for the help |
|
ok to test |
|
Test build #98831 has finished for PR 22867 at commit
|
|
Merging to master / 2.4. |
…due lack of access to tmpDir from $PWD to HDFS WriteAheadLogBackedBlockRDD usage of java.io.tmpdir will fail if $PWD resolves to a folder in HDFS and the Spark YARN Cluster job does not have the correct access to this folder in regards to the dummy folder. So this patch provides an option to set spark.streaming.receiver.blockStore.tmpdir to override java.io.tmpdir which sets $PWD from YARN Cluster mode. ## What changes were proposed in this pull request? This change provides an option to override the java.io.tmpdir option so that when $PWD is resolved in YARN Cluster mode Spark does not attempt to use this folder and instead use the folder provided with the following option: spark.streaming.receiver.blockStore.tmpdir ## How was this patch tested? Patch was manually tested on a Spark Streaming Job with Write Ahead logs in Cluster mode. Closes #22867 from gss2002/SPARK-25778. Authored-by: gss2002 <[email protected]> Signed-off-by: Marcelo Vanzin <[email protected]> (cherry picked from commit 2b671e7) Signed-off-by: Marcelo Vanzin <[email protected]>
…due lack of access to tmpDir from $PWD to HDFS WriteAheadLogBackedBlockRDD usage of java.io.tmpdir will fail if $PWD resolves to a folder in HDFS and the Spark YARN Cluster job does not have the correct access to this folder in regards to the dummy folder. So this patch provides an option to set spark.streaming.receiver.blockStore.tmpdir to override java.io.tmpdir which sets $PWD from YARN Cluster mode. ## What changes were proposed in this pull request? This change provides an option to override the java.io.tmpdir option so that when $PWD is resolved in YARN Cluster mode Spark does not attempt to use this folder and instead use the folder provided with the following option: spark.streaming.receiver.blockStore.tmpdir ## How was this patch tested? Patch was manually tested on a Spark Streaming Job with Write Ahead logs in Cluster mode. Closes apache#22867 from gss2002/SPARK-25778. Authored-by: gss2002 <[email protected]> Signed-off-by: Marcelo Vanzin <[email protected]>
…due lack of access to tmpDir from $PWD to HDFS WriteAheadLogBackedBlockRDD usage of java.io.tmpdir will fail if $PWD resolves to a folder in HDFS and the Spark YARN Cluster job does not have the correct access to this folder in regards to the dummy folder. So this patch provides an option to set spark.streaming.receiver.blockStore.tmpdir to override java.io.tmpdir which sets $PWD from YARN Cluster mode. ## What changes were proposed in this pull request? This change provides an option to override the java.io.tmpdir option so that when $PWD is resolved in YARN Cluster mode Spark does not attempt to use this folder and instead use the folder provided with the following option: spark.streaming.receiver.blockStore.tmpdir ## How was this patch tested? Patch was manually tested on a Spark Streaming Job with Write Ahead logs in Cluster mode. Closes apache#22867 from gss2002/SPARK-25778. Authored-by: gss2002 <[email protected]> Signed-off-by: Marcelo Vanzin <[email protected]> (cherry picked from commit 2b671e7) Signed-off-by: Marcelo Vanzin <[email protected]>
…due lack of access to tmpDir from $PWD to HDFS WriteAheadLogBackedBlockRDD usage of java.io.tmpdir will fail if $PWD resolves to a folder in HDFS and the Spark YARN Cluster job does not have the correct access to this folder in regards to the dummy folder. So this patch provides an option to set spark.streaming.receiver.blockStore.tmpdir to override java.io.tmpdir which sets $PWD from YARN Cluster mode. ## What changes were proposed in this pull request? This change provides an option to override the java.io.tmpdir option so that when $PWD is resolved in YARN Cluster mode Spark does not attempt to use this folder and instead use the folder provided with the following option: spark.streaming.receiver.blockStore.tmpdir ## How was this patch tested? Patch was manually tested on a Spark Streaming Job with Write Ahead logs in Cluster mode. Closes apache#22867 from gss2002/SPARK-25778. Authored-by: gss2002 <[email protected]> Signed-off-by: Marcelo Vanzin <[email protected]> (cherry picked from commit 2b671e7) Signed-off-by: Marcelo Vanzin <[email protected]>
…ails due lack of access to tmpDir from $PWD to HDFS apache#22867 WriteAheadLogBackedBlockRDD usage of java.io.tmpdir will fail if $PWD resolves to a folder in HDFS and the Spark YARN Cluster job does not have the correct access to this folder in regards to the dummy folder. So this patch provides an option to set spark.streaming.receiver.blockStore.tmpdir to override java.io.tmpdir which sets $PWD from YARN Cluster mode. What changes were proposed in this pull request? This change provides an option to override the java.io.tmpdir option so that when $PWD is resolved in YARN Cluster mode Spark does not attempt to use this folder and instead use the folder provided with the following option: spark.streaming.receiver.blockStore.tmpdir
…due lack of access to tmpDir from $PWD to HDFS
WriteAheadLogBackedBlockRDD usage of java.io.tmpdir will fail if $PWD resolves to a folder in HDFS and the Spark YARN Cluster job does not have the correct access to this folder in regards to the dummy folder. So this patch provides an option to set spark.streaming.receiver.blockStore.tmpdir to override java.io.tmpdir which sets $PWD from YARN Cluster mode.
What changes were proposed in this pull request?
This change provides an option to override the java.io.tmpdir option so that when $PWD is resolved in YARN Cluster mode Spark does not attempt to use this folder and instead use the folder provided with the following option: spark.streaming.receiver.blockStore.tmpdir
How was this patch tested?
Patch was manually tested on a Spark Streaming Job with Write Ahead logs in Cluster mode.