Skip to content
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

[BEAM-2467] Kinesis source watermark based on approximateArrivalTimestamp #3851

Closed

Conversation

pawel-kaczmarczyk
Copy link
Contributor

Follow this checklist to help us incorporate your contribution quickly and easily:

  • Make sure there is a JIRA issue filed for the change (usually before you start working on it). Trivial changes like typos do not require a JIRA issue. Your pull request should address just this issue, without pulling in other changes.
  • Each commit in the pull request should have a meaningful subject line and body.
  • Format the pull request title like [BEAM-XXX] Fixes bug in ApproximateQuantiles, where you replace BEAM-XXX with the appropriate JIRA issue.
  • Write a pull request description that is detailed enough to understand what the pull request does, how, and why.
  • Run mvn clean verify to make sure basic checks pass. A more thorough check will be performed on your pull request automatically.
  • If this contribution is large, please file an Apache Individual Contributor License Agreement.

@pawel-kaczmarczyk
Copy link
Contributor Author

@jkff @rangadi @rtshadow, can you please take a look?

@@ -92,6 +98,7 @@ public boolean advance() throws IOException {
} catch (TransientKinesisException e) {
LOG.warn("Transient exception occurred", e);
}
watermark = Instant.now();
Copy link
Contributor

Choose a reason for hiding this comment

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

@pawel-kaczmarczyk - I don't understand why do we do that. Let's say we have situation like this:

  • we're really lagging behind on reading Kinesis stream, i.e. unread records have approximateArrivalTimestamp much less than Instant.now()
  • we're trying to read next record, but there are some network problems and TransientKinesisException is thrown
  • we advance watermark to Instant.now() although we probably shouldn't?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Good catch, it should be rather done inside the try-catch block just after the loop. Will fix that

Instant approximateArrivalTimestamp = currentRecord.get()
.getApproximateArrivalTimestamp();
if (approximateArrivalTimestamp.isAfter(watermark)) {
watermark = approximateArrivalTimestamp;
Copy link
Contributor

Choose a reason for hiding this comment

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

This doesn't work nicely with a scheme where we read data from shards in round robin way, because you may advance watermark based on a first shared although you've got older data in other shards.

What about changing this scheme to actually read the shard where records are most outdated? This will improve the whole solution greatly.

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 know it's not ideal but I would treat it as a temporary solution. I'm planning to change the way the streams are read in BEAM-2468

Copy link
Contributor

Choose a reason for hiding this comment

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

Oh interesting, I didn't realize that the number of shards is fixed. Then at some point this connector can be considerably simplified using Splittable DoFn, by decomposing it into two DoFn's - one (non-splittable) listing the shards, and the other (splittable) reading a single shard. See https://beam.apache.org/blog/2017/08/16/splittable-do-fn.html .

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The number of shards is not fixed actually but the stream resharding is not supported currently by this connector. I'm planning to fix that in https://issues.apache.org/jira/browse/BEAM-2469.
I've heard about this Splittable DoFns idea some time ago but didn't delve into details yet. Sounds really interesting!

return response;
}

private boolean isEmptyResponseBeforeStreamEnd(GetKinesisRecordsResult response) {
Copy link
Contributor

Choose a reason for hiding this comment

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

this name doesn't tell me much. Maybe something like:
notEnoughRecordsPulled?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Current name also emphasises that we still haven't catch up with the stream but somehow we got the empty response (it can really happen). I would rather stay with current name. What do you think?

Copy link
Contributor

Choose a reason for hiding this comment

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

So maybe simply: gotEmptyResponseButIsBeforeEndOfTheStream

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ok, I like that

try {
response = kinesis.getRecords(shardIterator, checkpoint.getStreamName(),
checkpoint.getShardId());
shardIterator = response.getNextShardIterator();
Copy link
Contributor

Choose a reason for hiding this comment

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

Why do you get new shard iterator each time?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

ShardIterator has to be advanced after successful response in order to iterate over a shard. Isn't that correct?

Copy link
Contributor

Choose a reason for hiding this comment

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

Sorry, my bad. Please ignore my comment.

@pawel-kaczmarczyk
Copy link
Contributor Author

@jkff @rangadi can you please take a look?

Copy link
Contributor

@jkff jkff left a comment

Choose a reason for hiding this comment

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

Thanks!

Instant approximateArrivalTimestamp = currentRecord.get()
.getApproximateArrivalTimestamp();
if (approximateArrivalTimestamp.isAfter(watermark)) {
watermark = approximateArrivalTimestamp;
Copy link
Contributor

Choose a reason for hiding this comment

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

Oh interesting, I didn't realize that the number of shards is fixed. Then at some point this connector can be considerably simplified using Splittable DoFn, by decomposing it into two DoFn's - one (non-splittable) listing the shards, and the other (splittable) reading a single shard. See https://beam.apache.org/blog/2017/08/16/splittable-do-fn.html .

shardIterator = response.getNextShardIterator();
data.addAll(filter.apply(response.getRecords(), checkpoint));
}
} while (response == null || gotEmptyResponseButIsBeforeEndOfTheStream(response));
Copy link
Contributor

Choose a reason for hiding this comment

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

Hmm, the control flow here looks a bit convoluted. Can you refactor this into a "while(true)" loop with explicit breaks on exit conditions? Currently I'm confused by seeing "while (response == null..." whereas response is being used at line 91.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Actually I really prefer to not use breaks in the code so that the loop is controlled in one place only. This loop is also kind of a temporary solution (or more of a workaround) for Kinesis behaviour. Kinesis can return 0 records during iteration before reaching the end of the stream while there may be some more events waiting. Current KinesisReader.advance() logic would treat that case as reaching the end of the stream and move the watermark to current timestamp.
I'm planning to implement Kinesis reading in the background threads so that's only a temporary loop that won't be needed anymore then.

}

private boolean gotEmptyResponseButIsBeforeEndOfTheStream(GetKinesisRecordsResult response) {
return response.getRecords().isEmpty() && response.getMillisBehindLatest() > 0;
Copy link
Contributor

Choose a reason for hiding this comment

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

Hmm is this new exit condition related to watermark handling, or is it a bugfix? I'm not sure I understand the reason for the difference between old and new behavior.

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 actually answered that in my previous comment so just in short: Kinesis can return 0 records before reaching the end of the stream and that would cause problems with current watermark handling.

@@ -84,11 +85,17 @@ public boolean advance() throws IOException {
for (int i = 0; i < shardIterators.size(); ++i) {
currentRecord = shardIterators.getCurrent().next();
if (currentRecord.isPresent()) {
Instant approximateArrivalTimestamp = currentRecord.get()
.getApproximateArrivalTimestamp();
if (approximateArrivalTimestamp.isAfter(watermark)) {
Copy link
Contributor

Choose a reason for hiding this comment

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

So we declare that watermark is the latest arrival timestamp seen so far - this seems like a very aggressive strategy for advancing the watermark, because it means that if records are even slightly out of order, they will be declared late data. PubsubIO uses a less aggressive strategy: https://github.com/apache/beam/blob/master/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java#L88 - would it be a good idea to use something similar here? (if you believe the current PR is already a vast improvement over the state of the code before this PR, then I'm fine getting it in as-is, but it's something worth thinking about).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It sounds like a really good idea to implement it in a similar way. I'll prepare the changes shortly.

…tamp - less aggressive strategy for advancing watermark
@pawel-kaczmarczyk
Copy link
Contributor Author

@jkff I've just pushed new changes, could you please review?

Copy link
Contributor

@jkff jkff left a comment

Choose a reason for hiding this comment

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

Thanks! Mostly ready to merge (I can do final fixups myself), but can you confirm whether you've tried running this against a real Kinesis and got reasonable autoscaling behavior?

throws IOException, TransientKinesisException {
final long timestampMs = 1000L;

KinesisRecord firstRecord = prepareRecordMockWithArrivalTimestamp(timestampMs);
Copy link
Contributor

Choose a reason for hiding this comment

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

Seems like lines 135-142 can be abstracted into a nice helper method:
prepareRecordsWithArrivalTimestamps(timestampMs, 1 /* increment /, 5 / count */);

and it can be reused in the 2 other tests

firstIteratorStubbing.thenReturn(CustomOptional.<KinesisRecord>absent());
when(secondIterator.next()).thenReturn(CustomOptional.<KinesisRecord>absent());

reader.start();
Copy link
Contributor

Choose a reason for hiding this comment

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

Consider using the common pattern:

for (boolean more = reader.start(); more; more = reader.advance())

@pawel-kaczmarczyk
Copy link
Contributor Author

Thanks @jkff, I've already applied your test suggestions.
We are using our custom Kinesis source prepared for Dataflow SDKs and it autoscales reasonably with Kinesis. It uses exactly the same logic as proposed in #3551. So I can confirm that this approach should be fine here.

Copy link
Contributor

@jkff jkff left a comment

Choose a reason for hiding this comment

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

Thanks! LGTM, merging.

@asfgit asfgit closed this in 3bf65dc Sep 25, 2017
@coveralls
Copy link

Coverage Status

Coverage increased (+0.009%) to 69.525% when pulling 30fcbef on ocadotechnology:kinesis_watermark into fa4ecea on apache:master.

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.

None yet

4 participants