[
https://issues.apache.org/jira/browse/BEAM-8088?focusedWorklogId=304569&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-304569
]
ASF GitHub Bot logged work on BEAM-8088:
----------------------------------------
Author: ASF GitHub Bot
Created on: 30/Aug/19 20:08
Start Date: 30/Aug/19 20:08
Worklog Time Spent: 10m
Work Description: chadrik commented on issue #9426: [BEAM-8088] Track
PCollection boundedness in python sdk
URL: https://github.com/apache/beam/pull/9426#issuecomment-526732412
> Just to clarify, this is to have an unbounded PCollection during the
expansion of the WriteToPubSub write?
Correct.
`PubSubIO.Write.expand` looks like this:
```java
@Override
public PDone expand(PCollection<T> input) {
if (getTopicProvider() == null) {
throw new IllegalStateException("need to set the topic of a
PubsubIO.Write transform");
}
switch (input.isBounded()) {
case BOUNDED:
input.apply(
ParDo.of(
new PubsubBoundedWriter(
MoreObjects.firstNonNull(getMaxBatchSize(),
MAX_PUBLISH_BATCH_SIZE),
MoreObjects.firstNonNull(
getMaxBatchBytesSize(),
MAX_PUBLISH_BATCH_BYTE_SIZE_DEFAULT))));
return PDone.in(input.getPipeline());
case UNBOUNDED:
return input
.apply(MapElements.via(getFormatFn()))
.apply(
new PubsubUnboundedSink(
Optional.ofNullable(getPubsubClientFactory()).orElse(FACTORY),
NestedValueProvider.of(getTopicProvider(), new
TopicPathTranslator()),
getTimestampAttribute(),
getIdAttribute(),
100 /* numShards */,
MoreObjects.firstNonNull(
getMaxBatchSize(),
PubsubUnboundedSink.DEFAULT_PUBLISH_BATCH_SIZE),
MoreObjects.firstNonNull(
getMaxBatchBytesSize(),
PubsubUnboundedSink.DEFAULT_PUBLISH_BATCH_BYTES)));
}
throw new RuntimeException(); // cases are exhaustive.
}
```
The `ReadFromPubsub` was expanded by Java, but Python was not properly
handling the unbounded PCollections that it returned, so when `WriteToPubsub`
was subsequently expanded, it always produced a `PubsubBoundedWriter`.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 304569)
Time Spent: 40m (was: 0.5h)
> PCollection boundedness should be tracked and propagated
> --------------------------------------------------------
>
> Key: BEAM-8088
> URL: https://issues.apache.org/jira/browse/BEAM-8088
> Project: Beam
> Issue Type: Improvement
> Components: sdk-py-core
> Reporter: Chad Dombrova
> Assignee: Chad Dombrova
> Priority: Major
> Time Spent: 40m
> Remaining Estimate: 0h
>
> As far as I can tell Python does not care about boundedness of PCollections
> even in streaming mode, but external transforms _do_. In my ongoing effort
> to get PubsubIO external transforms working I discovered that I could not
> generate an unbounded write.
> My pipeline looks like this:
> {code:python}
> (
> pipe
> | 'PubSubInflow' >>
> external.pubsub.ReadFromPubSub(subscription=subscription,
> with_attributes=True)
> | 'PubSubOutflow' >>
> external.pubsub.WriteToPubSub(topic=OUTPUT_TOPIC, with_attributes=True)
> )
> {code}
> The PCollections returned from the external Read are Unbounded, as expected,
> but python is responsible for creating the intermediate PCollection, which is
> always Bounded, and thus external Write is always Bounded.
--
This message was sent by Atlassian Jira
(v8.3.2#803003)