Skip to content

Conversation

@boyuanzz
Copy link
Contributor

@boyuanzz boyuanzz commented Apr 14, 2020

Computation logic is copied from python OffsetRestrictionTracker.try_split()

r: @lukecwik
cc: @youngoli


Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:

  • Choose reviewer(s) and mention them in a comment (R: @username).
  • Format the pull request title like [BEAM-XXX] Fixes bug in ApproximateQuantiles, where you replace BEAM-XXX with the appropriate JIRA issue, if applicable. This will automatically link the pull request to the issue.
  • Update CHANGES.md with noteworthy changes.
  • If this contribution is large, please file an Apache Individual Contributor License Agreement.

See the Contributor Guide for more tips on how to make review process smoother.

Post-Commit Tests Status (on master branch)

Lang SDK Apex Dataflow Flink Gearpump Samza Spark
Go Build Status --- --- Build Status --- --- Build Status
Java Build Status Build Status Build Status
Build Status
Build Status
Build Status
Build Status
Build Status Build Status Build Status
Build Status
Build Status
Python Build Status
Build Status
Build Status
Build Status
--- Build Status
Build Status
Build Status
Build Status
Build Status
--- --- Build Status
XLang --- --- --- Build Status --- --- Build Status

Pre-Commit Tests Status (on master branch)

--- Java Python Go Website
Non-portable Build Status Build Status
Build Status
Build Status Build Status
Portable --- Build Status --- ---

See .test-infra/jenkins/README for trigger phrase, status and link of all Jenkins jobs.

Copy link
Member

@lukecwik lukecwik left a comment

Choose a reason for hiding this comment

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

I looked at the Python implementation as well and I think we can improve it so that trySplit returns a split even if nothing has been claimed and so that we can get rid of checkpointed.

Copy link
Member

@lukecwik lukecwik Apr 14, 2020

Choose a reason for hiding this comment

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

We can split before any successfully claimed block by returning [from, to) and updating the current range to be [from, from)

This makes sense in some cases where we want to handoff all the work to someone else for the active element while this bundle finishes other processing.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Allowing split before first claiming makes sense to me. Python has already allowed that.

Copy link
Member

Choose a reason for hiding this comment

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

Why do we need checkpointed?

Shouldn't the range restriction change so that to becomes lastClaimed (or from if nothing has been claimed)?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Just return early since we know there is no more split after checkpointing.

Copy link
Member

Choose a reason for hiding this comment

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

Changing the state makes the code more complicated though since the bounds checking varies based upon whether you got 0.0 or 0.00001 which is likely to produce the same final state with one having an additional boolean being set.

@boyuanzz
Copy link
Contributor Author

Run Java PreCommit

@boyuanzz
Copy link
Contributor Author

Run RAT PreCommit

OffsetRange checkpoint = tracker.trySplit(0).getResidual();
SplitResult res = tracker.trySplit(0);
assertEquals(new OffsetRange(100, 100), res.getPrimary());
assertEquals(new OffsetRange(100, 200), res.getResidual());
Copy link
Member

Choose a reason for hiding this comment

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

primary and residual shouldn't have the same value, primary should be an empty range like [100, 100)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

In this test case, the expected primary is [100, 100) and the expected residual is [100, 200)

Copy link
Member

Choose a reason for hiding this comment

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

Thanks, that was my mistake, I read both lines as being [100, 200)

// residual is null when the entire restriction has been processed.
if (processContext.numClaimedBlocks > 0) {
residual = checkNotNull(processContext.takeCheckpointNow());
residual = processContext.takeCheckpointNow();
Copy link
Member

Choose a reason for hiding this comment

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

takeCheckpointNow should work regardless whether numClaimedBlocks > 0 or not.

Even if tryClaim never happens, the watermark may advance.

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 guess the original assumption is, checkpoint should happen after at least one tryClaim called. Since we change the assumption, the numClaimedBlocks can also be removed.

residual = checkNotNull(processContext.takeCheckpointNow());
residual = processContext.takeCheckpointNow();
processContext.tracker.checkDone();
} else {
Copy link
Member

Choose a reason for hiding this comment

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

The comments below will likely need updating

assertEquals(new OffsetRange(100, 161), tracker.currentRestriction());
assertEquals(new OffsetRange(161, 200), checkpoint);
assertNull(tracker.trySplit(0));
}
Copy link
Member

Choose a reason for hiding this comment

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

Suggested change
}
tracker.checkDone();
}

import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;

import org.apache.beam.sdk.io.range.OffsetRange;
Copy link
Member

Choose a reason for hiding this comment

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

Can we add tests to verify tryClaim(0), tryClaim(0.1), tryClaim(1) on an empty range like [100, 100)

Can we also add tests to verify the behavior of tryClaim(1) on range [100, 200)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

trySplit right?

Copy link
Member

Choose a reason for hiding this comment

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

Yes. Your right. Need more sleep.

}
this.residualRestriction = residualRestriction;
this.futureOutputWatermark = futureOutputWatermark;
this.futureWatermarkEstimatorState = futureWatermarkEstimatorState;
Copy link
Member

Choose a reason for hiding this comment

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

I believe the comment below could be incorrect. If we get stop(), we shouldn't have a residual restriction.

@boyuanzz
Copy link
Contributor Author

Run Java PreCommit

@lukecwik lukecwik merged commit fa4f418 into apache:master Apr 18, 2020
@iemejia
Copy link
Member

iemejia commented Apr 21, 2020

It seems this PR broke the ValidatesRunner tests for most runners. Can you guys PTAL. SplittableDoFnTest is failing with NullPointerException

For ref.
https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/7267/
https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/7297/

@mxm
Copy link
Contributor

mxm commented Apr 21, 2020

Running post commits here would have caught this (especially considering this is a new feature). The Flink ValidatesRunner test suite takes less than 10 minutes to complete. Perhaps we should move it to pre commit to be sure to catch these.

@lukecwik
Copy link
Member

Fix in #11475

1 similar comment
@lukecwik
Copy link
Member

Fix in #11475

yirutang pushed a commit to yirutang/beam that referenced this pull request Jul 23, 2020
…11418)

* [BEAM-8872] Support split at fraction for OffsetRangeTracker

* Address comments

* Update checkDone
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