I created https://issues.apache.org/jira/browse/BEAM-490 which will track
the work for swapping the primitive from being GBK to CoGBK.

On Thu, Jul 21, 2016 at 9:24 AM, Lukasz Cwik <[email protected]> wrote:

> As of today, Cloud Dataflow will also be executed as a GBK.
>
> On Thu, Jul 21, 2016 at 2:56 AM, Aljoscha Krettek <[email protected]>
> wrote:
>
>> +1
>>
>> Out of curiosity, does Cloud Dataflow have a CoGBK primitive or will it
>> also be executed as a GBK there?
>>
>> On Thu, 21 Jul 2016 at 02:29 Kam Kasravi <[email protected]> wrote:
>>
>> > +1 - awesome Manu.
>> >
>> >     On Wednesday, July 20, 2016 1:53 PM, Kenneth Knowles
>> > <[email protected]> wrote:
>> >
>> >
>> >  +1
>> >
>> > I assume that the intent is for the semantics of both GBK and CoGBK to
>> be
>> > unchanged, just swapping their status as primitives.
>> >
>> > This seems like a good change, with strictly positive impact on users
>> and
>> > SDK authors, with only an extremely minor burden (doing an insertion of
>> the
>> > provided implementation in the worst case) on runner authors.
>> >
>> > Kenn
>> >
>> >
>> > On Wed, Jul 20, 2016 at 10:38 AM, Lukasz Cwik <[email protected]
>> >
>> > wrote:
>> >
>> > > I would like to propose a change to Beam to make CoGBK the basis for
>> > > grouping instead of GBK. The idea behind this proposal is that CoGBK
>> is a
>> > > more powerful operator then GBK allowing for two key benefits:
>> > >
>> > > 1) SDKs are simplified: transforming a CoGBK into a GBK is trivial
>> while
>> > > the reverse is not.
>> > > 2) It will be easier for runners to provide more efficient
>> > implementations
>> > > of CoGBK as they will be responsible for the logic which takes their
>> own
>> > > internal grouping implementation and maps it onto a CoGBK.
>> > >
>> > > This requires the following modifications to the Beam code base:
>> > >
>> > > 1) Make GBK a composite transform in terms of CoGBK.
>> > > 2) Move the CoGBK from contrib to runners-core as an adapter*. Runners
>> > that
>> > > more naturally support GBK can just use this and everything executes
>> > > exactly as before.
>> > >
>> > > *just like GroupByKeyViaGroupByKeyOnly and
>> UnboundedReadFromBoundedSource
>> > >
>> >
>> >
>> >
>>
>
>

Reply via email to