Skip to content

Conversation

@thinhha
Copy link
Contributor

@thinhha thinhha commented Apr 21, 2019

Make the recordId field from PubsubClient.IncomingMessage available as a new field called messageId in PubsubMessage.
Updated the coder for PubsubMessage to encode and decode the messageId.


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.
  • If this contribution is large, please file an Apache Individual Contributor License Agreement.

Post-Commit Tests Status (on master branch)

Lang SDK Apex Dataflow Flink Gearpump Samza Spark
Go Build Status --- --- --- --- --- ---
Java 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 --- --- ---

Pre-Commit Tests Status (on master branch)

--- Java Python Go Website
Non-portable 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.

@thinhha
Copy link
Contributor Author

thinhha commented Apr 21, 2019

R: @lukecwik

@thinhha
Copy link
Contributor Author

thinhha commented Apr 21, 2019

retest this please

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.

Changing the encoding of the coder will break pipeline update since the encoding used before is not compatible with the new encoding. You'll want to:

  1. add a new coder that is like PubsubMessageWithAttributesCoder
  2. add a method like readMessagesWithAttributes that uses the coders that you created in 1.

Repeat steps 1 and 2 for readMessages if you also want to create a readMessagesWithMessageId

You'll also want to update the implementation in the Dataflow specific PubsubReader here when constructing the PubsubMessage to pass in the message id as well:

@thinhha thinhha force-pushed the BEAM-3489 branch 2 times, most recently from a2d1aff to 6f781b2 Compare May 4, 2019 23:07
@thinhha
Copy link
Contributor Author

thinhha commented May 5, 2019

Thanks - I've added a few changes based on your comment. It's my first PR so sorry for all the questions!

  • For PubsubMesageWithMessageIdCoder, do I need to explicitly skip the attributes map which is second in pubsub.proto before getting to the message_id (which is third)?
  • Where would be a good place to add some tests for this PR?

public PubsubMessage(byte[] payload, Map<String, String> attributes) {
this.message = payload;
this.attributes = attributes;
this.messageId = "";
Copy link
Member

Choose a reason for hiding this comment

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

I think you should set this to null.

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.

Glad to help with the questions.

Your current implementation for PubsubMesageWithMessageIdCoder is correct. You don't need to encode/decode an empty map as there is no point.

Tests would typically go here:
https://github.com/apache/beam/tree/master/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub

You could add an integration test that shows that reading messages with a message id works here:
https://github.com/apache/beam/blob/master/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadIT.java
You could also add unit tests for your coders similar to
https://github.com/apache/beam/blob/master/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/MapCoderTest.java
Since your coder is not saying its deterministic or consistent with equals, you can ignore testing for those. Feel free to add additional unit tests for the other pubsub coders that are missing if you want.

import org.apache.beam.sdk.coders.StringUtf8Coder;
import org.apache.beam.sdk.values.TypeDescriptor;

/** A coder for PubsubMessage including attributes. */
Copy link
Member

Choose a reason for hiding this comment

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

please update comment to state that you also include the message id.

import org.apache.beam.sdk.coders.StringUtf8Coder;
import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableMap;

/** A coder for PubsubMessage treating the raw bytes being decoded as the message's payload. */
Copy link
Member

Choose a reason for hiding this comment

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

Please update this comment as well that it contains the message id.

* Class representing a Pub/Sub message. Each message contains a single message payload and a map of
* attached attributes.
* Class representing a Pub/Sub message. Each message contains a single message payload, a map of
* attached attributes, and an optional messageId.
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
* attached attributes, and an optional messageId.
* attached attributes, and a message id.

@lukecwik
Copy link
Member

lukecwik commented May 6, 2019

Looking good so far, minor comment updates and some tests and this will be merged.

@lukecwik
Copy link
Member

@thinhha any updates?

@thinhha
Copy link
Contributor Author

thinhha commented May 26, 2019

Sorry for the delay! I've added some changes based on your comment.

I've added an additional integration test, hopefully it will be OK.

@lukecwik
Copy link
Member

Run Java PostCommit

@lukecwik
Copy link
Member

I kicked off the postcommit to validate that the IT you added.

signal.signalSuccessWhen(
messages.getCoder(),
pubsubMessages ->
pubsubMessages.stream().noneMatch(m -> Strings.isNullOrEmpty(m.getMessageId()))));
Copy link
Member

Choose a reason for hiding this comment

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

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Looks like the IT passed this time. Could you take another look?

Thanks!

@thinhha
Copy link
Contributor Author

thinhha commented Jun 12, 2019

Run Java PostCommit

@thinhha
Copy link
Contributor Author

thinhha commented Jun 23, 2019

Run Java PostCommit

@thinhha
Copy link
Contributor Author

thinhha commented Jun 23, 2019

Run Java PreCommit

@thinhha
Copy link
Contributor Author

thinhha commented Jun 26, 2019

Run Java PostCommit

2 similar comments
@lukecwik
Copy link
Member

lukecwik commented Jul 9, 2019

Run Java PostCommit

@thinhha
Copy link
Contributor Author

thinhha commented Jul 28, 2019

Run Java PostCommit

@thinhha
Copy link
Contributor Author

thinhha commented Jul 28, 2019

Run Java PostCommit

@thinhha
Copy link
Contributor Author

thinhha commented Jul 28, 2019

Run Java PostCommit

@thinhha
Copy link
Contributor Author

thinhha commented Jul 28, 2019

Run Java PostCommit

1 similar comment
@thinhha
Copy link
Contributor Author

thinhha commented Jul 28, 2019

Run Java PostCommit

@thinhha
Copy link
Contributor Author

thinhha commented Jul 28, 2019

Run Java PostCommit

@thinhha
Copy link
Contributor Author

thinhha commented Jul 28, 2019

Run Java PreCommit

@thinhha
Copy link
Contributor Author

thinhha commented Jul 28, 2019

Run Java PostCommit

1 similar comment
@thinhha
Copy link
Contributor Author

thinhha commented Jul 28, 2019

Run Java PostCommit

@thinhha thinhha force-pushed the BEAM-3489 branch 2 times, most recently from e669585 to 8af31b1 Compare July 28, 2019 23:34
@thinhha
Copy link
Contributor Author

thinhha commented Jul 28, 2019

Run Java PostCommit

@thinhha
Copy link
Contributor Author

thinhha commented Aug 4, 2019

Hi @lukecwik, looks like the non-messageId coders are hard-coded in PubsubUnboundedSource:

Line 1088 of beam/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java:

    @Override
    public Coder<PubsubMessage> getOutputCoder() {
      return outer.getNeedsAttributes()
          ? PubsubMessageWithAttributesCoder.of()
          : PubsubMessagePayloadOnlyCoder.of();
    }

This means that while the PubsubMessage is created with the messageId field populated when using PubsubIO.readMessagesWithMessageId()/PubsubIO.readMessagesWithAttributesAndMessageId() , this field does not show up in the subsequent step.

Looks like setNeedsAttributes is a required field. I'm not sure how to change getOutputCoder()
without breaking something.

How do you think this case can be handled?

@lukecwik
Copy link
Member

lukecwik commented Aug 5, 2019

Add another constructor to PubsubUnboundedSource where you pass in a boolean as to whether you need the message id and update the getOuptutCoder to look at this property selecting between the 4 pubsub message coder variants. You'll also want to update

to look at this property and select between the coder with the message id and attributes or just the attributes coder based upon the new boolean that you added.

@lukecwik
Copy link
Member

lukecwik commented Aug 9, 2019

It turns out that one of the Dataflow services doesn't output the message id which makes this change not work with Dataflow until the service is updated. @udim is working on getting that resolved.

@thinhha thinhha force-pushed the BEAM-3489 branch 3 times, most recently from a2d935c to c02cdbd Compare August 11, 2019 17:25
@thinhha
Copy link
Contributor Author

thinhha commented Aug 11, 2019

Thanks for the update @lukecwik. Good thing the IT caught this!

I've added the changes we discussed above. We can rerun the postCommit tests once the dataflow backend change is complete.

@thinhha
Copy link
Contributor Author

thinhha commented Aug 17, 2019

Run Java PostCommit

@thinhha
Copy link
Contributor Author

thinhha commented Aug 18, 2019

Run Java PostCommit

@thinhha
Copy link
Contributor Author

thinhha commented Aug 28, 2019

All tests are passed. Friendly ping for review @lukecwik.

@lukecwik lukecwik merged commit a9e14ff into apache:master Aug 30, 2019
@lukecwik
Copy link
Member

Thanks a lot for contributing this and dealing with the additional complexity that Dataflow added.

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.

3 participants