...
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 , KTtables, etc. and KTables. Those functions are currently invariant in their parameterized the key and value typestype, when they should probably be contravariant in those types. For instance, KStream<K,
KStreamV>.filter(Predicate<K, V> predicate)
should be KStreambe 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 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)
This change will make it easier to write reusable code for some transformations, without requiring additional wrappers around existing code, or the use of unchecked casts.
The same reasoning applies to the key and value types used in methods that take Aggregator, StreamPartitioner, KeyValueMapper, ValueMapper, ProcessorSupplier, and ValueJoiner.
Public Interfaces
Affected methods | current Current argument type | new New argument type |
---|---|---|
(KGroupedStream|KGroupedTable).aggregate | Aggregator<K, V, T> | Aggregator<? super K, ? super V, T> |
(KTable|KStream).(filter*|, KStream.branch) | Predicate<K, V> | Predicate<? super K, ? super V> |
(KStream|KTable).groupBy, KStream.(selectKey|map|flatMap|groupBy) , KTable.toStream | KeyValueMapper<K KeyValueMapper<K, V, X> | KeyValueMapper<? super K, ? super V, X> |
(KStream|KTable).(mapValues|, KStream.flatMapValues) | KeyValueMapper<VKeyValueMapper<V, X> | ValueMapper<? super V, X> |
KStream.transform | TransformerSupplier<K, V, X> | TransformerSupplier<? super K, ? super V, X> |
| ValueTransformerSupplier<V, X> | ValueTransformerSupplier<? super V, X> |
(KStream|Ktable).foreach | ForeachAction<K, V> | ForeachAction<? super K, ? super V> |
| ProcessorSupplier<K, V> | ProcessorSupplier<? super K, ? super V> |
(KStream|KTable).*join | ValueJoiner<K, V, R> | ValueJoiner<? super K, ? super V, R> |
| StreamPartitioner<K, V> | StreamPartitioner<? super K, ? super V> |
All methods will have their arguments changed to
All methods will have their arguments changed to
KStream.to
A public interface is any change to the following:
Binary log format
The network protocol and api behavior
Any class in the public packages under clientsConfiguration, especially client configuration
org/apache/kafka/common/serialization
org/apache/kafka/common
org/apache/kafka/common/errors
org/apache/kafka/clients/producer
org/apache/kafka/clients/consumer (eventually, once stable)
Monitoring
Command line tools and arguments
- Anything else that will likely break existing users in some way when they upgrade
Proposed Changes
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.Describe the new thing you want to do in appropriate detail. This may be fairly extensive and have large subsections of its own. Or it may be a few sentences. Use judgement based on the scope of the change.
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 affectedthe affected classes / interfaces.
Rejected Alternatives
If there are alternative ways of accomplishing the same thing, what were they? The purpose of this section is to motivate why the design is the way it is and not some other way.None