Versions Compared

Key

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

...

Affected methodsCurrent argument typeNew argument type
(KGroupedStream|KGroupedTable).aggregateAggregator<K, V, T>Aggregator<? super K, ? super V, ? extends T>
(KTable|KStream).filter*, KStream.branchPredicate<K, V> Predicate<? super K, ? super V>
(KStream|KTable).groupBy, 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>

...

This KIP proposes changing the methods on the interfaces listed above to relax function arguments parameterized in key in , value, and return types to accept super-types of those key and values, and sub-types of those return types.

For KGroupedStream / KGroupedTable, the change is not as straightforward for groupBy and aggregate methods, since those methods may 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

  • This change is binary compatible
  • This change is source compatible for anyone merely calling the existing APIs
  • This change is not source compatible for anyone extending the affected classes / interfaces.

Rejected Alternatives

For the aggregate and groupBy case the following alternatives would have been more correct – if we could drop support for Java 7 – since they would enforce the same type for initializer, serializer, and serde. 

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, KG extends K1, VG extends V1> KGroupedTable<K1, V1> groupBy(
KeyValueMapper<? super K, ? super V, ? extends KeyValue<KG, VG>> selector,
Serde<KG> keySerde,
Serde<VG> valueSerde
);

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.

In light of that we were forced to either:

  1. make no changes to the output type, i.e. keep the existing output type invariant, leaving an inconsistent API which would require another API change once we can drop support for 1.7
  2. use the more correct <T, VAGG extends T> contraint, and break source compatibility for 1.7 targets, forcing those users to rely on ugly casts or intermediate variables.
  3. make the API consistent by making result types covariant using wildcards ? extends V, relaxing compile time correctness across initializer, aggregator, and serde output types.

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