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-3851] Option to preserve element timestamp while publishing to Kafka. #4868

Merged
merged 3 commits into from Mar 22, 2018

Conversation

rangadi
Copy link
Contributor

@rangadi rangadi commented Mar 14, 2018

KafkaIO sink support for setting Kafka message timestamps based on element timestamp. Otherwise there is no way for user to influence the timestamp of the messages in Kafka sink.

The implementation for for normal sink (KafkaWriter.java) is trivial: Just need to read the timestamp from the context. But EOS sink (KafkaExactlyOnceWriter.java) changes are a bit more involved.

In the case of the latter, the elements go through couple of shuffles and we need to include timestamp along with the the actual value. The implementation wraps timestamp and input KVs in TimestampedValue<>. This changes serialization of the elements shuffles. As a result EOS changes are not backward compatible (with upgrade or while using save points). I think the use of EOS sink is pretty minimal and this will have very little impact. I am not sure what the best practice is for handling such incompatibility in Beam. Ideally we want to error out early if a pipeline with exactly-once sink is being updated from version 2.3 to 2.4. PLMK. I can move timestamp support in EOS to Beam 3.0.

@rangadi
Copy link
Contributor Author

rangadi commented Mar 14, 2018

R: @xumingmin
cc: @aljoscha, @chamikaramj

@mingmxu
Copy link

mingmxu commented Mar 19, 2018

I would suggest to make the change together, but I would prefer to a TimestampFn instead of ctx.timestamp(), --I find ctx.timestamp() of earlyFiring/lateFiring/ontimeFiring is quite complex and is not what I want sometime.

@rangadi
Copy link
Contributor Author

rangadi commented Mar 20, 2018

Added .withPublishTimestampFunction() (this is lambda friendly). Also retained withInputTimestamp() which makes it simpler for users to use element timestamp.

@mingmxu
Copy link

mingmxu commented Mar 20, 2018

retest this please

@mingmxu
Copy link

mingmxu commented Mar 21, 2018

run java precommit

@mingmxu
Copy link

mingmxu commented Mar 21, 2018

Seems the failure is not related with this change, @rangadi can you help to double check?

@rangadi
Copy link
Contributor Author

rangadi commented Mar 22, 2018

@xumingmin, yep, they seem unrelated.

@@ -190,12 +193,13 @@ public void setup() {
@ProcessElement
public void processElement(ProcessContext ctx) {
shardId = (shardId + 1) % numShards; // round-robin among shards.
ctx.output(KV.of(shardId, ctx.element()));
ctx.output(KV.of(shardId, TimestampedValue.of(ctx.element(), ctx.timestamp())));
Copy link

Choose a reason for hiding this comment

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

should this be

spec.getPublishTimestampFunction() != null
            ? spec.getPublishTimestampFunction().getTimestamp(record.getValue(),
                                                              record.getTimestamp()).getMillis()

?

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 is done couple of stages later while writing to Kafka (line 438). The timestamp here is element timestamp. I was wondering if we should invoke user function here or later. One wrinkle if we want to do what you suggest is how to carry default 'null' down stream. We could have an optional to carry it.

Copy link

Choose a reason for hiding this comment

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

ahhh I see it, thanks for pointing out

Copy link

@mingmxu mingmxu left a comment

Choose a reason for hiding this comment

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

LGTM, merging

@mingmxu mingmxu merged commit 501dc4c into apache:master Mar 22, 2018
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

2 participants