...
JIRA:
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 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.
Another motivation of this KIP is to introduce on-demand commit()
feature.
Public Interfaces
The following methods will be added to related classes.
KStream.java
. 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 | ||
---|---|---|
| ||
KStream<K public interface RichInitializer<V, V> filter(RichPredicate<? super K, ? super V> predicateK> { V apply(K key); } KStream<K,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<? 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>, ? 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<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, ? superextends VOVR, ? extendssuper VR>K> joinervalueTransformerSupplier, final JoinWindows windows, final String... stateStoreNames); <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<? super K, ? super V, KR> final Serde<K> keySerdeselector); <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<? super K, ? super V, KR> selector, final Serde<V> thisValueSerde, final Serialized<KR, final Serde<VO> otherValueSerdeV> serialized); <VO, VR> KStream<K, VR> leftJoinjoin(final KStream<K, VO> otherStream, final RichValueJoiner<? super V, ? super VO, ? extends VR> joinerVR, ? super K> joiner, final JoinWindows windows); <VO, VR> KStream<K, VR> leftJoinjoin(final KStream<K, VO> otherStream, final RichValueJoiner<? super V, ? super VO, ? extends VR> joinerVR, ? super K> joiner, final JoinWindows windows, final Joined<K, V, VO> joined); <VO, VR> KStream<K, VR> leftJoin(final KStream<K, Serde<K>VO> keySerdeotherStream, final Serde<V> thisValSerdeRichValueJoiner<? super V, ? super VO, ? extends VR, ? super K> joiner, final Serde<VO>JoinWindows otherValueSerdewindows); <VO, VR> KStream<K, VR> outerJoinleftJoin(final KStream<K, VO> otherStream, final RichValueJoiner<? super V, ? super VO, ? extends VR> joinerVR, ? super K> joiner, final JoinWindows windows, final Joined<K, V, VO> joined); <VO, VR> KStream<K, VR> outerJoin(final KStream<K, VO> otherStream, final RichValueJoiner<? super V, ? super VO, ? extends VR> VR, ? super K> joiner, final JoinWindows windows); <VO, VR> KStream<K, VR> outerJoin(final KStream<K, VO> otherStream, final Serde<K> keySerde, final RichValueJoiner<? super V, ? super VO, ? extends VR, ? super K> joiner, final Serde<V>JoinWindows thisValueSerdewindows, final Serde<VO> otherValueSerdeJoined<K, V, VO> joined); <VT, VR> KStream<K, VR> join(final KTable<K, VT> table, final RichValueJoiner<? super K, ? super V, ? super VT, ? extends VR> joiner); <VT, VR> KStream<K, VR> join(final KTable<K, VT> table, final RichValueJoiner<? super K, ? super V, ? super VT, ? extends VR> joiner, final Serde<K>Joined<K, keySerdeV, VT> joined); <VT, VR> KStream<K, VR> final Serde<V> valSerde); <VT, VR> KStream<K, VR> leftJoin(final KTable<K, leftJoin(final KTable<K, VT> table, final RichValueJoiner<? super K, ? super V, ? super VT, ? extends VR> joiner); <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table, final RichValueJoiner<? super K, ? super V, ? super VT, ? extends VR> joiner, final Serde<K>Joined<K, keySerdeV, VT> joined); <GK, GV, RV> KStream<K, RV> join(final final Serde<V> valSerde); <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GlobalKTable<GK, GV> globalKTable, final RichKeyValueMapper<? super K, ? super V, ? extends GK> keyValueMapper, final RichValueJoiner<? super K, ? 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 K, ? super V, ? super GV, ? extends RV> valueJoiner); |
...
KTable
...
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); final Materialized<K, V, KeyValueStore<Bytes, byte[]>> materialized); 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, ? extendsfinal VR> mapperMaterialized<K, V, KeyValueStore<Bytes, byte[]>> materialized); <VR> KTable<K, VR> mapValues(final RichValueMapper<? super V, ? extends VR> mapperVR, final? Serde<VR>super valueSerde, final String queryableStoreNameK> mapper); <VR> KTable<K, VR> mapValues(final RichValueMapper<? super V, ? extends VR> VR, ? super K> mapper, final Serde<VR>Materialized<K, valueSerdeVR, KeyValueStore<Bytes, byte[]>> materialized); <KR> KStream<KR, V> toStream(final RichKeyValueMapper<? super K, ? super V, ? extends KR> mapper); <KR, final StateStoreSupplier<KeyValueStore> storeSupplier); void foreach(final RichForeachAction<? super K, ? super V> action); <KR> KStream<KR, V> toStreamVR> KGroupedTable<KR, VR> groupBy(final RichKeyValueMapper<? super K, ? super V, ?KeyValue<KR, extendsVR>> KR> mapperselector); <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> keySerdeSerialized<KR, final Serde<VR> valueSerdeVR> serialized); <VO, VR> KTable<K, VR> join(final KTable<K, VO> other, final RichValueJoiner<? super V, ? super VO, ? extends VR>VR, ? super K> joiner); <VO, VR> KTable<K, VR> join(final KTable<K, VO> other, final RichValueJoiner<? super V, ? super VO, ? extends VR> joinerVR, ? super K> joiner, final Serde<VR> joinSerde, final Materialized<K, VR, KeyValueStore<Bytes, final String queryableStoreNamebyte[]>> materialized); <VO, VR> KTable<K, VR> joinleftJoin(final KTable<K, VO> other, final RichValueJoiner<? super V, ? super VO, ? extends VR, ? super K> joiner); <VO, VR> joiner KTable<K, VR> leftJoin(final KTable<K, VO> other, final StateStoreSupplier<KeyValueStore> storeSupplier); <VO, VR> KTable<K, VR> leftJoin(final KTable<K, VO> other final ValueJoiner<? super K, ? super V, ? super VO, ? extends VR> joiner, final RichValueJoiner<? super V, ? super VO, ? extends VR> joinerMaterialized<K, VR, KeyValueStore<Bytes, byte[]>> materialized); <VO, VR> KTable<K, VR> leftJoinouterJoin(final KTable<K, VO> other, final RichValueJoiner<? super V, ? super VO, ? extends VR> joinerVR, ? super K> joiner); <VO, VR> KTable<K, VR> outerJoin(final KTable<K, VO> other, final Serde<VR> joinSerde, final RichValueJoiner<? super V, ? super VO, ? final String queryableStoreName); <VO, VR> KTable<K, VR> leftJoin(final KTable<K, VO> other, extends VR, ? super K> joiner, final RichValueJoiner<? super V, ? super VO, ? extends VR> joiner, 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, final Materialized<K, V, final StateStoreSupplier<KeyValueStore> storeSupplierKeyValueStore<Bytes, byte[]>> materialized); <VO, VR><VR> KTable<K, VR> outerJoinaggregate(final KTable<KRichInitializer<VR, VO>K> otherinitializer, final final RichValueJoiner<? RichAggregator<? super VK, ? super VOV, ? extends VR> joiner); <VO, VR> KTable<K, VR> outerJoin(final KTable<K, VO> other,aggregator, final Materialized<K, VR, final RichValueJoiner<? super V, ? super VO, ? extends VR> joinerKeyValueStore<Bytes, byte[]>> materialized); <VR> KTable<K, VR> aggregate(final RichInitializer<VR, 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 Serde<VR> joinSerde, final String queryableStoreName); <VO, VR> KTable<K, VR> outerJoin(final KTable<K, VO> other, final RichAggregator<? super K, ? super V, T> aggregator, final RichValueJoiner<RichMerger<? super VK, ? super VO, ? extends VR> joiner, T> sessionMerger); <VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR, Windowed<K>> initializer, final StateStoreSupplier<KeyValueStore> storeSupplier); |
KGroupedStream
Code Block | ||
---|---|---|
| ||
KTable<K, V> reduce(final RichReducer<V> reducer); KTable<K, V> reduce(final RichReducer<V> reducer, RichAggregator<? super K, ? super V, VR> aggregator, final String queryableStoreName); KTable<K, V> reduce(final RichReducer<V> reducer, final RichMerger<? super K, VR> sessionMerger, final StateStoreSupplier<KeyValueStore> storeSupplier); <W extends Window> KTable<Windowed<K>, V> reduce(final RichReducer<V> reducer, 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, final Windows<W> windowsK> reducer, final Materialized<K, V, SessionStore<Bytes, final String queryableStoreName); <W extends Window>byte[]>> materializedAs); , |
TimeWindowedKStream
Code Block | ||
---|---|---|
| ||
<VR> KTable<Windowed<K>, V>VR> reduceaggregate(final RichReducer<V> reducerRichInitializer<VR, K> initializer, final RichAggregator<? super K, ? super V, final Windows<W> windowsVR> aggregator); <W extends Window> <VR> KTable<Windowed<K>, V>VR> reduceaggregate(final RichInitializer<VR, RichReducer<V>K> reducerinitializer, final RichAggregator<? super K, ? super V, final Windows<W> windowsVR> aggregator, final Materialized<K, VR, WindowStore<Bytes, byte[]>> materialized); KTable<Windowed<K>, V> reduce(final RichReducer<V, K> final StateStoreSupplier<WindowStore> storeSupplierreducer); KTable<Windowed<K>, V> reduce(final RichReducer<V>RichReducer<V, K> reducer, final SessionWindowsMaterialized<K, sessionWindowsV, WindowStore<Bytes, byte[]>> materialized); |
KGroupedTable
Code Block | ||
---|---|---|
| ||
KTable<K, V> reduce(final RichReducer<V, K> adder, final String queryableStoreName); KTable<Windowed<K>, V> reduce(final RichReducer<V> reducer, final RichReducer<V, K> subtractor, final Materialized<K, V, KeyValueStore<Bytes, final SessionWindows sessionWindowsbyte[]>> materialized); KTable<Windowed<K>KTable<K, V> reduce(final RichReducer<V, RichReducer<V>K> reduceradder, final RichReducer<V, K> subtractor); <VR> KTable<K, VR> aggregate(final SessionWindowsRichInitializer<VR> sessionWindowsinitializer, final StateStoreSupplier<SessionStore> storeSupplier); <VR> KTable<K, VR> aggregate(final Initializer<VR> initializerRichAggregator<? super K, ? super V, VR> adder, final RichAggregator<? super K, ? super V, VR> aggregatorsubtractor, final Materialized<K, Serde<VR>VR, aggValueSerde, KeyValueStore<Bytes, byte[]>> materialized); <VR> KTable<K, VR> aggregate(final RichInitializer<VR> initializer, final String queryableStoreName); <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer, final RichAggregator<? super K, ? super V, VR> aggregatoradder, final Serde<VR> aggValueSerde, final String queryableStoreName); <VR> KTable<K, VR> aggregate(final Initializer<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 Initializer<VR> initializer, final RichAggregator<? 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); <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 Initializer<VR> initializer, final RichAggregator<? super K, ? super V, VR> aggregator, final Windows<W> windows, final StateStoreSupplier<WindowStore> storeSupplier); <T> KTable<Windowed<K>, T> aggregate(final Initializer<T> initializer, final RichAggregator<? super K, ? super V, T> aggregator, final Merger<? super K, T> sessionMerger, final SessionWindows sessionWindows, final Serde<T> aggValueSerde, final String queryableStoreName); <T> KTable<Windowed<K>, T> aggregate(final Initializer<T> initializer, final RichAggregator<? super K, ? super V, T> aggregator, final Merger<? super K, T> sessionMerger, final SessionWindows sessionWindows, final Serde<T> aggValueSerde); <T> KTable<Windowed<K>, T> aggregate(final Initializer<T> initializer, final RichAggregator<? super K, ? super V, T> aggregator, final Merger<? super K, T> sessionMerger, final SessionWindows sessionWindows, final Serde<T> aggValueSerde, final StateStoreSupplier<SessionStore> storeSupplier); |
KGroupedTable
Code Block | ||
---|---|---|
| ||
KTable<K, V> reduce(final RichReducer<V> adder,
final Reducer<V> subtractor,
final String queryableStoreName);
KTable<K, V> reduce(final Reducer<V> adder,
final RichReducer<V> subtractor,
final String queryableStoreName);
KTable<K, V> reduce(final RichReducer<V> adder,
final RichReducer<V> subtractor,
final String queryableStoreName);
KTable<K, V> reduce(final RichReducer<V> adder,
final Reducer<V> subtractor);
KTable<K, V> reduce(final Reducer<V> adder,
final RichReducer<V> subtractor);
KTable<K, V> reduce(final RichReducer<V> adder,
final RichReducer<V> subtractor);
KTable<K, V> reduce(final RichReducer<V> adder,
final Reducer<V> subtractor,
final StateStoreSupplier<KeyValueStore> storeSupplier);
KTable<K, V> reduce(final Reducer<V> adder,
final RichReducer<V> subtractor,
final StateStoreSupplier<KeyValueStore> storeSupplier);
KTable<K, V> reduce(final RichReducer<V> adder,
final RichReducer<V> subtractor,
final StateStoreSupplier<KeyValueStore> storeSupplier);
<VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
final RichAggregator<? super K, ? super V, VR> adder,
final RichAggregator<? super K, ? super V, VR> subtractor,
final String queryableStoreName);
<VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
final RichAggregator<? super K, ? super V, VR> adder,
final RichAggregator<? super K, ? super V, VR> subtractor);
<VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
final RichAggregator<? super K, ? super V, VR> adder,
final RichAggregator<? super K, ? super V, VR> subtractor,
final Serde<VR> aggValueSerde,
final String queryableStoreName);
<VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
final RichAggregator<? super K, ? super V, VR> adder,
final RichAggregator<? super K, ? super V, VR> subtractor,
final Serde<VR> aggValueSerde);
<VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
final RichAggregator<? super K, ? super V, VR> adder,
final RichAggregator<? super K, ? super V, VR> subtractor,
final StateStoreSupplier<KeyValueStore> storeSupplier);
|
Proposed changes
Make record context open to public
StreamTask.updateProcessorContext()
) :Code Block | ||
---|---|---|
| ||
private void updateProcessorContext(final StampedRecord record, final ProcessorNode currNode) {
processorContext.setRecordContext(new ProcessorRecordContext(record.timestamp, record.offset(), record.partition(), record.topic()));
processorContext.setCurrentNode(currNode);
}
|
Thus, the record context is not available in ProcessorContext. We make the following changes to make it "public"
Code Block | ||
---|---|---|
| ||
public interface ProcessorContext {
...
RecordContext recordContext();
}
public class ProcessorContextImpl extends AbstractProcessorContext implements RecordCollector.Supplier {
...
@Override
public RecordContext recordContext() {
return this.recordContext();
}
}
|
Add commit() to RecordContext
We already have RecordContext
class with desired set of methods. However, it does not have commit()
method. In this KIP we add commit()
method to RecordContext
interface.
Code Block | ||
---|---|---|
| ||
public interface RecordContext {
. . .
void commit ();
}
public class ProcessorRecordContext implements RecordContext {
. . .
@Override
void commit () {
throw new UnsupportedOperationException("commit() is not supported in this context");
}
}
|
Sample processor should look like this:
RichAggregator<? super K, ? super V, 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 is added in this KIP
@Override
public long offset( | ||
Code Block | ||
| ||
class KStreamKTableJoinProcessor<K1, K2, V1, V2, R> extends AbstractProcessor<K1, V1> { ... private RecordContext recordContext; ... @Override public void process(final K1 key, final V1 value) { recordContext = new RecordContext() { @Override public void commit() { context().commit(); } @Override public long offset() { return context().recordContext().offset(); } @Override public long timestamp() { return context().recordContext().timestamp(); } @Override public String topic() { return context().recordContext().topic(); } @Override public int partition() { return context().recordContext().partition(); } }; if (key != null && value != null) { final V2 value2 = valueGetter.get(keyMapper.apply(key, value)); return context().recordContext().offset(); } @Override if (leftJoin || value2 != null public long timestamp() { return context().forward(key, joiner.apply(value, value2, recordContext));recordContext().timestamp(); } @Override } public String } } } |
Rich Interfaces
Code Block | ||
---|---|---|
| ||
public interface RichValueMapper<V, VR> { topic() { 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 V value, final RecordContext recordContext); } public interface RichReducer<V> { V apply(final V value1, final V value2, 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); } return context().recordContext().topic(); } @Override public int partition() { return context().recordContext().partition(); } }; if (key != null && value != null) { final V2 value2 = valueGetter.get(keyMapper.apply(key, value)); if (leftJoin || value2 != null) { context().forward(key, joiner.apply(value, value2, recordContext)); } } } } |
Rejected Alternatives
Not yet.
...