You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 9 Next »

Status

Current state: Under Discussion

Discussion thread: TBD

JIRA: TBD

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

When multiple streams aggregate together to form a single larger object (eg. A shopping website may have a cart stream, a wish list stream, and a purchases stream. Together they make up a Customer.), it is very difficult to accommodate this in the Kafka-Streams DSL. It generally requires you to group and aggregate all of the streams to KTables then make multiple outerjoin calls to end up with a KTable with your desired object. This will create a state store for each stream and a long chain of ValueJoiners that each new record must go through to get to the final object.
Creating a cogroup method where you use a single state store will:

  1.  Reduce the number of gets from state stores. With the multiple joins when a new value comes into any of the streams a chain reaction happens where ValueGetters keep calling ValueGetters until we have accessed all state stores.
  2. Slight performance increase. As described above all ValueGetters are called also causing all ValueJoiners to be called forcing a recalculation of the current joined value of all other streams, impacting performance.

 

Example with Current API:

KTable<K, V1> table1 = builder.stream("topic1").groupByKey().aggregate(initializer1, aggregator1, aggValueSerde1, storeName1);
KTable<K, V2> table2 = builder.stream("topic2").groupByKey().aggregate(initializer2, aggregator2, aggValueSerde2, storeName2);
KTable<K, V3> table3 = builder.stream("topic3").groupByKey().aggregate(initializer3, aggregator3, aggValueSerde3, storeName3);
KTable<K, CG> cogrouped = table1.outerJoin(table2, joinerOneAndTwo).outerJoin(table3, joinerOneTwoAndThree);

 

As you can see this creates 3 StateStores, requires 3 initializers, and 3 aggValueSerdes. This also adds the pressure to user to define what the intermediate values are going to be (V1, V2, V3). They are left with a couple choices, first to make V1, V2, and V3 all the same as CG and the two joiners are more like mergers, or second make them intermediate states such as Topic1Map, Topic2Map, and Topic3Map and the joiners use those to build the final aggregate CG value. This is something the user could avoid thinking about with this KIP.

When a new input arrives lets say at "topic1" it will first go through a KStreamAggregate grabbing the current aggregate from storeName1. It will produce this in the form of the first intermediate value and get sent through a KTableKTableOuterJoin where it will look up the current value of the key in storeName2. It will use the first joiner to calculate the second intermediate value, which will go through an additional KTableKTableOuterJoin. Here it will look up the current value of the key in storeName3 and use the second joiner to build the final aggregate value.

If you think through all possibilities for incoming topics you will see that no matter which topic it comes in through all three stores are queried and all of the joiners must get used.

Topology wise for N incoming streams this creates N KStreamAggregates, 2*(N-1) KTableKTableOuterJoins, and N-1 KTableKTableJoinMergers.

 

Example with Proposed API:

KGroupedStream<K, V1> grouped1 = builder.stream("topic1").groupByKey();
KGroupedStream<K, V2> grouped2 = builder.stream("topic2").groupByKey();
KGroupedStream<K, V3> grouped3 = builder.stream("topic3").groupByKey();
KTable<K, CG> cogrouped = grouped1.cogroup(initializer1, aggregator1, aggValueSerde1, storeName1)
        .cogroup(grouped2, aggregator2)
        .cogroup(grouped3, aggregator3)
        .aggregate();

As you can see this creates 1 StateStore, requires 1 initializer, and 1 aggValueSerde. The user no longer has to worry about the intermediate values and the joiners. All they have to think about is how each stream impacts the creation of the final CG object.

When a new input arrives lets say at "topic1" it will first go through a KStreamAggreagte and grab the current aggregate from storeName1. It will add its incoming object to the aggregate, update the store and pass the new aggregate on. This new aggregate goes through the KStreamCogroup which is pretty much just a pass through processor and you are done. 

Topology wise for N incoming streams the new api will only every create N KStreamAggregates and 1 KStreamCogroup.

 

Concrete Example:

public class Customer {
    List<Item> cart;
    List<Item> purchases;
    List<Item> wishList;
}

There are 3 streams: cart, purchases, and wish-list.

We would construct 3 aggregators in which we add the item to the appropriate list. One of these would look like:

private static final Aggregator<String, Item, Customer> CART_AGGREGATOR = new Aggregator<String, Item, Customer>() {
    @Override
    public Patient apply(String key, Item value, Customer aggregate) {
        aggregate.cart.add(value);
        return aggregate;
    }
};

Then we would create the topology:

KGroupedStream<Long, Item> groupedCart = builder.stream("cart").groupByKey();
KGroupedStream<Long, Item> groupedPurchases = builder.stream("purchases").groupByKey();
KGroupedStream<Long, Item> groupedWishList = builder.stream("wish-list").groupByKey();
KTable<Long, Customer> customers = groupedCart.cogroup(Customer::new, CART_AGGREGATOR, aggValueSerde, "customerStore")
        .cogroup(groupedPurchases, PURCHASE_AGGREGATOR)
        .cogroup(groupedWishList, WISH_LIST_AGGREGATOR)
        .aggregate();
customers.to("customers");


Now imagine the streams get the following values:

Stream "cart":

    1L, Item[no:01]
    2L, Item[no:02]
    1L, Item[no:03]
    1L, Item[no:04]
    2L, Item[no:05]

Stream "purchases":

    2L, Item[no:06]
    1L, Item[no:07]
    1L, Item[no:08]
    2L, Item[no:09]
    2L, Item[no:10]

Stream "wish-list":

    1L, Item[no:11]
    2L, Item[no:12]
    2L, Item[no:13]
    2L, Item[no:14]
    2L, Item[no:15]

 

After all items have flown through the topology, you could expect to see the following outputs in "customers":

1L, Customer[

                      cart:{Item[no:01], Item[no:03], Item[no:04]},
                      purchases:{Item[no:07], Item[no:08]},
                      wishList:{Item[no:11]}
      ]
2L, Customer[
                      cart:{Item[no:02], Item[no:05]},
                      purchases:{Item[no:06], Item[no:09], Item[no:10]},
                      wishList:{Item[no:12], Item[no:13], Item[no:14], Item[no:15]}
      ]

It is important to note that intermediate values would also be produced, unless they are processed closely enough together that caching prevents this. (eg. After first item is processed from "cart" stream customer 1L would be output with only that first item in its cart and no items in the purchases or wishlist.)

Public Interfaces

KGroupedStream { //Copy of aggregate method signatures.
...
<T> CogroupedKStream<K, K, T> cogroup(final Initializer<T> initializer, final Aggregator<? super K, ? super V, T> aggregator, final Serde<T> aggValueSerde, final String storeName);
<T> CogroupedKStream<K, K, T> cogroup(final Initializer<T> initializer, final Aggregator<? super K, ? super V, T> aggregator, final StateStoreSupplier<KeyValueStore> storeSupplier);
<T> CogroupedKStream<K, Windowed<K>, T> cogroup(final Initializer<T> initializer, final Aggregator<? super K, ? super V, T> aggregator, final Merger<? super K, T> sessionMerger, final SessionWindows sessionWindows, final Serde<T> aggValueSerde, final String storeName);
<T> CogroupedKStream<K, Windowed<K>, T> cogroup(final Initializer<T> initializer, final Aggregator<? super K, ? super V, T> aggregator, final Merger<? super K, T> sessionMerger, final SessionWindows sessionWindows, final StateStoreSupplier<SessionStore> storeSupplier);
<W extends Window, T> CogroupedKStream<K, Windowed<K>, T> cogroup(final Initializer<T> initializer, final Aggregator<? super K, ? super V, T> aggregator, final Windows<W> windows, final Serde<T> aggValueSerde, final String storeName);
<W extends Window, T> CogroupedKStream<K, Windowed<K>, T> cogroup(final Initializer<T> initializer, final Aggregator<? super K, ? super V, T> aggregator, final Windows<W> windows, final StateStoreSupplier<WindowStore> storeSupplier);
}
/**
* {@code KCogroupedStream} is an abstraction of multiple <i>grouped</i> record streams of {@link KeyValue} pairs.
* It is an intermediate representation of one or more {@link KStream}s in order to apply one or more aggregation
* operations on the original {@link KStream} records.
* <p>
* It is an intermediate representation after a grouping of {@link KStream}s, before the aggregations are applied to
* the new partitions resulting in a {@link KTable}.
* <p>
* A {@code KCogroupedStream} must be obtained from a {@link KGroupedStream} via 
* {@link KGroupedStream#cogroup(Initializer, Aggregator, org.apache.kafka.common.serialization.Serde, String) cogroup(...)}.
*
* @param <K> Type of keys
* @param <RK> Type of key in the table, either K or Windowed&ltK&gt
* @param <V> Type of aggregate values
*/
public interface CogroupedKStream<K, RK, V> {
/**
* @return this CogroupedKStream so you can chain calls
*/
<T> CogroupedKStream<K, RK, V> cogroup(KGroupedStream<K, T> groupedStream, Aggregator<? super K, ? super T, V> aggregator);
KTable<RK, V> aggregate();
}

Expected use:

KTable<K, V> cogroupedTable = groupedStream1.cogroup(initializer, aggregator1, aggValueSerde, "aggValue").cogroup(groupedStream2, aggregator2).cogroup(groupedStream3, aggregator3) ... .cogroup(groupedStreamN, aggregatorN).aggregate();

 

Proposed Changes

  1. Construct the above Public Interfaces.
  2. Create an internal.KCogroupedStreamImpl that will keep track of the StateStoreSupplier, Initializer, Pairs of (KGroupedStream, Aggregator), and if needed Windows or SessionMerger and SessionWindows.
  3. Model the aggregate method of internal.KCogroupedStream after the doAggregate method of KGroupedStream by forcing the KGroupedStreams to repartitionIfRequired and adding the KStreamAggProcessorSupplier for each KGroupedStream. Additionally ensure all sources are copartitioned, processors have access to the state store, and add a KStreamCogroup processor.
  4. Create a KStreamCogroup that will passthrough all outputs from the KStreamAggregate. KStreamCogroup must also be a KStreamAggProcessorSupplier; it will keep track of all of its parent KStreamAggProcessorSuppliers in case it needs to enableSendingOldValues and it can have one of them create a KTableValueGetterSupplier if view is called.

Compatibility, Deprecation, and Migration Plan

  • Users must upgrade to new version if they want to use this functionality.

Rejected Alternatives

TBD

  • No labels