Github user squito commented on a diff in the pull request:
https://github.com/apache/spark/pull/6990#discussion_r33794783
--- Diff: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
---
@@ -833,8 +833,10 @@ private[spark] class BlockManager(
logDebug("Put block %s locally took %s".format(blockId,
Utils.getUsedTimeMs(startTimeMs)))
// Either we're storing bytes and we asynchronously started
replication, or we're storing
- // values and need to serialize and replicate them now:
- if (putLevel.replication > 1) {
+ // values and need to serialize and replicate them now.
+ // Should not replicate the block if its StorageLevel is
StorageLevel.NONE or
+ // putting it to local is failed.
+ if (!putBlockInfo.isFailed && putLevel.replication > 1) {
--- End diff --
@dibbhatt I think I see what you're getting at, but I'm with TD that I
don't know if this change is worth it. I believe the "de-optimizing" he is
referring to is that in normal spark, if you store remote but not local, it is
useful. Normal spark jobs will take advantage of that remote copy.
This change is really only designed to help out with streaming, where the
recommendation is to use MEMORY_AND_DISK anyway. IMO its not worth any of the
changes for that -- even option (3) would introduce a decent amount of
complexity in the BlockManager that would require a big comment to explain why
we do that ... and it would just be to support this one use case that isn't
recommended anyway.
Just to brainstorm, I want to throw one more idea out there. We don't want
the receiver to have to do a query after `doPut` to see if the block was
replicated somewhere, b/c that would be too expensive. But what if instead we
change the return type of `doPut` to already include that information? `doPut`
already knows if replication was successful, so instead of returning
`Seq[(BlockId, BlockStatus)]`, it could return `case class
DoPutResult(updatedBlocks: Seq[(BlockId, BlockStatus)], replicationLevel: Int)`
(probably with some better name). Then the receiver would have the information
it needs without needing to perform any expensive extra checks.
That said, I'm not really favor of this idea either -- again I feel its a
decent amount of complexity, without a compelling reason. thoughts @tdas?
sorry @dibbhatt but I'm leaning towards "won't fix / not a bug" for this
issue.
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]