[ 
https://issues.apache.org/jira/browse/BEAM-92?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15178173#comment-15178173
 ] 

Daniel Halperin commented on BEAM-92:
-------------------------------------

Abstractly, this is very easy to accomplish when the Sink is replaced by a 
DoFn. You can generate a KV<SinkT, T> with a ParDo(element -> 
KV.of(where(element), element))). Then a GroupByKey will give you a KV<SinkT, 
Iterable<T>> and then all the elements can be written to the sink.

However, of course, given that a single worker is processing all the elements, 
this will essentially be single-threaded writing.

This seems much more complicated when taking into account fault tolerance, 
finalizers, etc. Lots of design to do here.

> Data-dependent sinks
> --------------------
>
>                 Key: BEAM-92
>                 URL: https://issues.apache.org/jira/browse/BEAM-92
>             Project: Beam
>          Issue Type: New Feature
>            Reporter: Eugene Kirpichov
>
> Current sink API writes all data to a single destination, but there are many 
> use cases where different pieces of data need to be routed to different 
> destinations where the set of destinations is data-dependent (so can't be 
> implemented with a Partition transform).
> One internally discussed proposal was an API of the form:
> {code}
> PCollection<Void> PCollection<T>.apply(
>     Write.using(DoFn<T, SinkT> where,
>                 MapFn<SinkT, WriteOperation<WriteResultT, T>> how)
> {code}
> so an item T gets written to a destination (or multiple destinations) 
> determined by "where"; and the writing strategy is determined by "how" that 
> produces a WriteOperation (current API - global init/write/global finalize 
> hooks) for any given destination.
> This API also has other benefits:
> * allows the SinkT to be computed dynamically (in "where"), rather than 
> specified at pipeline construction time
> * removes the necessity for a Sink class entirely
> * is sequenceable w.r.t. downstream transforms (you can stick transforms onto 
> the returned PCollection<Void>, while the current Write.to() returns a PDone)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to