...
The same reasoning applies to the key, value and result types defined in methods that take Aggregator
, StreamPartitioner
, KeyValueMapper
, ValueMapper
, ProcessorSupplier
, TransformerSupplier
, ValueTransformerSupplier
, ForeachAction
, StreamPartitioner
, and ValueJoiner
.
Public Interfaces
Affected methods | Current argument type | New argument type |
---|---|---|
(KGroupedStream|KGroupedTable).aggregate |
|
|
(KTable|KStream).filter*, KStream.branch | Predicate<K, V> | Predicate<? super K, ? super V> |
(KStream|KTable).groupBy |
|
|
KStream.(selectKey|map|flatMap), KTable.toStream | KeyValueMapper<K, V, X> | KeyValueMapper<? super K, ? super V, ? extends X> |
(KStream|KTable).mapValues, KStream.flatMapValues | ValueMapper<V, X> | ValueMapper<? super V, ? extends 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, ? extends R> |
| StreamPartitioner<K, V> | StreamPartitioner<? super K, ? super V> |
KafkaStreams.metadataForKey | StreamPartitioner<K, V> | StreamPartitioner<? super K, ? super V> |
...
Once we drop support for 1.7 we can always decide to switch to approach 2. without breaking source compatibility, by making a proposal similar to this KIP.
Notes
...
Update 2017-01-18: In light of
Jira | ||||||
---|---|---|---|---|---|---|
|
TransformerSupplier
and ValueTransformerSupplier
...