mjsax commented on code in PR #18722:
URL: https://github.com/apache/kafka/pull/18722#discussion_r1933234358


##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java:
##########
@@ -91,20 +91,20 @@ Set<String> ensureCopartitionWith(final Collection<? 
extends AbstractStream<K, ?
         return allSourceNodes;
     }
 
-    static <T2, T1, R> ValueJoiner<T2, T1, R> reverseJoiner(final 
ValueJoiner<T1, T2, R> joiner) {
+    static <VRight, VLeft, VOut> ValueJoiner<VRight, VLeft, VOut> 
reverseJoiner(final ValueJoiner<VLeft, VRight, VOut> joiner) {
         return (value2, value1) -> joiner.apply(value1, value2);
     }
 
-    static <K, T2, T1, R> ValueJoinerWithKey<K, T2, T1, R> 
reverseJoinerWithKey(final ValueJoinerWithKey<K, T1, T2, R> joiner) {
+    static <K, VRight, VLeft, VOut> ValueJoinerWithKey<K, VRight, VLeft, VOut> 
reverseJoinerWithKey(final ValueJoinerWithKey<K, VLeft, VRight, VOut> joiner) {
         return (key, value2, value1) -> joiner.apply(key, value1, value2);
     }
 
-    static <K, V, VR> ValueMapperWithKey<K, V, VR> withKey(final 
ValueMapper<V, VR> valueMapper) {
+    static <K, V, VOut> ValueMapperWithKey<K, V, VOut> withKey(final 
ValueMapper<V, VOut> valueMapper) {
         Objects.requireNonNull(valueMapper, "valueMapper can't be null");
         return (readOnlyKey, value) -> valueMapper.apply(value);
     }
 
-    static <K, V1, V2, VR> ValueJoinerWithKey<K, V1, V2, VR> 
toValueJoinerWithKey(final ValueJoiner<V1, V2, VR> valueJoiner) {
+    static <K, VLeft, VRight, VOut> ValueJoinerWithKey<K, VLeft, VRight, VOut> 
toValueJoinerWithKey(final ValueJoiner<VLeft, VRight, VOut> valueJoiner) {

Review Comment:
   We don't need it from my understanding (please correct me if I am wrong).
   
   Assume, we have `KStreams<K, A>` and `KStream<K, B>` that we join, to result 
`KStream<K, C>`. (All K,A,B,C are concrete types)
   
   We use `ValueJoiner<KFoo, V, W, X>` (which are concrete types) with `K 
extend KFoo`, `A extends V`, `B extends W` and (reverser for output), `X 
extends C`, we just want to translate the `ValueJoiner` to `<KFoo, W, V, X>` 
which are still concrete types. And yes, these concreate types must match `? 
extends...` and `? super ` for the KStreams we process, but they still do.
   
   Thus, we don't need this helper to be able to translate a child/super type 
of `ValueJoiner<KFoo, V, W, X>` -- it's sufficient to just translate the 
concrete types `<KFoo, V, W, X>`.
   
   Or do I get this wrong?



-- 
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

Reply via email to