mridulm commented on code in PR #39459:
URL: https://github.com/apache/spark/pull/39459#discussion_r1071803105
##########
core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala:
##########
@@ -180,6 +186,14 @@ private[storage] class BlockInfoManager extends Logging {
//
----------------------------------------------------------------------------------------------
+ private[spark] def isRDDBlockVisible(blockId: RDDBlockId): Boolean = {
+ blockInfoWrappers.containsKey(blockId) &&
visibleRDDBlocks.contains(blockId)
+ }
Review Comment:
Wondering if there is a possibility of both of these datastructures
diverging - and driver ending up with metadata about blocks which have been
removed ....
Need to take a closer look.
##########
core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala:
##########
@@ -77,6 +77,11 @@ class BlockManagerMasterEndpoint(
// Mapping from block id to the set of block managers that have the block.
private val blockLocations = new JHashMap[BlockId,
mutable.HashSet[BlockManagerId]]
+ // Mapping from task id to the set of rdd blocks which are generated from
the task.
+ private val tidToRddBlockIds = new mutable.HashMap[Long,
mutable.HashSet[RDDBlockId]]
+ // Record the visible RDD blocks which have been generated at least from one
successful task.
+ private val visibleRDDBlocks = new mutable.HashSet[RDDBlockId]
Review Comment:
Instead of keeping track of visible blocks, should we be tracking those
which are not yet visible ?
The size would be much smaller - blocks by default get added to this `Set`,
and get promoted to visible explicitly.
##########
core/src/main/scala/org/apache/spark/storage/BlockManager.scala:
##########
@@ -1424,6 +1457,16 @@ private[spark] class BlockManager(
blockStoreUpdater.save()
}
+ // Check whether a rdd block is visible or not.
+ private[spark] def isRDDBlockVisible(blockId: RDDBlockId): Boolean = {
+ // If the rdd block visibility information not available in the block
manager,
+ // asking master for the information.
+ if (blockInfoManager.isRDDBlockVisible(blockId)) {
+ return true
+ }
+ master.isRDDBlockVisible(blockId)
Review Comment:
QQ: Should we be caching this instead of hitting driver each time ?
##########
core/src/main/scala/org/apache/spark/storage/BlockManager.scala:
##########
@@ -1325,14 +1325,47 @@ private[spark] class BlockManager(
blockInfoManager.releaseAllLocksForTask(taskAttemptId)
}
+ /**
+ * Retrieve the given rdd block if it exists and is visible, otherwise call
the provided
+ * `makeIterator` method to compute the block, persist it, and return its
values.
+ *
+ * @return either a BlockResult if the block was successfully cached, or an
iterator if the block
+ * could not be cached.
+ */
+ def getOrElseUpdateRDDBlock[T](
+ taskId: Long,
+ blockId: RDDBlockId,
+ level: StorageLevel,
+ classTag: ClassTag[T],
+ makeIterator: () => Iterator[T]): Either[BlockResult, Iterator[T]] = {
+ val isCacheVisible = isRDDBlockVisible(blockId)
+ var computed: Boolean = false
+ val getIterator = () => {
+ computed = true
+ makeIterator()
+ }
+
+ val res = getOrElseUpdate(blockId, level, classTag, getIterator)
+ if (res.isLeft && !isCacheVisible) {
+ if (!computed) {
+ // Loaded from cache, re-compute to update accumulators.
+ makeIterator()
+ }
Review Comment:
If a block is not visible, and it was not computed in this task, we should
not be using that block at all - even if cached here or elsewhere.
Particularly for unordered or indeterminate computation, this can impact the
accumulators generated - and it wont match the result returned.
So we cant simply call `makeIterator` here - as @Ngone51 suggested, we
should pass the flag to `getOrElseUpdate` and decide there based on visibility.
##########
core/src/main/scala/org/apache/spark/storage/BlockManager.scala:
##########
@@ -1325,14 +1325,47 @@ private[spark] class BlockManager(
blockInfoManager.releaseAllLocksForTask(taskAttemptId)
}
+ /**
+ * Retrieve the given rdd block if it exists and is visible, otherwise call
the provided
+ * `makeIterator` method to compute the block, persist it, and return its
values.
+ *
+ * @return either a BlockResult if the block was successfully cached, or an
iterator if the block
+ * could not be cached.
+ */
+ def getOrElseUpdateRDDBlock[T](
+ taskId: Long,
+ blockId: RDDBlockId,
+ level: StorageLevel,
+ classTag: ClassTag[T],
+ makeIterator: () => Iterator[T]): Either[BlockResult, Iterator[T]] = {
+ val isCacheVisible = isRDDBlockVisible(blockId)
+ var computed: Boolean = false
+ val getIterator = () => {
+ computed = true
+ makeIterator()
+ }
+
+ val res = getOrElseUpdate(blockId, level, classTag, getIterator)
+ if (res.isLeft && !isCacheVisible) {
+ if (!computed) {
+ // Loaded from cache, re-compute to update accumulators.
+ makeIterator()
+ }
+ // Block exists and not visible, report taskId -> blockId info to master.
+ master.updateRDDBlockTaskInfo(blockId, taskId)
+ }
+
+ res
+ }
+
/**
* Retrieve the given block if it exists, otherwise call the provided
`makeIterator` method
* to compute the block, persist it, and return its values.
*
* @return either a BlockResult if the block was successfully cached, or an
iterator if the block
* could not be cached.
*/
- def getOrElseUpdate[T](
+ private[spark] def getOrElseUpdate[T](
Review Comment:
Make this private instead ? And fix the tests which are depending on this
directly to use `getOrElseUpdateRDDBlock` instead ?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]