Skip to content

Conversation

@JoshRosen
Copy link
Contributor

What changes were proposed in this pull request?

If a BlockManager put() call failed after the BlockManagerMaster was notified of a block's availability then incomplete cleanup logic in a finally block would never send a second block status method to inform the master of the block's unavailability. This, in turn, leads to fetch failures and used to be capable of causing complete job failures before #15037 was fixed.

This patch addresses this issue via multiple small changes:

  • The finally block now calls removeBlockInternal when cleaning up from a failed put(); in addition to removing the BlockInfo entry (which was all that the old cleanup logic did), this code (redundantly) tries to remove the block from the memory and disk stores (as an added layer of defense against bugs lower down in the stack) and optionally notifies the master of block removal (which now happens during exception-triggered cleanup).
  • When a BlockManager receives a request for a block that it does not have it will now notify the master to update its block locations. This ensures that bad metadata pointing to non-existent blocks will eventually be fixed. Note that I could have implemented this logic in the block manager client (rather than in the remote server), but that would introduce the problem of distinguishing between transient and permanent failures; on the server, however, we know definitively that the block isn't present.
  • Catch NonFatal instead of Exception to avoid swallowing InterruptedExceptions thrown from synchronous block replication calls.

This patch depends upon the refactorings in #15036, so that other patch will also have to be backported when backporting this fix.

For more background on this issue, including example logs from a real production failure, see SPARK-17484.

How was this patch tested?

Two new regression tests in BlockManagerSuite.

@JoshRosen
Copy link
Contributor Author

/cc @ericl @srinathshankar for review

// notified the master about the availability of this block, so we need to send an update
// to remove this block location.
removeBlockInternal(
blockId, tellMaster = tellMaster && putBlockInfo.tellMaster && exceptionWasThrown)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

aren't these two tellMaster values equivalent since we set it in the block from the function arg above?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yep, tellMaster will equal putBlockInfo.tellMaster in this branch. Let me update this to clarify.

@SparkQA
Copy link

SparkQA commented Sep 13, 2016

Test build #65330 has finished for PR 15085 at commit 6609c2a.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

}
} else {
blockInfoManager.removeBlock(blockId)
if (exceptionWasThrown) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

could also combine this with the above else

btw, is it necessary to unlock the block in this path?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I didn't combine it so that the logWarning wouldn't need to be duplicated, but that's not a great rationale.

removeBlockInternal (which is used in both this if and else case now) will handle releasing the lock (this happens in the blockInfoManager.removeBlock call).

exceptionWasThrown = false
res
} finally {
if (blockWasSuccessfullyStored) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Wouldn't blockWasSuccessfullyStored be false if exceptionWasThrown were true ?
In that case, couldn't we write this as
try {
...
} catch (Exception e) {
removeBlock
addUpdatedBlock
} finally {
// WHatever was there before ?
}

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

One concern with using a catch here is handling of InterruptedException: if we use case NonFatal(e) that won't match InterruptedException and we'll miss out on cleanup following that. If we catch Throwable, on the other hand, then I think that we'll end up clearing the isInterrupted bit for InterruptedExceptions and it'll be awkward to match and re-set it when rethrowing. Therefore I'd like to keep the exception-handling case in the finally block with a simple check to see if we entered that block via an error case.

Note that I've seen this same exception-handling idiom used in Java code, where code that catches and re-throws Throwable won't compile in older Java versions because of the checked exception-handling (I think that newer versions are a bit more permissive about throwing exceptions from a catch block).

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That said, I think we could simplify this by moving the non-error-case code into the try block. Let me do that now.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok this is fine then. Could you leave a comment mentioning the InterruptedException problem ? Otherwise, this LGTM

@ericl
Copy link
Contributor

ericl commented Sep 13, 2016

Ok, lgtm then

On Tue, Sep 13, 2016, 3:48 PM Apache Spark QA [email protected]
wrote:

Test build #65339 has started
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65339/consoleFull

for PR 15085 at commit 8ab3108
8ab3108
.


You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHub
#15085 (comment), or mute
the thread
https://github.com/notifications/unsubscribe-auth/AAA6Sl9ErPKKdmjfmFXccG2eAwr92H20ks5qpyhMgaJpZM4J8FrF
.

@SparkQA
Copy link

SparkQA commented Sep 14, 2016

Test build #65339 has finished for PR 15085 at commit 8ab3108.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Sep 14, 2016

Test build #65337 has finished for PR 15085 at commit f69a5ea.

  • This patch fails from timeout after a configured wait of 250m.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Sep 14, 2016

Test build #65342 has finished for PR 15085 at commit f60c4be.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Sep 15, 2016

Test build #65410 has finished for PR 15085 at commit 47f9636.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@JoshRosen
Copy link
Contributor Author

I'm going to merge this to master and branch-2.0. Thanks for reviews @ericl and @srinathshankar!

asfgit pushed a commit that referenced this pull request Sep 15, 2016
…er put() exceptions

## What changes were proposed in this pull request?

If a BlockManager `put()` call failed after the BlockManagerMaster was notified of a block's availability then incomplete cleanup logic in a `finally` block would never send a second block status method to inform the master of the block's unavailability. This, in turn, leads to fetch failures and used to be capable of causing complete job failures before #15037 was fixed.

This patch addresses this issue via multiple small changes:

- The `finally` block now calls `removeBlockInternal` when cleaning up from a failed `put()`; in addition to removing the `BlockInfo` entry (which was _all_ that the old cleanup logic did), this code (redundantly) tries to remove the block from the memory and disk stores (as an added layer of defense against bugs lower down in the stack) and optionally notifies the master of block removal (which now happens during exception-triggered cleanup).
- When a BlockManager receives a request for a block that it does not have it will now notify the master to update its block locations. This ensures that bad metadata pointing to non-existent blocks will eventually be fixed. Note that I could have implemented this logic in the block manager client (rather than in the remote server), but that would introduce the problem of distinguishing between transient and permanent failures; on the server, however, we know definitively that the block isn't present.
- Catch `NonFatal` instead of `Exception` to avoid swallowing `InterruptedException`s thrown from synchronous block replication calls.

This patch depends upon the refactorings in #15036, so that other patch will also have to be backported when backporting this fix.

For more background on this issue, including example logs from a real production failure, see [SPARK-17484](https://issues.apache.org/jira/browse/SPARK-17484).

## How was this patch tested?

Two new regression tests in BlockManagerSuite.

Author: Josh Rosen <[email protected]>

Closes #15085 from JoshRosen/SPARK-17484.

(cherry picked from commit 1202075)
Signed-off-by: Josh Rosen <[email protected]>
@asfgit asfgit closed this in 1202075 Sep 15, 2016
@JoshRosen JoshRosen deleted the SPARK-17484 branch September 15, 2016 18:58
wgtmac pushed a commit to wgtmac/spark that referenced this pull request Sep 19, 2016
…er put() exceptions

## What changes were proposed in this pull request?

If a BlockManager `put()` call failed after the BlockManagerMaster was notified of a block's availability then incomplete cleanup logic in a `finally` block would never send a second block status method to inform the master of the block's unavailability. This, in turn, leads to fetch failures and used to be capable of causing complete job failures before apache#15037 was fixed.

This patch addresses this issue via multiple small changes:

- The `finally` block now calls `removeBlockInternal` when cleaning up from a failed `put()`; in addition to removing the `BlockInfo` entry (which was _all_ that the old cleanup logic did), this code (redundantly) tries to remove the block from the memory and disk stores (as an added layer of defense against bugs lower down in the stack) and optionally notifies the master of block removal (which now happens during exception-triggered cleanup).
- When a BlockManager receives a request for a block that it does not have it will now notify the master to update its block locations. This ensures that bad metadata pointing to non-existent blocks will eventually be fixed. Note that I could have implemented this logic in the block manager client (rather than in the remote server), but that would introduce the problem of distinguishing between transient and permanent failures; on the server, however, we know definitively that the block isn't present.
- Catch `NonFatal` instead of `Exception` to avoid swallowing `InterruptedException`s thrown from synchronous block replication calls.

This patch depends upon the refactorings in apache#15036, so that other patch will also have to be backported when backporting this fix.

For more background on this issue, including example logs from a real production failure, see [SPARK-17484](https://issues.apache.org/jira/browse/SPARK-17484).

## How was this patch tested?

Two new regression tests in BlockManagerSuite.

Author: Josh Rosen <[email protected]>

Closes apache#15085 from JoshRosen/SPARK-17484.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants