[ 
https://issues.apache.org/jira/browse/SPARK-21239?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16721662#comment-16721662
 ] 

ASF GitHub Bot commented on SPARK-21239:
----------------------------------------

vanzin closed pull request #18452: [SPARK-21239][STREAMING] Support WAL recover 
in windows
URL: https://github.com/apache/spark/pull/18452
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala
 
b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala
index 844760ab61d2e..2c97c0afa260e 100644
--- 
a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala
+++ 
b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala
@@ -135,8 +135,11 @@ class WriteAheadLogBackedBlockRDD[T: ClassTag](
         // FileBasedWriteAheadLog will not create any file or directory at 
that path. Also,
         // this dummy directory should not already exist otherwise the WAL 
will try to recover
         // past events from the directory and throw errors.
+        // Specifically, the nonExistentDirectory will contain a colon in 
windows, this is invalid
+        // for hadoop. Remove the drive letter and colon, e.g. "D:" out of 
this path by default
         val nonExistentDirectory = new File(
-          System.getProperty("java.io.tmpdir"), 
UUID.randomUUID().toString).getAbsolutePath
+          System.getProperty("java.io.tmpdir"), 
+          
UUID.randomUUID().toString).getAbsolutePath.replaceFirst("[a-zA-Z]:", "")
         writeAheadLog = WriteAheadLogUtils.createLogForReceiver(
           SparkEnv.get.conf, nonExistentDirectory, hadoopConf)
         dataRead = writeAheadLog.read(partition.walRecordHandle)


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Support WAL recover in windows
> ------------------------------
>
>                 Key: SPARK-21239
>                 URL: https://issues.apache.org/jira/browse/SPARK-21239
>             Project: Spark
>          Issue Type: Bug
>          Components: DStreams, Windows
>    Affects Versions: 1.6.3, 2.1.0, 2.1.1, 2.2.0
>            Reporter: Yun Tang
>            Priority: Major
>
> When driver failed over, it will read WAL from HDFS by calling 
> WriteAheadLogBackedBlockRDD.getBlockFromWriteAheadLog(), however, it need a 
> dummy local path to satisfy the method parameter requirements, but the path 
> in windows will contain a colon which is not valid for hadoop. I removed the 
> potential driver letter and colon.
> I found one email from spark-user ever talked about this bug 
> (https://www.mail-archive.com/user@spark.apache.org/msg55030.html)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org

Reply via email to