Skip to content

Conversation

StephanEwen
Copy link
Contributor

@StephanEwen StephanEwen commented Jul 14, 2016

This pull request fixes the issue that state restore operations can get stuck when tasks are cancelled during state restore. That happens due to a bug in HDFS, which deadlocks (or livelocks) when the reading thread is interrupted.

This introduces two things:

  1. All state handles and key/value snapshots are now Closable. This does not delete any checkpoint data, but simply closes pending streams and data fetch handles. Operations concurrently accessing the state handles state should fail.
  2. The StreamTask holds a set of "Closables" that it closes upon cancellation. This is a cleaner way of stopping in-progress work than relying on "interrupt()" to interrupt that work.

This mechanism should eventually be extended to also cancel operators and state handles pending asynchronous materialization.

There is a test that has an interrupt sensitive state handle (mimicking HDFS's deadlock behavior) that causes a stall without this pull request and cleanly finishes with the changes in this pull request.

This also adds a test validating that all state handled and key/value snapshots add a proper serialVersionUID.

* A simple base for closable handles.
*
* Offers to register a stream (or other closable object) that close calls are delegated to if
* the handel is closed or was already closed.
Copy link
Contributor

Choose a reason for hiding this comment

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

typo: handel => handle

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Thanks, will fix it.

@uce
Copy link
Contributor

uce commented Jul 15, 2016

Looks very good! The test failures seem unrelated:

The added tests and refactorings are very readable.

I think this is good to merge mod some minor inline comments.

@StephanEwen
Copy link
Contributor Author

Thanks, I'll address your comments and merge this...

@StephanEwen StephanEwen force-pushed the state_handle_cancellation branch from c411b37 to ff52e0e Compare July 15, 2016 11:43
State handles are cancelable, to make sure long running checkpoint restore operations do
finish early on cancallation, even if the code does not properly react to interrupts.

This is especially important since HDFS client code is so buggy that it deadlocks when
interrupted without closing.
@StephanEwen StephanEwen force-pushed the state_handle_cancellation branch from ff52e0e to a340bf2 Compare July 15, 2016 13:24
@StephanEwen
Copy link
Contributor Author

Manually merged in e9f660d

@StephanEwen StephanEwen deleted the state_handle_cancellation branch August 1, 2016 18:09
@StephanEwen StephanEwen restored the state_handle_cancellation branch August 1, 2016 18:09
@liuml07
Copy link
Member

liuml07 commented Mar 22, 2017

Is this related to https://issues.apache.org/jira/browse/HADOOP-14214? Thanks,

@StephanEwen
Copy link
Contributor Author

I think it is yes. We worked around it in the meantime...

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants