aboutsummaryrefslogtreecommitdiff
path: root/streaming/src/main/scala/org
diff options
context:
space:
mode:
authorTathagata Das <tathagata.das1565@gmail.com>2015-08-25 00:35:51 -0700
committerTathagata Das <tathagata.das1565@gmail.com>2015-08-25 00:35:51 -0700
commit1fc37581a52530bac5d555dbf14927a5780c3b75 (patch)
tree03b493c92f0614cbf848dfade28e5d7ed82d4b07 /streaming/src/main/scala/org
parent57b960bf3706728513f9e089455a533f0244312e (diff)
downloadspark-1fc37581a52530bac5d555dbf14927a5780c3b75.tar.gz
spark-1fc37581a52530bac5d555dbf14927a5780c3b75.tar.bz2
spark-1fc37581a52530bac5d555dbf14927a5780c3b75.zip
[SPARK-10210] [STREAMING] Filter out non-existent blocks before creating BlockRDD
When write ahead log is not enabled, a recovered streaming driver still tries to run jobs using pre-failure block ids, and fails as the block do not exists in-memory any more (and cannot be recovered as receiver WAL is not enabled). This occurs because the driver-side WAL of ReceivedBlockTracker is recovers that past block information, and ReceiveInputDStream creates BlockRDDs even if those blocks do not exist. The solution in this PR is to filter out block ids that do not exist before creating the BlockRDD. In addition, it adds unit tests to verify other logic in ReceiverInputDStream. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #8405 from tdas/SPARK-10210.
Diffstat (limited to 'streaming/src/main/scala/org')
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala10
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala2
2 files changed, 10 insertions, 2 deletions
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala
index a15800917c..6c139f32da 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala
@@ -116,7 +116,15 @@ abstract class ReceiverInputDStream[T: ClassTag](@transient ssc_ : StreamingCont
logWarning("Some blocks have Write Ahead Log information; this is unexpected")
}
}
- new BlockRDD[T](ssc.sc, blockIds)
+ val validBlockIds = blockIds.filter { id =>
+ ssc.sparkContext.env.blockManager.master.contains(id)
+ }
+ if (validBlockIds.size != blockIds.size) {
+ logWarning("Some blocks could not be recovered as they were not found in memory. " +
+ "To prevent such data loss, enabled Write Ahead Log (see programming guide " +
+ "for more details.")
+ }
+ new BlockRDD[T](ssc.sc, validBlockIds)
}
} else {
// If no block is ready now, creating WriteAheadLogBackedBlockRDD or BlockRDD
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 620b8a36a2..e081ffe46f 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
@@ -75,7 +75,7 @@ private[streaming]
class WriteAheadLogBackedBlockRDD[T: ClassTag](
@transient sc: SparkContext,
@transient blockIds: Array[BlockId],
- @transient walRecordHandles: Array[WriteAheadLogRecordHandle],
+ @transient val walRecordHandles: Array[WriteAheadLogRecordHandle],
@transient isBlockIdValid: Array[Boolean] = Array.empty,
storeInBlockManager: Boolean = false,
storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER)