Skip to content

Commit c539c94

Browse files
author
gss2002
committed
[SPARK-25778] WriteAheadLogBackedBlockRDD in YARN Cluster Mode Fails 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
1 parent f26cd18 commit c539c94

File tree

1 file changed

+1
-1
lines changed

1 file changed

+1
-1
lines changed

streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -136,7 +136,7 @@ class WriteAheadLogBackedBlockRDD[T: ClassTag](
136136
// this dummy directory should not already exist otherwise the WAL will try to recover
137137
// past events from the directory and throw errors.
138138
val nonExistentDirectory = new File(
139-
System.getProperty("java.io.tmpdir"), UUID.randomUUID().toString).getAbsolutePath
139+
System.getProperty("java.io.tmpdir"), UUID.randomUUID().toString).toURI.toString
140140
writeAheadLog = WriteAheadLogUtils.createLogForReceiver(
141141
SparkEnv.get.conf, nonExistentDirectory, hadoopConf)
142142
dataRead = writeAheadLog.read(partition.walRecordHandle)

0 commit comments

Comments
 (0)