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

Compare with Current View Page History

« Previous Version 2 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

Kafka Streams methods that apply transformations to KStreams, KTtables, etc. are currently invariant in their parameterized key and value types, when they should probably be contravariant in those types. For instance, KStream.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.

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 methodscurrent argument typenew argument type
(KGroupedStream|KGroupedTable).aggregateAggregator<K, V, T>Aggregator<? super K, ? super V, T>
(KTable|KStream).(filter*|branch)Predicate<K, V> Predicate<? super K, ? super V>
(KStream).(selectKey|map|flatMap|groupBy)KeyValueMapper<K, V, X>KeyValueMapper<? super K, ? super V, X>
(KStream).(mapValues|flatMapValues)KeyValueMapper<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.foreachForeachAction<K, V>ForeachAction<? super K, ? super V>

KStream.process

ProcessorSupplier<K, V>ProcessorSupplier<? super K, ? super V>
KStream.*joinValueJoiner<K, V, R>ValueJoiner<? super K, ? super V, R>

KStream.(to|through)

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

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 affectedclasses / 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.

  • No labels