...
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, ? extends X> |
| ValueTransformerSupplier<V, X> | ValueTransformerSupplier<? super V, ? extends 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> |
...
- 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.
- Update (2017-01-18): This change is not source compatible for anyone calling the Kafka Streams API from Scala due to differences in how Scala infers types.
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.
...
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