Skip to content

Commit

Permalink
Fix flaky WALBackedBlockRDDSuite
Browse files Browse the repository at this point in the history
  • Loading branch information
tdas committed May 1, 2015
1 parent 5c1faba commit b08d4f1
Showing 1 changed file with 8 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,9 @@
*/
package org.apache.spark.streaming.rdd

import java.io.File
import java.nio.ByteBuffer
import java.util.UUID

import scala.reflect.ClassTag
import scala.util.control.NonFatal
Expand Down Expand Up @@ -108,9 +110,13 @@ class WriteAheadLogBackedBlockRDD[T: ClassTag](
// writing log data. However, the directory is not needed if data needs to be read, hence
// a dummy path is provided to satisfy the method parameter requirements.
// FileBasedWriteAheadLog will not create any file or directory at that path.
val dummyDirectory = FileUtils.getTempDirectoryPath()
// 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.
val nonExistentDirectory = new File(
FileUtils.getTempDirectory(), UUID.randomUUID().toString).getAbsolutePath
writeAheadLog = WriteAheadLogUtils.createLogForReceiver(
SparkEnv.get.conf, dummyDirectory, hadoopConf)
SparkEnv.get.conf, nonExistentDirectory, hadoopConf)
dataRead = writeAheadLog.read(partition.walRecordHandle)
} catch {
case NonFatal(e) =>
Expand Down

0 comments on commit b08d4f1

Please sign in to comment.