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-4652] Allow PubsubIO to read public data #5788

Merged
merged 10 commits into from
Jun 30, 2018

Conversation

kennknowles
Copy link
Member

This adds an integration test run for GCP IOs and then adds an integration test for reading a read public data set.

It does not pass, but it also does not fail in the same way that my local testing fails. When I write a pipeline to read this dataset I get a 403 on creating a subscription. The logs that gradle captures are a bit different.


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

  • 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.

It will help us expedite review of your Pull Request if you tag someone (e.g. @username) to look at it.

Post-Commit Tests Status (on master branch)

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

Copy link
Member Author

@kennknowles kennknowles left a comment

Choose a reason for hiding this comment

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

@chamikaramj @lgajowy do you know anything about how I should set this up?

PCollection<PubsubMessage> messages =
pipeline.apply(
PubsubIO.readMessages()
.fromTopic("projects/pubsub-public-data/topics/taxirides-realtime"));
Copy link
Member Author

Choose a reason for hiding this comment

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

@rangadi do you know about this?

"waitForAnyMessage", signal.signalSuccessWhen(messages.getCoder(), anyMessage -> true));

Duration timeout =
pipeline.getOptions().getRunner().getSimpleName().endsWith("DataflowRunner")
Copy link
Member Author

Choose a reason for hiding this comment

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

Hack so we don't have to it forever waiting for a DirectRunner to fail.

? Duration.standardMinutes(5)
: Duration.standardSeconds(30);
signal.waitForSuccess(timeout);
}
Copy link
Contributor

@lgajowy lgajowy Jun 28, 2018

Choose a reason for hiding this comment

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

I don't know much about Pubsub and testing it (we tested batch scenarios only yet), but doesn't this test require pipeline.run() invocation (sorry if this is my ignorance)?

Copy link
Member Author

Choose a reason for hiding this comment

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

Lol. Yes. And I was sprinkling debugging throughout, asking myself "why isn't createSubscription ever begin called?!"

Copy link
Member Author

Choose a reason for hiding this comment

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

OTOH the TestPipeline is supposed to complain if run() is not called. But I guess it never reaches that error because of the signal timeout.

Copy link
Contributor

Choose a reason for hiding this comment

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

Yes, I also think this is the case. I'm glad this helped a little bit. :)

@kennknowles
Copy link
Member Author

OK, I have now confirmed that if I run the pipeline the test fails before my fix and succeeds after. So it is a real problem.

@@ -1297,7 +1298,9 @@ private SubscriptionPath createRandomSubscription(PipelineOptions options) {
timestampAttribute, idAttribute, options.as(PubsubOptions.class))) {
SubscriptionPath subscriptionPath =
pubsubClient.createRandomSubscription(
project.get(), topic.get(), DEAULT_ACK_TIMEOUT_SEC);
PubsubClient.projectPathFromId(options.as(GcpOptions.class).getProject()),
Copy link
Member Author

Choose a reason for hiding this comment

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

Copy link
Member Author

Choose a reason for hiding this comment

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

But this is not quite right - it should respect the project ValueProvider if one was provided, yet default it to providing this.

@kennknowles
Copy link
Member Author

run java postcommit

@kennknowles kennknowles changed the title [BEAM-4652] Pubsub public data [BEAM-4652] Allow PubsubIO to read public data Jun 28, 2018
@kennknowles
Copy link
Member Author

run java postcommit

@@ -1235,7 +1235,7 @@ public PubsubUnboundedSource(
/** Get the project path. */
@Nullable
public ProjectPath getProject() {
return project == null ? null : project.get();
return project != null ? null : project.get();
Copy link
Contributor

Choose a reason for hiding this comment

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

typo?

Copy link
Member Author

Choose a reason for hiding this comment

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

Done

@@ -693,9 +693,9 @@ public String toString() {

@Nullable
ValueProvider<ProjectPath> projectPath =
getTopicProvider() == null
getSubscriptionProvider() == null
Copy link
Contributor

Choose a reason for hiding this comment

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

Does this require user to provide subscription rather than topic for read? The integration test test added here specifies topic.

Copy link
Member Author

Choose a reason for hiding this comment

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

Here is what I can tell:

  • PubsubUnboundedSource is made so that you can customize the project for the subscription.
  • But PubsubIO is not made to customize the project.

Before my change:

  • PubsubIO always uses the project off the topic if it is read().fromTopic() and leaves it null if it is read().fromSubscription()
  • PubsubUnboundedSource always requires a project if it is given a topic because it needs it to create the random subscription

After my change:

  • PubsubIO always uses the project from the subscription and leaves it null for read().fromTopic()
  • PubsubUnboundedSource never requires a project, because it defaults to getting it from PipelineOptions

So I guess actually since PubsubIO never actually provides a useful project, it could always be left null. Or probably better to refactor PubsubUnboundedSource to have two variants with some shared internals so there are zero nullable fields.

Copy link
Contributor

Choose a reason for hiding this comment

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

Thanks. That makes sense. I was also wondering why createReader() also creates a random subscription if required (that does not sound correct). As you mentioned, may be best to always require project to be present in GcpOptions.


StatefulPredicateCheck(Coder<T> coder, SerializableFunction<Set<T>, Boolean> successPredicate) {
this.seenEvents = StateSpecs.set(coder);
this.seenEvents = StateSpecs.bag(coder);
Copy link
Member Author

Choose a reason for hiding this comment

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

FYI @akedin the only reason to use SetState is if you want to do efficient membership checking. Since the code here reads the whole thing, it doesn't add any functionality. And more runners support BagState so I switched it.

@kennknowles
Copy link
Member Author

Precommit failed due to formatting and Postcommit failed because Dataflow doesn't support SetState. Should both be fixed. I'll run them both again.

@kennknowles
Copy link
Member Author

run java postcommit

@rangadi
Copy link
Contributor

rangadi commented Jun 28, 2018

PubsubIO/PubsubUnboundedSource changes LGTM. We can even just require --project option in GcpOptions to be always present, which is quite normal for many GCP components.

@kennknowles
Copy link
Member Author

run java postcommit

1 similar comment
@kennknowles
Copy link
Member Author

run java postcommit

@kennknowles kennknowles force-pushed the pubsub-public-data branch 3 times, most recently from de5f515 to 93ea4b4 Compare June 29, 2018 15:17
@kennknowles
Copy link
Member Author

run java postcommit

@kennknowles kennknowles force-pushed the pubsub-public-data branch 5 times, most recently from 3de0308 to 6ff715e Compare June 29, 2018 21:20
@kennknowles
Copy link
Member Author

run java postcommit

@kennknowles
Copy link
Member Author

run java postcommit

@kennknowles
Copy link
Member Author

run java postcommit

@kennknowles
Copy link
Member Author

run java postcommit

@kennknowles
Copy link
Member Author

It passed! The Python failure is on master, and fixed separately.

@kennknowles kennknowles merged commit d6c5bf9 into apache:master Jun 30, 2018
}
}
});
Future<List<String>> queryResult =
Copy link
Member Author

Choose a reason for hiding this comment

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

@kennknowles kennknowles deleted the pubsub-public-data branch July 3, 2018 21:55
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

3 participants