Table of Contents |
---|
Status
Current state: Under Discussion Accepted
Discussion thread: here
JIRA: KAFKA-4481
...
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 their input in output and result types, when they should probably be contravariant in their key in / value input types, and covariant in their result type.
...
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, ? 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> |
...
For KGroupedStream/KGroupedTable groupBy
and aggregate
methods it was decided to leave the return type invariant, since the change is not as straightforward for groupBy and aggregate methods, since those methods may also require passing a Serde for the corresponding . Those methods sometimes require passing a Serde<T>
or Initializer<T>
where T needs to be consistent with the Aggregator
(for aggregate) or KeyValueMapper
(for groupBy) result type.
For backwards compatibility reasons, and to avoid runtime class cast exceptions, the choice was made to not enforce type contraints across initializer, aggregate, and serde, make the result type covariant, even though that would have been more correct (see rejected alternatives).
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.
- 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.
public <VR, VAGG extends VR> KTable<K, VR> aggregate(
final Initializer<VAGG> initializer,
final Aggregator<? super K, ? super V, VAGG> aggregator,
final Serde<VAGG> aggValueSerde
);
<K1, V1<KR, KG extends K1, VG extends V1> KGroupedTable<K1KR> KGroupedStream<KR, V1>V> groupBy(
final KeyValueMapper<? super K, ? super V, ? extends KeyValue<KG, VG>> KG> selector,
final Serde<KG> keySerde,
final Serde<VG>Serde<V> valueSerdevalSerde
);
Unfortunately, when compiling against 1.7 source target, passing Aggregator<X, X, String> has the compiler incorrectly infer the result type as being KStream<T, Object>, whereas when compiling against 1.8 source target, the compiler correctly infers the result type as KStream<T, String>. It is still possible to coerce the 1.7 compiler into inferring the correct type by introducing an intermediate variable of type KStream<T, String>, or by explicitly casting to the correct type, however this makes it inconvenient to chain method calls. In addition, this would also break source compatibility for existing code compiled against 1.7 target.
...
- make no changes to the output type, i.e. keep the existing output type invariant, leaving an the inconsistent API which would require and do another API change once we can drop support for 1.7
- use the more correct <T, VAGG extends T> contraint, and break source compatibility for 1.7 targets, forcing those users to rely on ugly casts or intermediate variables.
make the API consistent by making result types covariant using wildcards
? extends V
, relaxing compile time correctness across initializer, aggregator, and serde output types. We initially explored this route because if was backward compatible, but decided to drop it because it would a) require non-trivial changes to the existing streams code, and b) introduce lots of unchecked casts that could blow up at runtime if a user is not careful to ensure consistency across output types in aggregate / groupBy
We decided to chose this last approach 1. at the expense of a more consistent API, to ensure backwards compatibility for 1.7 users and avoid the complexity and potential pitfalls of the last approach.
(number 3.) in order to ensure a consistent (and more correct) API while ensuring backwards compatibility. Once we drop support for 1.8 7 we can always go back to the more correct approach 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