-
Notifications
You must be signed in to change notification settings - Fork 4.5k
[BEAM-3499, BEAM-2607] Gives the runner access to positions of SDF claimed blocks #4483
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
Conversation
|
retest this please |
tgroh
left a comment
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.
There may be a couple of duplicate comments, because I went through the commits one-by-one first
| this.claimObserver = claimObserver; | ||
| } | ||
|
|
||
| public final boolean tryClaim(PositionT position) { |
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 would strongly consider inverting these names in some way (so the author implements tryClaim, and this is executeTryClaim or something that signals that it is using the tryClaim method)
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'm coming from the assumption that new RestrictionTrackers are written much more rarely than new SDFs using existing trackers, and I'd like the caller to use tryClaim. I guess I could rename tryClaimImpl to executeTryClaim but it seems about equally descriptive. (side note: I considered a number of other alternatives for this design, e.g. passing a claim callback as a context parameter to @ProcessElement; allowing a RestrictionTracker to simply refuse a checkpoint etc. to address just the checkpointing issue, but they all were much worse in various ways)
| this.terminationState = state.terminationState; | ||
| this.pending = Lists.newLinkedList(state.pending); | ||
| this.pending = Maps.newLinkedHashMapWithExpectedSize(state.pending.size()); | ||
| for (Map.Entry<HashCode, TimestampedValue<OutputT>> entry : state.pending.entrySet()) { |
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.
this.pending.putAll(state.pending)? or this.pending = new LinkedHashMap<>(state.pending)
| }, | ||
| maxDuration.getMillis(), | ||
| TimeUnit.MILLISECONDS); | ||
| void checkClaimHasNotFailed() { |
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.
Can this just be inlined?
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.
Done.
| */ | ||
| public interface RestrictionTracker<RestrictionT> { | ||
| public abstract class RestrictionTracker<RestrictionT, PositionT> { | ||
| interface ClaimObserver<PositionT> { |
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.
Do we expect the ClaimObserver to ever interact with the PositionT?
| } | ||
| } | ||
|
|
||
| private static <T> T unsafeClone(Coder<T> coder, T value) { |
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.
s/unsafe/unchecked/
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.
Done.
| new InMemoryCombiningState<>(combineFn, accumCoder); | ||
| if (!this.isCleared) { | ||
| that.isCleared = this.isCleared; | ||
| that.addAccum(accum); |
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.
Should this be cloned?
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.
Yup, thanks for the catch
| this.tracker = tracker; | ||
| } | ||
|
|
||
| void checkClaimHasNotFailed() { |
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.
Inline?
| public abstract class RestrictionTracker<RestrictionT, PositionT> { | ||
| /** Internal interface allowing a runner to observe the calls to {@link #tryClaim}. */ | ||
| @Internal | ||
| public interface ClaimObserver<PositionT> { |
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.
Do we expect this to ever do anything notable with the position? I can't think of a case where the invoker would be concerned with the actual position, which is an implementation detail of the DoFn.
If you've got an idea of when it might, I'd love an example; otherwise I'd remove the parameters from this interface
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.
Yes, the observer will eventually need to store the positions and pass them back to new methods of SDF for splitting, as part of implementation of splitting/checkpointing over Fn API.
jkff
left a comment
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.
Thanks!
| new InMemoryCombiningState<>(combineFn, accumCoder); | ||
| if (!this.isCleared) { | ||
| that.isCleared = this.isCleared; | ||
| that.addAccum(accum); |
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.
Yup, thanks for the catch
| } | ||
| } | ||
|
|
||
| private static <T> T unsafeClone(Coder<T> coder, T value) { |
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.
Done.
| }, | ||
| maxDuration.getMillis(), | ||
| TimeUnit.MILLISECONDS); | ||
| void checkClaimHasNotFailed() { |
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.
Done.
| public abstract class RestrictionTracker<RestrictionT, PositionT> { | ||
| /** Internal interface allowing a runner to observe the calls to {@link #tryClaim}. */ | ||
| @Internal | ||
| public interface ClaimObserver<PositionT> { |
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.
Yes, the observer will eventually need to store the positions and pass them back to new methods of SDF for splitting, as part of implementation of splitting/checkpointing over Fn API.
| this.claimObserver = claimObserver; | ||
| } | ||
|
|
||
| public final boolean tryClaim(PositionT position) { |
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'm coming from the assumption that new RestrictionTrackers are written much more rarely than new SDFs using existing trackers, and I'd like the caller to use tryClaim. I guess I could rename tryClaimImpl to executeTryClaim but it seems about equally descriptive. (side note: I considered a number of other alternatives for this design, e.g. passing a claim callback as a context parameter to @ProcessElement; allowing a RestrictionTracker to simply refuse a checkpoint etc. to address just the checkpointing issue, but they all were much worse in various ways)
|
Apologies, forgot to actually push the changes. |
chamikaramj
left a comment
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.
Thanks.
|
|
||
| private void noteOutput() { | ||
| checkState(!hasClaimFailed, "Output is not allowed after a failed tryClaim()"); | ||
| checkState(numClaimedBlocks > 0, "Output is not allowed before tryClaim()"); |
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.
Nice :)
|
|
||
| @Override | ||
| public synchronized void updateWatermark(Instant watermark) { | ||
| // Updating the watermark without any claimed blocks is allowed. |
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.
Why ? Should we at least warn ?
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.
Clarified in a comment.
|
|
||
| @Override | ||
| public synchronized GrowthState<OutputT, KeyT, TerminationStateT> checkpoint() { | ||
| checkState( |
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.
Should we be rejecting the checkpoint request instead of failing here ?
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.
Rejecting the checkpoint is not allowed. Allowing it is one of the alternatives I considered, but since runner needs access to positions anyway, I preferred to just do that.
| // unless output is complete or termination condition is reached. | ||
| if (tracker.shouldPollMore()) { | ||
| LOG.info( | ||
| "{} - emitted all known results so far; will resume polling in {} ms", |
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.
Mention numEmitted (total) here ?
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.
Done.
| KV.of(c.element(), nextPending.getValue()), nextPending.getTimestamp()); | ||
| ++numEmitted; | ||
| } | ||
| LOG.debug("{} - emitted {} new results.", c.element(), numEmitted); |
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.
This log might be a bit confusing. It says new results but do not reset numEmitted after this log.
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.
Made this log more comprehensive and added some clarifying variables.
| // of any work to be done at the moment, but more might emerge later. In this case, | ||
| // we must simply reschedule the original restriction - checkpointing a tracker that | ||
| // hasn't claimed any work is not allowed. | ||
| residual = KV.of(tracker.currentRestriction(), processContext.getLastReportedWatermark()); |
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.
Why not just fail ? This might result in an infinite scheduling loop due to a bug, no ?
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.
Clarified in a comment.
| } | ||
| } | ||
|
|
||
| private static <T> T uncheckedClone(Coder<T> coder, T value) { |
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.
Why "unchecked" ? Add a comment ?
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.
Added a comment.
| } | ||
| } | ||
|
|
||
| private static class SnappyCoder<T> extends StructuredCoder<T> { |
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.
Should we take make this public (and it's own Java file) ? This might be useful for other transforms.
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.
Done.
jkff
left a comment
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.
Thanks!
| } | ||
| } | ||
|
|
||
| private static <T> T uncheckedClone(Coder<T> coder, T value) { |
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.
Added a comment.
| // of any work to be done at the moment, but more might emerge later. In this case, | ||
| // we must simply reschedule the original restriction - checkpointing a tracker that | ||
| // hasn't claimed any work is not allowed. | ||
| residual = KV.of(tracker.currentRestriction(), processContext.getLastReportedWatermark()); |
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.
Clarified in a comment.
|
|
||
| @Override | ||
| public synchronized void updateWatermark(Instant watermark) { | ||
| // Updating the watermark without any claimed blocks is allowed. |
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.
Clarified in a comment.
| KV.of(c.element(), nextPending.getValue()), nextPending.getTimestamp()); | ||
| ++numEmitted; | ||
| } | ||
| LOG.debug("{} - emitted {} new results.", c.element(), numEmitted); |
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.
Made this log more comprehensive and added some clarifying variables.
| // unless output is complete or termination condition is reached. | ||
| if (tracker.shouldPollMore()) { | ||
| LOG.info( | ||
| "{} - emitted all known results so far; will resume polling in {} ms", |
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.
Done.
|
|
||
| @Override | ||
| public synchronized GrowthState<OutputT, KeyT, TerminationStateT> checkpoint() { | ||
| checkState( |
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.
Rejecting the checkpoint is not allowed. Allowing it is one of the alternatives I considered, but since runner needs access to positions anyway, I preferred to just do that.
| } | ||
| } | ||
|
|
||
| private static class SnappyCoder<T> extends StructuredCoder<T> { |
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.
Done.
chamikaramj
left a comment
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.
Thanks.
LGTM other than one comment.
| // the original restriction, i.e. pointless. | ||
| this.scheduledCheckpoint = | ||
| executor.schedule( | ||
| this::takeCheckpointNow, maxDuration.getMillis(), TimeUnit.MILLISECONDS); |
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.
This should be max((maxDuration - "time up to now"), 0) no ?
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'm not sure. I think both "10 seconds of claimed work" and "10 seconds of total work" are valid options, but I'm slightly in favor of the former because it's less likely to lead to pathological behavior, e.g. imagine that opening a connection to Kafka consistently takes 10+ seconds due to network issues, then the first behavior will lead to repeatedly reading just 1 record from Kafka (compared to 0 before this PR...), but the second will provide 10 seconds of useful work.
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.
Sounds good.
|
Would like @tgroh 's LGTM as well before proceeding with Dataflow worker changes. |
|
LGTM |
|
Test failures are unrelated. |
tgroh
left a comment
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.
My naming thing is because I really dislike the use of Impl as a signifier of the implementation, and generally want to give users the nicer name if possible, but it's purely a style thing.
c46da1b to
44ad0fe
Compare
|
Rebased but still waiting for some Dataflow worker side Google-internal stuff before I can merge. |
…first claim, and verifies more invariants
44ad0fe to
6857cb9
Compare
|
Run Dataflow ValidatesRunner |
|
Dataflow runner tests failed somewhere towards the end due to unrelated issues - I confirmed by looking at Jenkins output that SDF tests passed. Merging. |
|
Noting here, too, that |
This addresses the following issues:
The former is the primary motivation for this PR. This PR changes SDF checkpointing timer countdown to start from the first claimed block, rather than from the beginning of
@ProcessElement. This requires giving the runner visibility into claimed blocks. Such visibility enables fixing BEAM-2607 as well. It also is a required part of implementing SDF splitting over Fn API (tracked separately).This PR also, of course, changes the Watch transform to the new API; and, while we're at it, does some related improvements:
This PR is update-incompatible for users of the Watch transform, e.g. FileIO.match().continuously(). This is an experimental and very recent transform, so I'm going to ignore the incompatibility. It also requires a traditional Dataflow worker dance to get the worker container in sync with these runners-core changes - I'll perform that when the rest of the PR is approved.
R: @tgroh @chamikaramj
CC: @kennknowles @reuvenlax