...
Current state: "Under Discussion"
Discussion thread: TBD here
JIRA: TBD
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
Motivation
Rich functions are one of the essential parts of stream processing. There are several use-cases where users cannot express their business logic with current un-rich methods especially when init(Some params)
, close()
methods are needed.
Public Interfaces
KStream.java
Code Block | ||
---|---|---|
| ||
KStream<K, V> filter(RichPredicate<? super K, ? super V> predicate);
KStream<K, V> filterNot(RichPredicate<? super K, ? super V> predicate);
<KR> KStream<KR, V> selectKey(RichKeyValueMapper<? super K, ? super V, ? extends KR> mapper);
<KR, VR> KStream<KR, VR> map(RichKeyValueMapper<? super K, ? super V, ? extends KeyValue<? extends KR, ? extends VR>> mapper);
<VR> KStream<K, VR> mapValues(RichValueMapper<? super V, ? extends VR> mapper);
<KR, VR> KStream<KR, VR> flatMap(final RichKeyValueMapper<? super K, ? super V, ? extends Iterable<? extends KeyValue<? extends KR, ? extends VR>>> mapper);
<VR> KStream<K, VR> flatMapValues(final RichValueMapper<? super V, ? extends Iterable<? extends VR>> processor);
void foreach(final RichForeachAction<? super K, ? super V> action);
KStream<K, V> peek(final RichForeachAction<? super K, ? super V> action);
KStream<K, V>[] branch(final RichPredicate<? super K, ? super V>... predicates);
<KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<? super K, ? super V, KR> selector);
<KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<? super K, ? super V, KR> selector,
final Serde<KR> keySerde,
final Serde<V> valSerde);
<VO, VR> KStream<K, VR> join(final KStream<K, VO> otherStream,
final RichValueJoiner<? super V, ? super VO, ? extends VR> joiner,
final JoinWindows windows);
<VO, VR> KStream<K, VR> join(final KStream<K, VO> otherStream,
final RichValueJoiner<? super V, ? super VO, ? extends VR> joiner,
final JoinWindows windows,
final Serde<K> keySerde,
final Serde<V> thisValueSerde,
final Serde<VO> otherValueSerde);
<VO, VR> KStream<K, VR> leftJoin(final KStream<K, VO> otherStream,
final RichValueJoiner<? super V, ? super VO, ? extends VR> joiner,
final JoinWindows windows);
<VO, VR> KStream<K, VR> leftJoin(final KStream<K, VO> otherStream,
final RichValueJoiner<? super V, ? super VO, ? extends VR> joiner,
final JoinWindows windows,
final Serde<K> keySerde,
final Serde<V> thisValSerde,
final Serde<VO> otherValueSerde);
<VO, VR> KStream<K, VR> outerJoin(final KStream<K, VO> otherStream,
final RichValueJoiner<? super V, ? super VO, ? extends VR> joiner,
final JoinWindows windows);
<VO, VR> KStream<K, VR> outerJoin(final KStream<K, VO> otherStream,
final RichValueJoiner<? super V, ? super VO, ? extends VR> joiner,
final JoinWindows windows,
final Serde<K> keySerde,
final Serde<V> thisValueSerde,
final Serde<VO> otherValueSerde);
<VT, VR> KStream<K, VR> join(final KTable<K, VT> table,
final RichValueJoiner<? super V, ? super VT, ? extends VR> joiner);
<VT, VR> KStream<K, VR> join(final KTable<K, VT> table,
final RichValueJoiner<? super V, ? super VT, ? extends VR> joiner,
final Serde<K> keySerde,
final Serde<V> valSerde);
<VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
final RichValueJoiner<? super V, ? super VT, ? extends VR> joiner);
<VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
final RichValueJoiner<? super V, ? super VT, ? extends VR> joiner,
final Serde<K> keySerde,
final Serde<V> valSerde);
<GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV> globalKTable,
final RichKeyValueMapper<? super K, ? super V, ? extends GK> keyValueMapper,
final RichValueJoiner<? super V, ? super GV, ? extends RV> joiner);
<GK, GV, RV> KStream<K, RV> leftJoin(final GlobalKTable<GK, GV> globalKTable,
final RichKeyValueMapper<? super K, ? super V, ? extends GK> keyValueMapper,
final RichValueJoiner<? super V, ? super GV, ? extends RV> valueJoiner);
|
KTable.java
Code Block | ||
---|---|---|
| ||
KTable<K, V> filter(final RichPredicate<? super K, ? super V> predicate);
KTable<K, V> filter(final RichPredicate<? super K, ? super V> predicate, final String queryableStoreName);
KTable<K, V> filter(final RichPredicate<? super K, ? super V> predicate, final StateStoreSupplier<KeyValueStore> storeSupplier);
KTable<K, V> filterNot(final RichPredicate<? super K, ? super V> predicate);
KTable<K, V> filterNot(final RichPredicate<? super K, ? super V> predicate, final StateStoreSupplier<KeyValueStore> storeSupplier);
KTable<K, V> filterNot(final RichPredicate<? super K, ? super V> predicate, final String queryableStoreName);
<VR> KTable<K, VR> mapValues(final RichValueMapper<? super V, ? extends VR> mapper);
<VR> KTable<K, VR> mapValues(final RichValueMapper<? super V, ? extends VR> mapper, final Serde<VR> valueSerde, final String queryableStoreName);
<VR> KTable<K, VR> mapValues(final RichValueMapper<? super V, ? extends VR> mapper,
final Serde<VR> valueSerde,
final StateStoreSupplier<KeyValueStore> storeSupplier);
void foreach(final RichForeachAction<? super K, ? super V> action);
<KR> KStream<KR, V> toStream(final RichKeyValueMapper<? super K, ? super V, ? extends KR> mapper);
<KR, VR> KGroupedTable<KR, VR> groupBy(final RichKeyValueMapper<? super K, ? super V, KeyValue<KR, VR>> selector);
<KR, VR> KGroupedTable<KR, VR> groupBy(final RichKeyValueMapper<? super K, ? super V, KeyValue<KR, VR>> selector,
final Serde<KR> keySerde,
final Serde<VR> valueSerde);
<VO, VR> KTable<K, VR> join(final KTable<K, VO> other,
final RichValueJoiner<? super V, ? super VO, ? extends VR> joiner);
<VO, VR> KTable<K, VR> join(final KTable<K, VO> other,
final RichValueJoiner<? super V, ? super VO, ? extends VR> joiner,
final Serde<VR> joinSerde,
final String queryableStoreName);
<VO, VR> KTable<K, VR> join(final KTable<K, VO> other,
final RichValueJoiner<? super V, ? super VO, ? extends VR> joiner,
final StateStoreSupplier<KeyValueStore> storeSupplier);
<VO, VR> KTable<K, VR> leftJoin(final KTable<K, VO> other,
final RichValueJoiner<? super V, ? super VO, ? extends VR> joiner);
<VO, VR> KTable<K, VR> leftJoin(final KTable<K, VO> other,
final RichValueJoiner<? super V, ? super VO, ? extends VR> joiner,
final Serde<VR> joinSerde,
final String queryableStoreName);
<VO, VR> KTable<K, VR> leftJoin(final KTable<K, VO> other,
final RichValueJoiner<? super V, ? super VO, ? extends VR> joiner,
final StateStoreSupplier<KeyValueStore> storeSupplier);
<VO, VR> KTable<K, VR> outerJoin(final KTable<K, VO> other,
final RichValueJoiner<? super V, ? super VO, ? extends VR> joiner);
<VO, VR> KTable<K, VR> outerJoin(final KTable<K, VO> other,
final RichValueJoiner<? super V, ? super VO, ? extends VR> joiner,
final Serde<VR> joinSerde,
final String queryableStoreName);
<VO, VR> KTable<K, VR> outerJoin(final KTable<K, VO> other,
final RichValueJoiner<? super V, ? super VO, ? extends VR> joiner,
final StateStoreSupplier<KeyValueStore> storeSupplier);
|
KGroupedStream
Jira | ||||||
---|---|---|---|---|---|---|
|
Jira | ||||||||
---|---|---|---|---|---|---|---|---|
|
Jira | ||||||||
---|---|---|---|---|---|---|---|---|
|
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
Motivation
This KIP combines KIP-149 and provides a hybrid solution to rich functions in Streams and accessing read-only keys within ValueJoiner, ValueTransformer, ValueMapper interfaces.
Rich functions are one of the essential parts of stream processing. There are several use-cases where users cannot express their business logic with current un-rich methods. For example:
- having access to RecordContext within an operator
- having access to a read-only key for ValueJoiner, ValueTransformer, ValueMapper interfaces
Rich Interfaces
Code Block | ||
---|---|---|
| ||
public interface RichInitializer<V, K> {
V apply(K key);
}
public interface RichValueMapper<V, VR, K> {
VR apply(final V value, final K key, final RecordContext recordContext);
}
public interface RichValueJoiner<V1, V2, VR, K> {
VR apply(final V1 value1, final V2 value2, final K key, final RecordContext recordContext);
}
public interface RichKeyValueMapper<K, V, VR> {
VR apply(final K key, final V value, final RecordContext recordContext);
}
public interface RichReducer<V, K> {
V apply(final V value1, final V value2, final K key, final RecordContext recordContext);
}
public interface RichAggregator<K, V, VA> {
VA apply(final K key, final V value, final VA aggregate, final RecordContext recordContext);
}
public interface RichForeachAction<K, V> {
void apply(final K key, final V value, final RecordContext recordContext);
}
public interface RichPredicate<K, V> {
boolean test(final K key, final V value, final RecordContext recordContext);
}
public interface RichMerger<K, V> {
V apply(final K aggKey, final V aggOne, final V aggTwo, final RecordContext recordContext);
}
public interface RichValueTransformer<V, VR, K> {
void init(final ProcessorContext context);
VR transform(final V value, final K key);
void close();
}
public interface RichValueTransformerSupplier<V, VR, K> {
RichValueTransformer<V, VR, K> get();
}
|
Public Interfaces
KStream
Code Block | ||
---|---|---|
| ||
KStream<K, V> filter(RichPredicate<? super K, ? super V> predicate);
KStream<K, V> filterNot(RichPredicate<? super K, ? super V> predicate);
<KR> KStream<KR, V> selectKey(RichKeyValueMapper<? super K, ? super V, ? extends KR> mapper);
<KR, VR> KStream<KR, VR> map(RichKeyValueMapper<? super K, ? super V, ? extends KeyValue<? extends KR, ? extends VR>> mapper);
<VR> KStream<K, VR> mapValues(RichValueMapper<? super V, ? extends VR, ? super K> mapper);
<KR, VR> KStream<KR, VR> flatMap(final RichKeyValueMapper< | ||
Code Block | ||
| ||
KTable<K, V> reduce(final RichReducer<V> reducer); KTable<K, V> reduce(final RichReducer<V> reducer, final String queryableStoreName); KTable<K, V> reduce(final RichReducer<V> reducer, final StateStoreSupplier<KeyValueStore> storeSupplier); <W extends Window> KTable<Windowed<K>, V> reduce(final RichReducer<V> reducer, final Windows<W> windows, final String queryableStoreName); <W extends Window> KTable<Windowed<K>, V> reduce(final RichReducer<V> reducer, final Windows<W> windows); <W extends Window> KTable<Windowed<K>, V> reduce(final RichReducer<V> reducer, final Windows<W> windows, final StateStoreSupplier<WindowStore> storeSupplier); KTable<Windowed<K>, V> reduce(final RichReducer<V> reducer, final SessionWindows sessionWindows, final String queryableStoreName); KTable<Windowed<K>, V> reduce(final RichReducer<V> reducer, final SessionWindows sessionWindows); KTable<Windowed<K>, V> reduce(final RichReducer<V> reducer, final SessionWindows sessionWindows, final StateStoreSupplier<SessionStore> storeSupplier); <VR> KTable<K, VR> aggregate(final RichInitializer<VR> initializer, final Aggregator<? super K, ? super V, VR> aggregator, final Serde<VR> aggValueSerde, final String queryableStoreName); <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer, final RichAggregator<? super K, ? super V, VR> aggregator, final Serde<VR> aggValueSerde, final String queryableStoreName); <VR> KTable<K, VR> aggregate(final RichInitializer<VR> initializer, final RichAggregator<? super K, ? super V, VR> aggregator, final Serde<VR> aggValueSerde, final String queryableStoreName); <VR> KTable<K, VR> aggregate(final RichInitializer<VR> initializer, final Aggregator<? super K, ? super V, VR> aggregator, final Serde<VR> aggValueSerde); <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer, final RichAggregator<? super K, ? super V, VR> aggregator, final Serde<VR> aggValueSerde); <VR> KTable<K, VR> aggregate(final RichInitializer<VR> initializer, final RichAggregator<? super K, ? super V, VR> aggregator, final Serde<VR> aggValueSerde); <VR> KTable<K, VR> aggregate(final RichInitializer<VR> initializer, final Aggregator<? super K, ? super V, VR> aggregator, final StateStoreSupplier<KeyValueStore> storeSupplier); <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer, final RichAggregator<? super K, ? super V, VR> aggregator, final StateStoreSupplier<KeyValueStore> storeSupplier); <VR> KTable<K, VR> aggregate(final RichInitializer<VR> initializer, final RichAggregator<? super K, ? super V, VR> aggregator, final StateStoreSupplier<KeyValueStore> storeSupplier); <W extends Window, VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR> initializer, final Aggregator<? super K, ? super V, VR> aggregator, final Windows<W> windows, final Serde<VR> aggValueSerde, final String queryableStoreName); <W extends Window, VR> KTable<Windowed<K>, VR> aggregate(final Initializer<VR> initializer, final RichAggregator<? super K, ? super V, VR> aggregator, final Windows<W> windows, final Serde<VR> aggValueSerde, final String queryableStoreName); <W extends Window, VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR> initializer, final RichAggregator<? super K, ? super V, VR> aggregator, final Windows<W> windows, final Serde<VR> aggValueSerde, final String queryableStoreName); <W extends Window, VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR> initializer, final Aggregator<? super K, ? super V, VR>? aggregator, extends Iterable<? extends KeyValue<? extends KR, ? extends VR>>> mapper); <VR> KStream<K, VR> final Windows<W> windows, flatMapValues(final RichValueMapper<? super V, ? extends Iterable<? extends VR>, ? super K> mapper); void foreach(final RichForeachAction<? super K, ? super V> action); KStream<K, V> peek(final RichForeachAction<? super K, ? super V> action); KStream<K, V>[] branch(final RichPredicate<? super K, ? super V>... predicates); <VR> KStream<K, VR> transformValues(final RichValueTransformerSupplier<? super V, ? extends VR, ? super K> valueTransformerSupplier, final StateStoreSupplier<WindowStore> storeSupplierString... stateStoreNames); <W extends Window<KR> KGroupedStream<KR, VR> KTable<Windowed<K>, VR> aggregateV> groupBy(final RichKeyValueMapper<? Initializer<VR>super initializerK, ? super V, KR> selector); <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<? super K, ? super V, KR> selector, final Serialized<KR, V> final RichAggregator<? super K, ? super V, VR> aggregatorserialized); <VO, VR> KStream<K, VR> join(final KStream<K, VO> otherStream, final RichValueJoiner<? super V, ? super VO, ? extends VR, ? super K> joiner, final Windows<W> windows, final JoinWindows windows); <VO, VR> KStream<K, VR> join(final KStream<K, VO> otherStream, final RichValueJoiner<? super V, ? super VO, final StateStoreSupplier<WindowStore> storeSupplier); <W ? extends WindowVR, VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR> initializer, ? super K> joiner, final JoinWindows windows, final RichAggregator<? super K, ? super V, VR> aggregator Joined<K, V, VO> joined); <VO, VR> KStream<K, VR> leftJoin(final KStream<K, VO> otherStream, final RichValueJoiner<? super V, ? super VO, ? extends VR, ? super K> joiner, final Windows<W> windows, final JoinWindows windows); <VO, VR> KStream<K, VR> leftJoin(final KStream<K, VO> otherStream, final RichValueJoiner<? super V, ? super VO, ? extends VR, final? StateStoreSupplier<WindowStore> storeSupplier); <T> KTable<Windowed<K>, T> aggregate(final RichInitializer<T> initializer, super K> joiner, final JoinWindows windows, final Aggregator<? super K, ? super V, T> aggregator, final Joined<K, V, VO> joined); <VO, VR> KStream<K, VR> outerJoin(final KStream<K, VO> otherStream, final Merger<? super K, T> sessionMerger, final RichValueJoiner<? super V, ? super VO, ? extends VR, ? super K> joiner, final SessionWindows sessionWindows, final JoinWindows windows); <VO, VR> KStream<K, VR> outerJoin(final KStream<K, Serde<T>VO> aggValueSerdeotherStream, final RichValueJoiner<? super V, ? super finalVO, String queryableStoreName); <T> KTable<Windowed<K>, T> aggregate(final Initializer<T> initializer, ? extends VR, ? super K> joiner, final JoinWindows windows, final RichAggregator<? super K, ? super V, T> aggregator, final Joined<K, V, VO> joined); <VT, VR> KStream<K, VR> join(final KTable<K, VT> table, final Merger<? super K, T> sessionMerger, final RichValueJoiner<? super K, ? super V, ? super VT, ? extends VR> joiner); <VT, VR> KStream<K, VR> join(final SessionWindows sessionWindowsKTable<K, VT> table, final RichValueJoiner<? super K, ? super V, ? finalsuper Serde<T> aggValueSerdeVT, ? extends VR> joiner, final Joined<K, V, VT> final String queryableStoreNamejoined); <T> KTable<Windowed<K><VT, VR> KStream<K, T>VR> aggregateleftJoin(final KTable<K, RichInitializer<T>VT> initializertable, final RichAggregator<RichValueJoiner<? super K, ? super V, ? T>super aggregatorVT, final Merger<? super K, T> sessionMerger, ? extends VR> joiner); <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table, final RichValueJoiner<? super K, ? super V, ? super VT, ? finalextends SessionWindowsVR> sessionWindowsjoiner, final Joined<K, V, VT> final Serde<T> aggValueSerde, joined); <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV> globalKTable, final String queryableStoreName); <T> KTable<Windowed<K>, T> aggregate(final RichInitializer<T> initializer, RichKeyValueMapper<? super K, ? super V, ? extends GK> keyValueMapper, final Aggregator<RichValueJoiner<? super K, ? super V, ? T>super aggregatorGV, ? extends RV> joiner); <GK, GV, RV> KStream<K, RV> leftJoin(final GlobalKTable<GK, GV> globalKTable, final Merger<? super K, T> sessionMerger, final RichKeyValueMapper<? super K, ? super V, ? extends GK> keyValueMapper, final SessionWindows sessionWindows, final RichValueJoiner<? super K, ? super V, ? super GV, ? extends RV> valueJoiner); |
KTable
Code Block | ||
---|---|---|
| ||
KTable<K, V> filter(final RichPredicate<? super K, ? finalsuper Serde<T>V> aggValueSerdepredicate); <T> KTable<Windowed<K>KTable<K, T>V> aggregatefilter(final Initializer<T> initializer RichPredicate<? super K, ? super V> predicate, final Materialized<K, V, KeyValueStore<Bytes, byte[]>> materialized); KTable<K, V> filterNot(final RichPredicate<? super K, ? super V> predicate); KTable<K, V> filterNot(final RichAggregator<RichPredicate<? super K, ? super V, T> aggregatorV> predicate, final Materialized<K, V, KeyValueStore<Bytes, byte[]>> materialized); <VR> KTable<K, VR> mapValues(final RichValueMapper<? super V, ? extends finalVR, Merger<? super K, T> sessionMerger, K> mapper); <VR> KTable<K, VR> mapValues(final RichValueMapper<? super V, ? extends VR, ? super K> mapper, final SessionWindowsMaterialized<K, sessionWindowsVR, KeyValueStore<Bytes, byte[]>> materialized); <KR> KStream<KR, V> toStream(final RichKeyValueMapper<? super K, ? super V, ? extends KR> mapper); <KR, VR> KGroupedTable<KR, VR> groupBy(final RichKeyValueMapper<? super K, ? super V, KeyValue<KR, VR>> selector); <KR, VR> KGroupedTable<KR, VR> groupBy(final RichKeyValueMapper<? super K, final? Serde<T> aggValueSerde); <T> KTable<Windowed<K>, T> aggregate(final RichInitializer<T> initializer, super V, KeyValue<KR, VR>> selector, final RichAggregator<? super K, ? super V, T> aggregator Serialized<KR, VR> serialized); <VO, VR> KTable<K, VR> join(final KTable<K, VO> other, final RichValueJoiner<? super V, ? super VO, ? extends finalVR, Merger<? super K K> joiner); <VO, T>VR> sessionMergerKTable<K, VR> join(final KTable<K, VO> other, final RichValueJoiner<? super V, final? SessionWindowssuper sessionWindowsVO, ? extends VR, ? super K> joiner, final Materialized<K, VR, final Serde<T> aggValueSerdeKeyValueStore<Bytes, byte[]>> materialized); <T> KTable<Windowed<K><VO, VR> KTable<K, T>VR> aggregateleftJoin(final KTable<K, Initializer<T>VO> initializerother, final RichAggregator<RichValueJoiner<? super KV, ? super VVO, ? T>extends aggregatorVR, ? super K> joiner); <VO, VR> KTable<K, VR> leftJoin(final KTable<K, VO> other, final Merger<? super K, T> sessionMerger, final ValueJoiner<? super K, ? super V, ? super VO, ? extends VR> joiner, final SessionWindows sessionWindows, final Materialized<K, VR, KeyValueStore<Bytes, byte[]>> materialized); <VO, VR> KTable<K, VR> outerJoin(final KTable<K, VO> other, final Serde<T> aggValueSerde, final RichValueJoiner<? super V, ? super VO, ? extends VR, ? finalsuper StateStoreSupplier<SessionStore>K> storeSupplierjoiner); <T> KTable<Windowed<K><VO, VR> KTable<K, T>VR> aggregateouterJoin(final KTable<K, RichInitializer<T>VO> initializerother, final Aggregator<RichValueJoiner<? super KV, ? super VVO, ? T>extends aggregatorVR, ? super K> joiner, final Materialized<K, VR, KeyValueStore<Bytes, byte[]>> materialized); |
KGroupedStream
Code Block | ||
---|---|---|
| ||
KTable<K, V> reduce(final RichReducer<V, K> reducer); KTable<K, V> reduce(final RichReducer<V, K> reducer, Merger<? super K, T> sessionMerger, final Materialized<K, V, KeyValueStore<Bytes, byte[]>> materialized); <VR> KTable<K, VR> aggregate(final RichInitializer<VR, finalK> SessionWindows sessionWindowsinitializer, final RichAggregator<? super K, ? super V, VR> final Serde<T> aggValueSerdeaggregator, final Materialized<K, VR, KeyValueStore<Bytes, final StateStoreSupplier<SessionStore> storeSupplierbyte[]>> materialized); <T><VR> KTable<Windowed<K>KTable<K, T>VR> aggregate(final RichInitializer<VR, RichInitializer<T>K> initializer, final RichAggregator<? super K, ? super V, VR> aggregator); |
SessionWindowedKStream
There are 3 rich interfaces in aggregate() methods. So converting all possible combinations to their rich counterparts can cause a lot of overloads. So, I propose to overload one method with all rich interfaces.
Code Block | ||
---|---|---|
| ||
<T> KTable<Windowed<K>, T> aggregate(final RichInitializer<T, Windowed<K>> initializer final RichAggregator<? super K, ? super V, T> aggregator, final Merger<RichAggregator<? super K, ? super V, T> sessionMergeraggregator, final RichMerger<? SessionWindowssuper sessionWindows, K, T> sessionMerger); <VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR, Windowed<K>> initializer, final Serde<T> aggValueSerde, final RichAggregator<? super K, ? super V, VR> aggregator, final StateStoreSupplier<SessionStore> storeSupplier); |
Limiting the ProcessorContext - RecordContext interface
We create a subset of features from ProcessorContext
and put into RecordContext
interface
Code Block | ||
---|---|---|
| ||
public interface RecordContext {
String applicationId();
TaskId taskId();
StreamsMetrics metrics();
String topic();
int partition();
void commit();
long offset();
long timestamp();
Map<String, Object> appConfigs();
Map<String, Object> appConfigsWithPrefix(String prefix);
}
public interface ProcessorContext extends RecordContext {
// all methods but the ones in RecordContext
} |
Once we need a conversion from ProcessorContext
and RecordContext, we just cast:
final RichMerger<? super K, VR> sessionMerger,
final Materialized<K, VR, SessionStore<Bytes, byte[]>> materialized);
KTable<Windowed<K>, V> reduce(final RichReducer<V, K> reducer);
KTable<Windowed<K>, V> reduce(final RichReducer<V, K> reducer,
final Materialized<K, V, SessionStore<Bytes, byte[]>> materializedAs);
, |
TimeWindowedKStream
Code Block | ||
---|---|---|
| ||
<VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR, K> initializer,
final RichAggregator<? super K, ? super V, VR> aggregator);
<VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR, K> initializer,
| ||
Code Block | ||
| ||
private class KStreamMapProcessor extends AbstractProcessor<K, V> { @Override public void init(ProcessorContext processorContext) { super.init(processorContext); richMapper.init((RecordContext) processorContext); // HERE WE MAKE A CAST } @Override public void process(final K key, final V value) { V1 newValue = mapper.apply(key, value); context().forward(key, newValue); } @Override public void close() { super.close(); mapper.close(); } } |
Rich Interfaces
Code Block | ||
---|---|---|
| ||
public interface RichValueMapper<V final RichAggregator<? super K, ? super V, VR> {aggregator, VR apply(final V value, final RecordContext recordContext); } public interface RichValueJoiner<V1, V2, VR> { VR apply(final V1 value1, final V2 value2, final RecordContext recordContext); } public interface RichKeyValueMapper<K, V, VR> { VR apply(final K key, final VMaterialized<K, valueVR, final RecordContext recordContextWindowStore<Bytes, byte[]>> materialized); } publicKTable<Windowed<K>, interfaceV> RichReducer<V> { V apply(final V value1, final V value2, final RecordContext recordContext); } public interface RichInitializer<VA> { VA apply(final RecordContext recordContext); } public interface Aggregator<K, V, VA> { VA apply(final K key, final V value, final VA aggregate, final RecordContext recordContext); } public interface RichForeachAction<K, V> { void apply(final K key, final V value, final RecordContext recordContext); } public interface RichPredicate<K, V> { boolean test(final K key, final V value, final RecordContext recordContext); } |
The same semantics apply to other interfaces as well.
So we don't need to add any overloaded methods for public APIs. Internally we perform 2 changes:
- Change the constructor type of all related Processors to accept rich interfaces
- Create converters from non-rich to rich interfaces
reduce(final RichReducer<V, K> reducer);
KTable<Windowed<K>, V> reduce(final RichReducer<V, K> reducer,
final Materialized<K, V, WindowStore<Bytes, byte[]>> materialized);
|
KGroupedTable
Code Block | ||
---|---|---|
| ||
KTable<K, V> reduce(final RichReducer<V, K> adder,
final RichReducer<V, K> subtractor,
final Materialized<K, V, KeyValueStore<Bytes, byte[]>> materialized);
KTable<K, V> reduce(final RichReducer<V, K> adder,
final RichReducer<V, K> subtractor);
<VR> KTable<K, VR> aggregate(final RichInitializer<VR> initializer,
| ||
Code Block | ||
| ||
class KStreamMapValues<K, V, V1> implements ProcessorSupplier<K, V> { private final RichValueMapper<K, V, V1> mapper; public KStreamMapValues(RichValueMapper<K, V, V1> mapper) { this.mapper = mapper; } @Override public Processor<K, V> get() { final RichAggregator<? super K, ? super returnV, new KStreamMapProcessor();VR> adder, } private class KStreamMapProcessor extends AbstractProcessor<K, V> { @Override final RichAggregator<? publicsuper void init(ProcessorContext processorContext) { K, ? super V, VR> subtractor, super.init(processorContext); mapper.init((RecordContext) processorContext); final } @OverrideMaterialized<K, VR, KeyValueStore<Bytes, byte[]>> materialized); <VR> KTable<K, VR> aggregate(final RichInitializer<VR> initializer, public void process(final K key, final V value) { V1final newValueRichAggregator<? = mapper.apply(key, value); super K, ? super V, VR> adder, context().forward(key, newValue); } @Override final RichAggregator<? super K, ? super publicV, void close() { super.close(); mapper.close(); } } } static <K, T1, T2, R> RichValueJoiner<K, T1, T2, R> convertToRichValueJoiner(final ValueJoinerWithKey<K, T1, T2, R> valueJoinerWithKey) { Objects.requireNonNull(valueJoinerWithKey, "valueJoiner can't be null"); if (valueJoinerWithKey instanceof RichValueJoiner) { return (RichValueJoiner<K, T1, T2, R>) valueJoinerWithKey; } else { return new RichValueJoiner<K, T1, T2, R>() { VR> subtractor); |
Proposed changes
Move
RecordContext
from.
processor.internals
to.processor
Make record context open to public
StreamTask.updateProcessorContext()
) :Code Block | ||
---|---|---|
| ||
// the below code snippet already exists, this is just for background.
private void updateProcessorContext(final StampedRecord record, final ProcessorNode currNode) {
processorContext.setRecordContext(new ProcessorRecordContext(record.timestamp, record.offset(), record.partition(), record.topic()));
processorContext.setCurrentNode(currNode);
} |
Sample processor should look like this:
Code Block | ||
---|---|---|
| ||
class KStreamKTableJoinProcessor<K1, K2, V1, V2, R> extends AbstractProcessor<K1, V1> { ... private RecordContext recordContext; // this line is added in this KIP ... @Override public void process(final K1 key, final V1 value) { recordContext = new RecordContext() { // recordContext initialization @Override is added in this KIP @Override public voidlong initoffset() {} return context().recordContext().offset(); @Override } @Override public voidlong closetimestamp() {} @Override return context().recordContext().timestamp(); } @Override public RString apply(K key, T1 value1, T2 value2topic() { return valueJoinerWithKeycontext().apply(key, value1, value2); recordContext().topic(); } @Override }; } } static <K, T1, T2, R> ValueJoinerWithKey<K, T1, T2, R> convertToValueJoinerWithKey(final ValueJoiner<T1, T2, R> valueJoiner) { public int partition() { return context().recordContext().partition(); } Objects.requireNonNull(valueJoiner, "valueJoiner can't be null"); return new ValueJoinerWithKey<K, T1, T2, R>( }; if (key != null && value != null) { final @Override V2 value2 = valueGetter.get(keyMapper.apply(key, value)); public R apply(K key, T1if value1, T2(leftJoin || value2 != null) { return valueJoinercontext().forward(key, joiner.apply(value1value, value2, recordContext)); } }; } } |
Rejected Alternatives
Not yet.