-
Notifications
You must be signed in to change notification settings - Fork 28.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-7139][Streaming] Allow received block metadata to be saved to WAL and recovered on driver failure #5732
Changes from all commits
d06fa21
1bc5bc3
5f67a59
466212c
637bc9c
685fab3
19668ba
575476e
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -23,6 +23,8 @@ import java.util.UUID | |
import scala.reflect.ClassTag | ||
import scala.util.control.NonFatal | ||
|
||
import org.apache.commons.io.FileUtils | ||
|
||
import org.apache.spark._ | ||
import org.apache.spark.rdd.BlockRDD | ||
import org.apache.spark.storage.{BlockId, StorageLevel} | ||
|
@@ -31,30 +33,42 @@ import org.apache.spark.streaming.util._ | |
/** | ||
* Partition class for [[org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD]]. | ||
* It contains information about the id of the blocks having this partition's data and | ||
* the segment of the write ahead log that backs the partition. | ||
* the corresponding record handle in the write ahead log that backs the partition. | ||
* @param index index of the partition | ||
* @param blockId id of the block having the partition data | ||
* @param isBlockIdValid Whether the block Ids are valid (i.e., the blocks are present in the Spark | ||
* executors). If not, then block lookups by the block ids will be skipped. | ||
* By default, this is an empty array signifying true for all the blocks. | ||
* @param walRecordHandle Handle of the record in a write ahead log having the partition data | ||
*/ | ||
private[streaming] | ||
class WriteAheadLogBackedBlockRDDPartition( | ||
val index: Int, | ||
val blockId: BlockId, | ||
val walRecordHandle: WriteAheadLogRecordHandle) | ||
extends Partition | ||
val isBlockIdValid: Boolean, | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can you add this param to the scaladoc? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Good catch. |
||
val walRecordHandle: WriteAheadLogRecordHandle | ||
) extends Partition | ||
|
||
|
||
/** | ||
* This class represents a special case of the BlockRDD where the data blocks in | ||
* the block manager are also backed by data in write ahead logs. For reading | ||
* the data, this RDD first looks up the blocks by their ids in the block manager. | ||
* If it does not find them, it looks up the corresponding data in the write ahead log. | ||
* If it does not find them, it looks up the WAL using the corresponding record handle. | ||
* The lookup of the blocks from the block manager can be skipped by setting the corresponding | ||
* element in isBlockIdValid to false. This is a performance optimization which does not affect | ||
* correctness, and it can be used in situations where it is known that the block | ||
* does not exist in the Spark executors (e.g. after a failed driver is restarted). | ||
* | ||
* | ||
* @param sc SparkContext | ||
* @param blockIds Ids of the blocks that contains this RDD's data | ||
* @param walRecordHandles Record handles in write ahead logs that contain this RDD's data | ||
* @param storeInBlockManager Whether to store in the block manager after reading | ||
* from the WAL record | ||
* @param isBlockIdValid Whether the block Ids are valid (i.e., the blocks are present in the Spark | ||
* executors). If not, then block lookups by the block ids will be skipped. | ||
* By default, this is an empty array signifying true for all the blocks. | ||
* @param storeInBlockManager Whether to store a block in the block manager | ||
* after reading it from the WAL | ||
* @param storageLevel storage level to store when storing in block manager | ||
* (applicable when storeInBlockManager = true) | ||
*/ | ||
|
@@ -63,23 +77,32 @@ class WriteAheadLogBackedBlockRDD[T: ClassTag]( | |
@transient sc: SparkContext, | ||
@transient blockIds: Array[BlockId], | ||
@transient walRecordHandles: Array[WriteAheadLogRecordHandle], | ||
storeInBlockManager: Boolean, | ||
storageLevel: StorageLevel) | ||
@transient isBlockIdValid: Array[Boolean] = Array.empty, | ||
storeInBlockManager: Boolean = false, | ||
storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER) | ||
extends BlockRDD[T](sc, blockIds) { | ||
|
||
require( | ||
blockIds.length == walRecordHandles.length, | ||
s"Number of block ids (${blockIds.length}) must be " + | ||
s"the same as number of WAL record handles (${walRecordHandles.length}})!") | ||
s"Number of block Ids (${blockIds.length}) must be " + | ||
s" same as number of WAL record handles (${walRecordHandles.length}})") | ||
|
||
require( | ||
isBlockIdValid.isEmpty || isBlockIdValid.length == blockIds.length, | ||
s"Number of elements in isBlockIdValid (${isBlockIdValid.length}) must be " + | ||
s" same as number of block Ids (${blockIds.length})") | ||
|
||
// Hadoop configuration is not serializable, so broadcast it as a serializable. | ||
@transient private val hadoopConfig = sc.hadoopConfiguration | ||
private val broadcastedHadoopConf = new SerializableWritable(hadoopConfig) | ||
|
||
override def isValid(): Boolean = true | ||
|
||
override def getPartitions: Array[Partition] = { | ||
assertValid() | ||
Array.tabulate(blockIds.size) { i => | ||
new WriteAheadLogBackedBlockRDDPartition(i, blockIds(i), walRecordHandles(i)) | ||
Array.tabulate(blockIds.length) { i => | ||
val isValid = if (isBlockIdValid.length == 0) true else isBlockIdValid(i) | ||
new WriteAheadLogBackedBlockRDDPartition(i, blockIds(i), isValid, walRecordHandles(i)) | ||
} | ||
} | ||
|
||
|
@@ -94,51 +117,57 @@ class WriteAheadLogBackedBlockRDD[T: ClassTag]( | |
val blockManager = SparkEnv.get.blockManager | ||
val partition = split.asInstanceOf[WriteAheadLogBackedBlockRDDPartition] | ||
val blockId = partition.blockId | ||
blockManager.get(blockId) match { | ||
case Some(block) => // Data is in Block Manager | ||
val iterator = block.data.asInstanceOf[Iterator[T]] | ||
logDebug(s"Read partition data of $this from block manager, block $blockId") | ||
iterator | ||
case None => // Data not found in Block Manager, grab it from write ahead log file | ||
var dataRead: ByteBuffer = null | ||
var writeAheadLog: WriteAheadLog = null | ||
try { | ||
// The WriteAheadLogUtils.createLog*** method needs a directory to create a | ||
// WriteAheadLog object as the default FileBasedWriteAheadLog needs a directory for | ||
// 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. | ||
// 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( | ||
System.getProperty("java.io.tmpdir"), UUID.randomUUID().toString).getAbsolutePath | ||
writeAheadLog = WriteAheadLogUtils.createLogForReceiver( | ||
SparkEnv.get.conf, nonExistentDirectory, hadoopConf) | ||
dataRead = writeAheadLog.read(partition.walRecordHandle) | ||
} catch { | ||
case NonFatal(e) => | ||
throw new SparkException( | ||
s"Could not read data from write ahead log record ${partition.walRecordHandle}", e) | ||
} finally { | ||
if (writeAheadLog != null) { | ||
writeAheadLog.close() | ||
writeAheadLog = null | ||
} | ||
} | ||
if (dataRead == null) { | ||
|
||
def getBlockFromBlockManager(): Option[Iterator[T]] = { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This whole thing just puts the old code within inner functions, not really a big change. The real change is that these functions are called selectively in lines 157..160. |
||
blockManager.get(blockId).map(_.data.asInstanceOf[Iterator[T]]) | ||
} | ||
|
||
def getBlockFromWriteAheadLog(): Iterator[T] = { | ||
var dataRead: ByteBuffer = null | ||
var writeAheadLog: WriteAheadLog = null | ||
try { | ||
// The WriteAheadLogUtils.createLog*** method needs a directory to create a | ||
// WriteAheadLog object as the default FileBasedWriteAheadLog needs a directory for | ||
// 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. | ||
// 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( | ||
System.getProperty("java.io.tmpdir"), UUID.randomUUID().toString).getAbsolutePath | ||
writeAheadLog = WriteAheadLogUtils.createLogForReceiver( | ||
SparkEnv.get.conf, nonExistentDirectory, hadoopConf) | ||
dataRead = writeAheadLog.read(partition.walRecordHandle) | ||
} catch { | ||
case NonFatal(e) => | ||
throw new SparkException( | ||
s"Could not read data from write ahead log record ${partition.walRecordHandle}, " + | ||
s"read returned null") | ||
s"Could not read data from write ahead log record ${partition.walRecordHandle}", e) | ||
} finally { | ||
if (writeAheadLog != null) { | ||
writeAheadLog.close() | ||
writeAheadLog = null | ||
} | ||
logInfo(s"Read partition data of $this from write ahead log, record handle " + | ||
partition.walRecordHandle) | ||
if (storeInBlockManager) { | ||
blockManager.putBytes(blockId, dataRead, storageLevel) | ||
logDebug(s"Stored partition data of $this into block manager with level $storageLevel") | ||
dataRead.rewind() | ||
} | ||
blockManager.dataDeserialize(blockId, dataRead).asInstanceOf[Iterator[T]] | ||
} | ||
if (dataRead == null) { | ||
throw new SparkException( | ||
s"Could not read data from write ahead log record ${partition.walRecordHandle}, " + | ||
s"read returned null") | ||
} | ||
logInfo(s"Read partition data of $this from write ahead log, record handle " + | ||
partition.walRecordHandle) | ||
if (storeInBlockManager) { | ||
blockManager.putBytes(blockId, dataRead, storageLevel) | ||
logDebug(s"Stored partition data of $this into block manager with level $storageLevel") | ||
dataRead.rewind() | ||
} | ||
blockManager.dataDeserialize(blockId, dataRead).asInstanceOf[Iterator[T]] | ||
} | ||
|
||
if (partition.isBlockIdValid) { | ||
getBlockFromBlockManager().getOrElse { getBlockFromWriteAheadLog() } | ||
} else { | ||
getBlockFromWriteAheadLog() | ||
} | ||
} | ||
|
||
|
@@ -149,12 +178,23 @@ class WriteAheadLogBackedBlockRDD[T: ClassTag]( | |
*/ | ||
override def getPreferredLocations(split: Partition): Seq[String] = { | ||
val partition = split.asInstanceOf[WriteAheadLogBackedBlockRDDPartition] | ||
val blockLocations = getBlockIdLocations().get(partition.blockId) | ||
val blockLocations = if (partition.isBlockIdValid) { | ||
getBlockIdLocations().get(partition.blockId) | ||
} else { | ||
None | ||
} | ||
|
||
blockLocations.getOrElse { | ||
partition.walRecordHandle match { | ||
case fileSegment: FileBasedWriteAheadLogSegment => | ||
HdfsUtils.getFileSegmentLocations( | ||
fileSegment.path, fileSegment.offset, fileSegment.length, hadoopConfig) | ||
try { | ||
HdfsUtils.getFileSegmentLocations( | ||
fileSegment.path, fileSegment.offset, fileSegment.length, hadoopConfig) | ||
} catch { | ||
case NonFatal(e) => | ||
logError("Error getting WAL file segment locations", e) | ||
Seq.empty | ||
} | ||
case _ => | ||
Seq.empty | ||
} | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I do wonder if this should just throw an exception if any have missing wal records... that just means there is an exception right?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Or rather, can we tell whether the WAL is in use with this receiver and then just throw an exception if we see any blocks that do not have a record handle?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We can do that. Since it is a conf property.