...
Code Block | ||||
---|---|---|---|---|
| ||||
void print(final PrintOptions<K, V> printOptions); KStream<K, V> through(final String topic, final Topic<KTopicOptions<K, V> topicOptions); void to(final String topic, final Topic<VTopicOptions<V, V> topicOptions); KGroupedStream<K, V> groupByKey(final GroupByOptions<K, V> groupByOptions); <KR> KGroupedStream<KR, V> groupBy(final KeyValueMapper<? super K, ? super V, KR> selector, GroupByOptions<KR, V> groupByOptions); <VO, VR> KStream<K, VR> join(final KStream<K, VO> other, final ValueJoiner<? super V, ? super VO, ? extends VR> joiner, final JoinWindows windows, final JoinOptions<K, V, VO> options); <VT, VR> KStream<K, VR> join(final KTable<K, VT> other, final ValueJoiner<? super V, ? super VT, ? extends VR> joiner, final JoinOptions<K, V, VT> options); <VO, VR> KStream<K, VR> leftJoin(final KStream<K, VO> other, final ValueJoiner<? super V, ? super VO, ? extends VR> joiner, final JoinWindows windows, final JoinOptions<K, V, VO> options); <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> other, final ValueJoiner<? super V, ? super VT, ? extends VR> joiner, final JoinWindows windows, final JoinOptions<K, V, VT> options); <VO, VR> KStream<K, VR> outerJoin(final KStream<K, VO> other, final ValueJoiner<? super V, ? super VO, ? extends VR> joiner, final JoinWindows windows, final JoinOptions<K, V, VO> options); <VT, VR> KStream<K, VR> outerJoin(final KTable<K, VT> other, final ValueJoiner<? super V, ? super VT, ? extends VR> joiner, final JoinWindows windows, final JoinOptions<K, V, VT> options); |
Code Block | ||||
---|---|---|---|---|
| ||||
<KR, VR> KGroupedTable<KR, VR> groupBy(final KeyValueMapper<? super K, ? super V, KeyValue<KR, VR>> selector, GroupByOptions<KR, VR> groupByOptions); KTable<K, V> filter(final Predicate<? super K, ? super V> predicate, final Materialized Materialized<K, V, KeyValueStore<K, V>> materialized); KTable<K, V> filterNot(final Predicate<? super K, ? super V> predicate, final Materialized Materialized<K, V, KeyValueStore<K, V>> materialized); <VR> KTable<K, VR> mapValues(final ValueMapper<? super V, ? extends VR> mapper, final Materialized Materialized<K, V, KeyValueStore<K, V>> materialized); void to(final String topic, final Topic<VTopicOptions<V, V> options); KTable<K, V> through(final String topic, final Table<KMaterialized<K, V> options); <VO, VR> KTable<K, VR> join(final KTable<K, VO> other, final ValueJoiner<? super V, ? super VO, ? extends VR> joiner, final Materialized Materialized<K, V, KeyValueStore<K, VR>> materialized); <VO, VR> KTable<K, VR> leftJoin(final KTable<K, VO> other, final ValueJoiner<? super V, ? super VO, ? extends VR> joiner, final Materialized Materialized<K, V, KeyValueStore<K, VR>> materialized); <VO, VR> KTable<K, VR> outerJoin(final KTable<K, VO> other, final ValueJoiner<? super V, ? super VO, ? extends VR> joiner, final Materialized Materialized<K, V, KeyValueStore<K, VR>> materialized); |
Code Block |
---|
<W extends Window> WindowedKStream<K, V> windowedBy(Windows<W> timeWindows); SessionWindowedKStream<K, V> sessionWindowedBy(SessionWindows sessionWindows); KTable<K, Long> count(final Materialized materialized); KTable<K, V> reduce(final Reducer<V> reducer, final Materialized Materialized<K, V, KeyValueStore<K, V>> materialized); <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer, final Aggregator<? super K, ? super V, VR> aggregator, final Serde<VR> aggValueSerde, final Materialized Materialized<K, V, KeyValueStore<K, V>> materialized); |
Code Block | ||||
---|---|---|---|---|
| ||||
public interface WindowedKStream<K, V> { KTable<Windowed<K>, Long> count(); KTable<Windowed<K>, Long> count(final Materialized<K, Long, WindowStore<K, Long>> materializedAs); <VR> KTable<Windowed<K>, VR> aggregate(final Initializer<VR> initializer, final Aggregator<? super K, ? super V, VR> aggregator); <VR> KTable<Windowed<K>, VR> aggregate(final Initializer<VR> initializer, final Aggregator<? super K, ? super V, VR> aggregator, final Materialized<K, VR, WindowStore<K, VR>> materializedAs); KTable<Windowed<K>, V> reduce(final Reducer<V> reducer); KTable<Windowed<K>, V> reduce(final Reducer<V> reducer, final Materialized<K, V, WindowStore<K, V>> materializedAs); } |
...