mjsax commented on code in PR #18722: URL: https://github.com/apache/kafka/pull/18722#discussion_r1933225385
########## streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java: ########## @@ -153,9 +153,9 @@ public KStream<K, V> filter(final Predicate<? super K, ? super V> predicate, Objects.requireNonNull(named, "named can't be null"); final String name = new NamedInternal(named).orElseGenerateWithPrefix(builder, FILTER_NAME); - final ProcessorParameters<? super K, ? super V, ?, ?> processorParameters = + final ProcessorParameters<K, V, K, V> processorParameters = Review Comment: We need `? super K` only for the user-implemented interfaces / UDFs. This allows to process a KStream of type `<K,V>`, with a UDF which accepts a supertype of K and or V (or returns child type, eg, `? extend VOut`) However, it does not change the type of the input KStream itself -- and also not the type of the result KStream... The actual/concrete type is still `<K,V>`. Assume we filter a `KStream<Void,Cat>` and apply an `Predicate<Void,Animal>` which gives us animals with black fur. The result `KStream` is still `<Void,Cat>`... If we would map `KStream<Void,Something>` using `ValueMapper<Cats>`, we can still treat the result as `KStream<Void,Animal>`. Thus, for the internal `ProcessParameter` we only need to track the _actual/concrete_ types, which are independent of potential super/child type for UDF generics. The code compiles both ways for `ProcessorParameter`, but it's just "silly" to use anything except the concrete types, as the compiler cannot check anything else anyway.... -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org