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 the key and value typetheir input in output types, when they should probably be contravariant in those types. their key in value input types, and covariant in their result type.

For instance, KStream<K, V>.filter(Predicate<K, V> predicate) should be KStream.filter(Predicate<? super K, ? super V> predicate) to accept predicates that can act on any supertype of K, or V. More concretely, if Cat extends Animal, and I have Predicate<Animal, Object> animalPredicate, then I should be able to call KStream<Cat, Picture>.filter(animalPredicate)

Conversely for result types, KStream<K, V>.map(ValueMapper<V, R> mapper) should be KStream<K, V>.map(ValueMapper<? super V, ? extends R> mapper)For example I can apply ValueTransformer<Object, String> toStringTransformer  to KStream<K, Serializable>.map(toStringTransformer) and the result can safely be used as either KStream<K, String> or as KStream<K, Serializable> without relying on unchecked casts.

This change will make it easier to write reusable code for transformations, without requiring additional wrappers around existing code, or the unnecessary use of unchecked casts.

More concretely, if Cat extends Animal, and I have Predicate<Animal, Object> animalPredicate, then I should be able to call KStream<Cat, Picture>.filter(animalPredicate)

The same reasoning applies to the key, value and value result types defined in methods that take AggregatorStreamPartitionerKeyValueMapperValueMapperProcessorSupplier, TransformerSupplierValueTransformerSupplierForeachActionStreamPartitioner, and  and ValueJoiner.

Public Interfaces

Affected methodsCurrent argument typeNew argument type
(KGroupedStream|KGroupedTable).aggregateAggregator<K, V, T>Aggregator<? super K, ? super V, 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, 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>

...