Skip to content

Commit ecc6eb5

Browse files
committed
[SPARK-7315] [STREAMING] [TEST] Fix flaky WALBackedBlockRDDSuite
`FileUtils.getTempDirectoryPath()` path may or may not exist. We want to make sure that it does not exist. Author: Tathagata Das <[email protected]> Closes #5853 from tdas/SPARK-7315 and squashes the following commits: 141afd5 [Tathagata Das] Removed use of FileUtils b08d4f1 [Tathagata Das] Fix flaky WALBackedBlockRDDSuite
1 parent 7394e7a commit ecc6eb5

File tree

1 file changed

+8
-4
lines changed

1 file changed

+8
-4
lines changed

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

Lines changed: 8 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -16,13 +16,13 @@
1616
*/
1717
package org.apache.spark.streaming.rdd
1818

19+
import java.io.File
1920
import java.nio.ByteBuffer
21+
import java.util.UUID
2022

2123
import scala.reflect.ClassTag
2224
import scala.util.control.NonFatal
2325

24-
import org.apache.commons.io.FileUtils
25-
2626
import org.apache.spark._
2727
import org.apache.spark.rdd.BlockRDD
2828
import org.apache.spark.storage.{BlockId, StorageLevel}
@@ -108,9 +108,13 @@ class WriteAheadLogBackedBlockRDD[T: ClassTag](
108108
// writing log data. However, the directory is not needed if data needs to be read, hence
109109
// a dummy path is provided to satisfy the method parameter requirements.
110110
// FileBasedWriteAheadLog will not create any file or directory at that path.
111-
val dummyDirectory = FileUtils.getTempDirectoryPath()
111+
// FileBasedWriteAheadLog will not create any file or directory at that path. Also,
112+
// this dummy directory should not already exist otherwise the WAL will try to recover
113+
// past events from the directory and throw errors.
114+
val nonExistentDirectory = new File(
115+
System.getProperty("java.io.tmpdir"), UUID.randomUUID().toString).getAbsolutePath
112116
writeAheadLog = WriteAheadLogUtils.createLogForReceiver(
113-
SparkEnv.get.conf, dummyDirectory, hadoopConf)
117+
SparkEnv.get.conf, nonExistentDirectory, hadoopConf)
114118
dataRead = writeAheadLog.read(partition.walRecordHandle)
115119
} catch {
116120
case NonFatal(e) =>

0 commit comments

Comments
 (0)