[
https://issues.apache.org/jira/browse/KAFKA-3337?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15229181#comment-15229181
]
Guozhang Wang commented on KAFKA-3337:
--------------------------------------
[~mjsax] The pros of this approach is that 1) we can have a skinner "aggregate"
function with less parameters, 2) the "groupBy" following "aggregate" syntax is
more intuitive for SQL-like users.
The cons though, is that it introduces a third citizen of "KTableGroup" besides
KStream and KTable into the public APIs. So we want to see what is the outcome
of this approach with some demo examples and have a concrete feeling of the
trade-off.
> Extract selector as a separate groupBy operator for KTable aggregations
> -----------------------------------------------------------------------
>
> Key: KAFKA-3337
> URL: https://issues.apache.org/jira/browse/KAFKA-3337
> Project: Kafka
> Issue Type: Sub-task
> Components: kafka streams
> Reporter: Guozhang Wang
> Assignee: Matthias J. Sax
> Labels: api, newbie++
> Fix For: 0.10.1.0
>
>
> Currently KTable aggregation takes a selector used for selecting the
> aggregate key.and an aggregator for aggregating the values with the same
> selected key, which makes the function a little bit "heavy":
> {code}
> table.groupBy(initializer, adder, substractor, selector, /* optional serde*/);
> {code}
> It is better to extract the selector in a separate groupBy function such that
> {code}
> KTableGrouped KTable#groupBy(selector);
> KTable KTableGrouped#aggregate(initializer, adder, substractor, /* optional
> serde*/);
> {code}
> Note that "KTableGrouped" only have APIs for aggregate and reduce, and none
> else. So users have to follow the pattern below:
> {code}
> table.groupBy(...).aggregate(...);
> {code}
> This pattern is more natural for users who are familiar with SQL / Pig or
> Spark DSL, etc.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)