Versions Compared

Key

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

...

Several Kafka Streams methods currently take arguments that are functions parameterized in the key and value types to apply various transformations to KStreams and KTables. Those functions are currently invariant in their input in output and result types, when they should probably be contravariant in their key in / value input types, and covariant in their result type.

...

Affected methodsCurrent argument typeNew argument type
(KGroupedStream|KGroupedTable).aggregate

Aggregator<K, V, T>

Initializer<T>

Serde<T>

Aggregator<? super K, ? super V, ? extends T>

Initializer<? extends T>

Serde<? extends T>

(KTable|KStream).filter*, KStream.branchPredicate<K, V> Predicate<? super K, ? super V>
(KStream|KTable).groupBy

KeyValueMapper<K, V, X>

Serde<X>

KeyValueMapper<? super K, ? super V, ? extends X>

Serde<? extends X>

KStream.(selectKey|map|flatMap), KTable.toStreamKeyValueMapper<K, V, X>KeyValueMapper<? super K, ? super V, ? extends X>
(KStream|KTable).mapValues, KStream.flatMapValuesValueMapper<V, X>ValueMapper<? super V, ? extends X>
KStream.transformTransformerSupplier<K, V, X>TransformerSupplier<? super K, ? super V, ? extends X>

KStream.transformValues

ValueTransformerSupplier<V, X>ValueTransformerSupplier<? super V, ? extends X>
(KStream|Ktable).foreachForeachAction<K, V>ForeachAction<? super K, ? super V>

KStream.process

ProcessorSupplier<K, V>ProcessorSupplier<? super K, ? super V>
(KStream|KTable).*joinValueJoiner<K, V, R>ValueJoiner<? super K, ? super V, ? extends R>

(KStream|KTable).(to|through)

StreamPartitioner<K, V>StreamPartitioner<? super K, ? super V>
KafkaStreams.metadataForKey
StreamPartitioner<K, V>StreamPartitioner<? super K, ? super V>

...

For KGroupedStream / KGroupedTable, the change is not as straightforward for groupBy and aggregate methods, since those methods may sometimes also require passing a Serde for the corresponding result type.
For backwards compatibility reasons, the choice was made to not enforce type contraints across initializer, aggregate, and serde, even though that would have been more correct (see rejected alternatives).

Compatibility, Deprecation, and Migration Plan

...

public <VR, VAGG extends VR> KTable<K, VR> aggregate(
final Initializer<VAGG> initializer,
final Aggregator<? super K, ? super V, VAGG> aggregator,
final Serde<VAGG> aggValueSerde
);
<K1, V1<KR, KG extends K1, VG extends V1> KGroupedTable<K1KR> KGroupedStream<KR, V1>V> groupBy(
final KeyValueMapper<? super K, ? super V, ? extends KeyValue<KG, VG>> KG> selector,
final Serde<KG> keySerde,
Serde<VG>final Serde<V> valueSerdevalSerde
);

Unfortunately, when compiling against 1.7 source target, passing Aggregator<X, X, String> has the compiler incorrectly infer the result type as being KStream<T, Object>, whereas when compiling against 1.8 source target, the compiler correctly infers the result type as KStream<T, String>. It is still possible to coerce the 1.7 compiler into inferring the correct type by introducing an intermediate variable of type KStream<T, String>, or by explicitly casting to the correct type, however this makes it inconvenient to chain method calls. In addition, this would also break source compatibility for existing code compiled against 1.7 target.

...

We chose this last approach (number 3.) in order to ensure a consistent (and more correct) API while ensuring backwards compatibility for 1.7 users. Once we drop support for 1.8 we can always go back to the more correct approach decide to switch to approach 2. without breaking source compatibility.