-
Notifications
You must be signed in to change notification settings - Fork 15.1k
KAFKA-6958: Overload methods for group and windowed stream to allow to name operation name using the new Named class #6413
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Large diffs are not rendered by default.
Large diffs are not rendered by default.
Large diffs are not rendered by default.
Large diffs are not rendered by default.
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -23,6 +23,7 @@ | |
| import org.apache.kafka.streams.kstream.KGroupedStream; | ||
| import org.apache.kafka.streams.kstream.KTable; | ||
| import org.apache.kafka.streams.kstream.Materialized; | ||
| import org.apache.kafka.streams.kstream.Named; | ||
| import org.apache.kafka.streams.kstream.Reducer; | ||
| import org.apache.kafka.streams.kstream.SessionWindowedKStream; | ||
| import org.apache.kafka.streams.kstream.SessionWindows; | ||
|
|
@@ -67,8 +68,16 @@ public KTable<K, V> reduce(final Reducer<V> reducer) { | |
| @Override | ||
| public KTable<K, V> reduce(final Reducer<V> reducer, | ||
| final Materialized<K, V, KeyValueStore<Bytes, byte[]>> materialized) { | ||
| return reduce(reducer, NamedInternal.empty(), materialized); | ||
| } | ||
|
|
||
| @Override | ||
| public KTable<K, V> reduce(final Reducer<V> reducer, | ||
| final Named named, | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. did you want to add a null check for this as well (like you did for |
||
| final Materialized<K, V, KeyValueStore<Bytes, byte[]>> materialized) { | ||
| Objects.requireNonNull(reducer, "reducer can't be null"); | ||
| Objects.requireNonNull(materialized, "materialized can't be null"); | ||
| Objects.requireNonNull(named, "name can't be null"); | ||
|
|
||
| final MaterializedInternal<K, V, KeyValueStore<Bytes, byte[]>> materializedInternal = | ||
| new MaterializedInternal<>(materialized, builder, REDUCE_NAME); | ||
|
|
@@ -80,9 +89,10 @@ public KTable<K, V> reduce(final Reducer<V> reducer, | |
| materializedInternal.withValueSerde(valSerde); | ||
| } | ||
|
|
||
| final String name = new NamedInternal(named).orElseGenerateWithPrefix(builder, REDUCE_NAME); | ||
| return doAggregate( | ||
| new KStreamReduce<>(materializedInternal.storeName(), reducer), | ||
| REDUCE_NAME, | ||
| name, | ||
| materializedInternal | ||
| ); | ||
| } | ||
|
|
@@ -91,9 +101,18 @@ public KTable<K, V> reduce(final Reducer<V> reducer, | |
| public <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer, | ||
| final Aggregator<? super K, ? super V, VR> aggregator, | ||
| final Materialized<K, VR, KeyValueStore<Bytes, byte[]>> materialized) { | ||
| return aggregate(initializer, aggregator, NamedInternal.empty(), materialized); | ||
| } | ||
|
|
||
| @Override | ||
| public <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer, | ||
| final Aggregator<? super K, ? super V, VR> aggregator, | ||
| final Named named, | ||
| final Materialized<K, VR, KeyValueStore<Bytes, byte[]>> materialized) { | ||
| Objects.requireNonNull(initializer, "initializer can't be null"); | ||
| Objects.requireNonNull(aggregator, "aggregator can't be null"); | ||
| Objects.requireNonNull(materialized, "materialized can't be null"); | ||
| Objects.requireNonNull(named, "named can't be null"); | ||
|
|
||
| final MaterializedInternal<K, VR, KeyValueStore<Bytes, byte[]>> materializedInternal = | ||
| new MaterializedInternal<>(materialized, builder, AGGREGATE_NAME); | ||
|
|
@@ -102,9 +121,10 @@ public <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer, | |
| materializedInternal.withKeySerde(keySerde); | ||
| } | ||
|
|
||
| final String name = new NamedInternal(named).orElseGenerateWithPrefix(builder, AGGREGATE_NAME); | ||
| return doAggregate( | ||
| new KStreamAggregate<>(materializedInternal.storeName(), initializer, aggregator), | ||
| AGGREGATE_NAME, | ||
| name, | ||
| materializedInternal | ||
| ); | ||
| } | ||
|
|
@@ -117,11 +137,22 @@ public <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer, | |
|
|
||
| @Override | ||
| public KTable<K, Long> count() { | ||
| return doCount(Materialized.with(keySerde, Serdes.Long())); | ||
| return doCount(NamedInternal.empty(), Materialized.with(keySerde, Serdes.Long())); | ||
| } | ||
|
|
||
| @Override | ||
| public KTable<K, Long> count(final Named named) { | ||
| Objects.requireNonNull(named, "named can't be null"); | ||
| return doCount(named, Materialized.with(keySerde, Serdes.Long())); | ||
| } | ||
|
|
||
| @Override | ||
| public KTable<K, Long> count(final Materialized<K, Long, KeyValueStore<Bytes, byte[]>> materialized) { | ||
| return count(NamedInternal.empty(), materialized); | ||
| } | ||
|
|
||
| @Override | ||
| public KTable<K, Long> count(final Named named, final Materialized<K, Long, KeyValueStore<Bytes, byte[]>> materialized) { | ||
| Objects.requireNonNull(materialized, "materialized can't be null"); | ||
|
|
||
| // TODO: remove this when we do a topology-incompatible release | ||
|
|
@@ -130,10 +161,10 @@ public KTable<K, Long> count(final Materialized<K, Long, KeyValueStore<Bytes, by | |
| builder.newStoreName(AGGREGATE_NAME); | ||
| } | ||
|
|
||
| return doCount(materialized); | ||
| return doCount(named, materialized); | ||
| } | ||
|
|
||
| private KTable<K, Long> doCount(final Materialized<K, Long, KeyValueStore<Bytes, byte[]>> materialized) { | ||
| private KTable<K, Long> doCount(final Named named, final Materialized<K, Long, KeyValueStore<Bytes, byte[]>> materialized) { | ||
| final MaterializedInternal<K, Long, KeyValueStore<Bytes, byte[]>> materializedInternal = | ||
| new MaterializedInternal<>(materialized, builder, AGGREGATE_NAME); | ||
|
|
||
|
|
@@ -144,9 +175,10 @@ private KTable<K, Long> doCount(final Materialized<K, Long, KeyValueStore<Bytes, | |
| materializedInternal.withValueSerde(Serdes.Long()); | ||
| } | ||
|
|
||
| final String name = new NamedInternal(named).orElseGenerateWithPrefix(builder, AGGREGATE_NAME); | ||
| return doAggregate( | ||
| new KStreamAggregate<>(materializedInternal.storeName(), aggregateBuilder.countInitializer, aggregateBuilder.countAggregator), | ||
| AGGREGATE_NAME, | ||
| name, | ||
| materializedInternal); | ||
| } | ||
|
|
||
|
|
@@ -184,7 +216,7 @@ private <T> KTable<K, T> doAggregate(final KStreamAggProcessorSupplier<K, K, V, | |
| final String functionName, | ||
| final MaterializedInternal<K, T, KeyValueStore<Bytes, byte[]>> materializedInternal) { | ||
| return aggregateBuilder.build( | ||
| functionName, | ||
| new NamedInternal(functionName), | ||
| new TimestampedKeyValueStoreMaterializer<>(materializedInternal).materialize(), | ||
| aggregateSupplier, | ||
| materializedInternal.queryableStoreName(), | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Would this result in a different name for the source than the prior code? (Not sure if it matters...)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hey @fhussonnois or @bbejeck , what do you think about this?