Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

...

As you can see this creates 1 StateStore, and in the Materialized parameter 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. The idea is that you can collect many grouped streams with overlapping key spaces and any kind of value types. Once aggregated its value will be reduced into one type. This is why the user need only one initializer. Each aggregator will need to integrate the new value with the new object made in the initializer.

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. 

...

Code Block
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(CART_AGGREGATOR, materialized)
        .cogroup(groupedPurchases, PURCHASE_AGGREGATOR)
        .cogroup(groupedWishList, WISH_LIST_AGGREGATOR)
        .aggregate(initializer), materialized);
customers.to("customers");

...

Public Interfaces

Code Block
KGroupedStream <K, V> {
...
	<T> CogroupedKStream<K, T> cogroup(final Aggregator<? super K, ? super V, T> aggregator);

	<T> CogroupedKStream<K, T> cogroup(final Aggregator<? super K, ? super V, T> aggregator, final Materialized<K, VR, KeyValueStore<Bytes, byte[]>> materialized);
}


Code Block
/**
* {@code CogroupedKStream} 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 CogroupedKStream} 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 <V> Type of aggregate values
*/
public interface CogroupedKStream<K, V>Vout> {
    <T><Vin> CogroupedKStream<K, V>Vout> cogroup(final KGroupedStream<K, T>Vin> groupedStream,
                                       final Aggregator<? super K, ? super TVin, V>Vout> aggregator);

    KTable<K, V>Vout> aggregate(final Initializer<V>Initializer<Vout> initializer,
                           final Materialized<K, VRVout, KeyValueStore<Bytes, byte[]>> materialized);
    
    KTable<K, V>Vout> aggregate(final Initializer<V>Initializer<Vout> initializer,);

    KTable<K, Vout> aggregate(final Initializer<Vout> initializer,
						   final Named named,
              final StateStoreSupplier<KeyValueStore> storeSupplier);

    KTable<Windowed<K>, V> aggregate(final Initializer<V> initializer,
   final Materialized<K, Vout, KeyValueStore<Bytes, byte[]>> materialized);
    
    KTable<K, Vout> aggregate(final Initializer<Vout> initializer,
						   final Named named);

    <W extends Window> TimeWindowedCogroupedKStream<K, Vout> windowedBy(final Windows<W> windows);

 final Merger<? super KSessionWindowedCogroupedKStream<K, V> sessionMerger,
                                     final SessionWindows sessionWindows,
                                     final Materialized<K, VR, KeyValueStore<Bytes, byte[]>> materialized);

    KTable<Windowed<K>, V> aggregate(final Initializer<V> initializer,
                                     final Merger<? super K, V> sessionMerger,
                                     final SessionWindows sessionWindows,
                                     final StateStoreSupplier<SessionStore> storeSupplier);

    <W extends Window> KTable<Windowed<K>, V> aggregate(final Initializer<V> initializer,
                       Vout> windowedBy(final SessionWindows sessionWindows);

}


Code Block
/**
 * {@code SessionWindowedCogroupKStream} is an abstraction of a <i>windowed</i> record stream of {@link org.apache.kafka.streams.KeyValue} pairs.
 * It is an intermediate representation of a {@link KGroupedStream} in order to apply a windowed aggregation operation on the original
 * {@link KGroupedStream} records.
 * <p>
 * It is an intermediate representation after a grouping, cogrouping and windowing of a {@link KStream} before an aggregation is applied to the
 * new (partitioned) windows resulting in a windowed {@link KTable}
 * (a <emph>windowed</emph> {@code KTable} is a {@link KTable} with key type {@link Windowed Windowed<K>}.
 * <p>
 * The specified {@code SessionWindows} define the gap between windows.
 * The result is written into a local windowed {@link org.apache.kafka.streams.state.KeyValueStore} (which is basically an ever-updating
 * materialized view) that can be queried using the name provided in the {@link Materialized} instance.
 *
 * New events are added to windows until their grace period ends (see {@link TimeWindows#grace(Duration)}).
 *
 * Furthermore, updates to the store are sent downstream into a windowed {@link KTable} changelog stream, where
 * "windowed" implies that the {@link KTable} key is a combined key of the original record key and a window ID.

 * A {@code WindowedKStream} must be obtained from a {@link KGroupedStream} via {@link KGroupedStream#windowedBy(Windows)} .
 *
 * @param <K> Type of keys
 * @param <V> Type of values
 * @see KStream
 * @see KGroupedStream
 * @see CogroupedKStream
 */
import org.apache.kafka.streams.state.SessionStore;

public interface SessionWindowedCogroupedKStream<K, V> {
    KTable<Windowed<K>, Vout> aggregate(final Initializer<V> initializer,
                                 final Windows<W> windows,
  final Merger<? super K,          V> sessionMerger,
                                         final Materialized<K, VRV, KeyValueStore<BytesSessionStore<Bytes, byte[]>> materialized);

    <W extends Window> KTable<Windowed<K>, V> aggregate(final Initializer<V> initializer,
                                     final Merger<? super K, V> sessionMerger);

    KTable<Windowed<K>, Vout> aggregate(final Initializer<V> initializer,
									 final Named named,
                final   Windows<W> windows,
                 final Merger<? super K, V> sessionMerger,
                                     final Materialized<K, StateStoreSupplier<WindowStore> storeSupplier);
}

Proposed Changes

  1. Construct the above Public Interfaces.
  2. Create an internal.KCogroupedStreamImpl that will keep track of the KeyValueSerde, AggValueSerde, Initializer, and Pairs of (KGroupedStream, Aggregator).
  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

V, SessionStore<Bytes, byte[]>> materialized);

    KTable<Windowed<K>, V> aggregate(final Initializer<V> initializer,
									 final Named named,
                                     final Merger<? super K, V> sessionMerger);

}


Code Block

/**
 * {@code TimeWindowedCogroupKStream} is an abstraction of a <i>windowed</i> record stream of {@link org.apache.kafka.streams.KeyValue} pairs.
 * It is an intermediate representation of a {@link KGroupedStream} in order to apply a windowed aggregation operation on the original
 * {@link KGroupedStream} records.
 * <p>
 * It is an intermediate representation after a grouping, cogrouping and windowing of a {@link KStream} before an aggregation is applied to the
 * new (partitioned) windows resulting in a windowed {@link KTable}
 * (a <emph>windowed</emph> {@code KTable} is a {@link KTable} with key type {@link Windowed Windowed<K>}.
 * <p>
 * The specified {@code windows} define either hopping time windows that can be overlapping or tumbling (c.f.
 * {@link TimeWindows}) or they define landmark windows (c.f. {@link UnlimitedWindows}).
 * The result is written into a local windowed {@link org.apache.kafka.streams.state.KeyValueStore} (which is basically an ever-updating
 * materialized view) that can be queried using the name provided in the {@link Materialized} instance.
 *
 * New events are added to windows until their grace period ends (see {@link TimeWindows#grace(Duration)}).
 *
 * Furthermore, updates to the store are sent downstream into a windowed {@link KTable} changelog stream, where
 * "windowed" implies that the {@link KTable} key is a combined key of the original record key and a window ID.

 * A {@code WindowedKStream} must be obtained from a {@link KGroupedStream} via {@link KGroupedStream#windowedBy(Windows)} .
 *
 * @param <K> Type of keys
 * @param <T> Type of values
 * @see KStream
 * @see KGroupedStream
 * @see CogroupedKStream
 */

public interface TimeWindowedCogroupedKStream<K, V> {

    KTable<Windowed<K>, V> aggregate(final Initializer<V> initializer,
                                     final Materialized<K, V, WindowStore<Bytes, byte[]>> materialized);

    KTable<Windowed<K>, V> aggregate(final Initializer<V> initializer);

    KTable<Windowed<K>, V> aggregate(final Initializer<V> initializer,
									 final Named named,
                                     final Materialized<K, V, WindowStore<Bytes, byte[]>> materialized);

    KTable<Windowed<K>, V> aggregate(final Initializer<V> initializer,
									 final Named named);

}


Proposed Changes

  1. Construct the above Public Interfaces.
  2. Create an internal.KCogroupedStreamImpl that will keep track of the KeyValueSerde, AggValueSerde, Initializer, and Pairs of (KGroupedStream, Aggregator).
  3. Create an internal TimeWindowedKCogroupedStreamImpl and SessionWindowedKCogroupedStreamImpl that will complete the  windowed aggregations.
  4. make the KStreamAggProcessorSupplier for each KGroupedStream. Additionally ensure all sources are copartitioned, processors have access to the state store.
  5. create a cogroupedStreamAggregateBuilder to take the groupStream and aggregate pairs apply the aggregations

Compatibility, Deprecation, and Migration Plan

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

Rejected Alternatives

  • Introducing Named in the cogroup

Rational for edits to the KIP

Due to changes to the project since KIP #150 was written there are a few items that may need to be updated.

First item that changed is the adoption of Materialized parameter.

The second item is the WindowedBy. How the KIP currently handles windowing is that it overloads the aggregate function to taking a Window object and an initializer. Currently the practice to window grouped streams is to call windowedBy and receive a windowed stream object. The same interface for a windowed stream made from a grouped stream will not work for Cogrouped streams as grouped streams SO we have to make new ones for CogroupedWindows. This is because when a cogroup is created and aggregated the aggregator is associated with each grouped stream then stored in the cogroup. When aggregating what is needed is an initializer but not an aggregator.TBD