Skip to content

Conversation

@mshields822
Copy link
Contributor

This also refines the handling of record ids in the sink to be random-but-reused-on-failure, using the same trick as we do for the BigQuery sink.

Still need to re-do the load tests I did a few weeks back with the actual change.
Note that last time I tested the DataflowPipelineTranslator does not kick in and replace the new transforms with the correct native transforms. Need to dig deeper.

R: @dhalperi

context.addStep(transform, "ParallelRead");
context.addInput(PropertyNames.FORMAT, "pubsub");
if (transform.getTopic() != null) {
context.addInput(PropertyNames.PUBSUB_TOPIC, transform.getTopic().asV1Beta1Path());
Copy link
Contributor

Choose a reason for hiding this comment

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

is this no longer supported? is this a regression of some sort?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Only the dataflow service has a notion of 'cleanup when the job is done', and so can delete random subscriptions created if PubsubIO.Read is only given a topic. So PubsubUnboundedSource requires a subscription and PubsubIO.Read requires it to be specified.

But then this translation approach won't work since the topic has not been captured by the PubsubUnboundedSource. Dang, missed that, thanks for noticing. Perhaps I can look at the parent PubsubIO.Read PTransform?

Copy link
Contributor

@dhalperi dhalperi May 18, 2016

Choose a reason for hiding this comment

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

I think that we should leave in the ability to create a subscription on-demand, but record an ugly log message that there will be a leftover subscription and proceed. We have finalizers/fault tolerance on the pipeline roadmap, but it's not here yet.

I feel like there's a JIRA issue for this general issue, but cannot find it.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ugly, but I capture both the subscription and topic and defer we're given a subscription to the PubsubUnboundedSource.apply.

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 really don't like it.
The batch direct runner creates the subscription on one worker when the pipeline begins data processing, cleans up when done.
The streaming Google Cloud Dataflow runner creates the subscription on dax when the job is setup, cleans up when the job is terminated.
The streaming Java-only runner would create it when the pipeline graph is constructed, and never cleans it up.
The lack of cleanup plus phase difference of creation time all feels wrong...

... but I'll do it anyway.

@dhalperi
Copy link
Contributor

Keep pinging me or @peihe as you get tests going and if you have trouble with the translator.

@mshields822
Copy link
Contributor Author

No luck running against apache_beam-on-google runner (NoClassDefFound exceptions).
Can you give this one more look and if you're happy I'll port to dataflow and confirm all is well there, then return back here?

@mshields822
Copy link
Contributor Author

Ok I'm back after getting the Google Cloud Dataflow runner rewrites working.

// In streaming mode must use either the custom Pubsub unbounded source/sink or
// defer to Windmill's built-in implementation.
builder.put(PubsubIO.Read.Bound.PubsubBoundedReader.class, UnsupportedIO.class);
builder.put(PubsubIO.Write.Bound.PubsubBoundedWriter.class, UnsupportedIO.class);
Copy link
Contributor

Choose a reason for hiding this comment

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

These are both DoFns, not PTransforms, so I do not think this will have any effect.

@kennknowles any suggestions?

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 added support for this in UnsupportedIO below.

Copy link
Contributor

Choose a reason for hiding this comment

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

ack

@mshields822
Copy link
Contributor Author

PTAL

@mshields822
Copy link
Contributor Author

Confirmed working with beam-on-dataflow + beam-worker for both java-only and internal pubsub sources/sinks.

? "streaming" : "batch";
String name =
transform == null ? approximateSimpleName(doFn.getClass()) :
approximatePTransformName(transform.getClass());
Copy link
Contributor

Choose a reason for hiding this comment

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

fix wrapping here? : should be at the start of the line

@dhalperi
Copy link
Contributor

LGTM. I can fix the : and then merge.

@mshields822
Copy link
Contributor Author

Thanks!

On Thu, May 19, 2016 at 6:27 PM, asfgit notifications@github.com wrote:

Closed #346 #346 via
d0b9ca9
d0b9ca9
.


You are receiving this because you authored the thread.
Reply to this email directly or view it on GitHub
#346 (comment)

dhalperi pushed a commit to dhalperi/beam that referenced this pull request Aug 23, 2016
* Add ValueProvider

* Update DataflowPipelineRunner.java

* Update DataflowPipelineRunnerHooks.java

* Update DataflowPipelineRunnerHooks.java

* Create TemplatingDataflowPipelineRunner.java

* Add test.'

* update

* add test

* Fix messages

* Remove ValueProvider

* Fix formatting.

* Remove dupe

* Fix typo

* Fix style errors.

* Fix style errors.

* Fix things

* fixes

* Update TemplatingDataflowPipelineRunnerTest.java

* Do not return null, return a useless Job instead.

* Update TemplatingDataflowPipelineRunner.java

* Update TemplatingDataflowPipelineRunner.java
iemejia pushed a commit to iemejia/beam that referenced this pull request Jan 12, 2018
pl04351820 pushed a commit to pl04351820/beam that referenced this pull request Dec 20, 2023
* fix: Fixes apache#346 by reseeding for each auto id on py3.6
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.

4 participants