You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 5 Next »

Status

Current state: Under Discussion

Discussion thread: here

JIRA: KAFKA-4481

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

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 type, when they should probably be contravariant in those types. 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.

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 and value types defined in methods that take AggregatorStreamPartitionerKeyValueMapperValueMapperProcessorSupplier, TransformerSupplierValueTransformerSupplierForeachActionStreamPartitionerand 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, X>
KStream.transformTransformerSupplier<K, V, X>TransformerSupplier<? super K, ? super V, X>

KStream.transformValues

ValueTransformerSupplier<V, X>ValueTransformerSupplier<? super V, 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, R>

(KStream|KTable).(to|through)

StreamPartitioner<K, V>StreamPartitioner<? super K, ? super V>


Proposed Changes

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

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

None

  • No labels