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


##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java:
##########
@@ -1204,75 +1201,6 @@ private <VO, VR> KStream<K, VR> doStreamTableJoin(final 
KTable<K, VO> table,
             builder);
     }
 
-    @Override
-    @Deprecated
-    public <VR> KStream<K, VR> flatTransformValues(final 
org.apache.kafka.streams.kstream.ValueTransformerSupplier<? super V, 
Iterable<VR>> valueTransformerSupplier,
-                                                   final String... 
stateStoreNames) {
-        Objects.requireNonNull(valueTransformerSupplier, 
"valueTransformerSupplier can't be null");
-        return doFlatTransformValues(
-            toValueTransformerWithKeySupplier(valueTransformerSupplier),
-            NamedInternal.empty(),
-            stateStoreNames);
-    }
-
-    @Override
-    @Deprecated
-    public <VR> KStream<K, VR> flatTransformValues(final 
org.apache.kafka.streams.kstream.ValueTransformerSupplier<? super V, 
Iterable<VR>> valueTransformerSupplier,
-                                                   final Named named,
-                                                   final String... 
stateStoreNames) {
-        Objects.requireNonNull(valueTransformerSupplier, 
"valueTransformerSupplier can't be null");
-        return doFlatTransformValues(
-            toValueTransformerWithKeySupplier(valueTransformerSupplier),
-            named,
-            stateStoreNames);
-    }
-
-    @Override
-    @Deprecated
-    public <VR> KStream<K, VR> flatTransformValues(final 
ValueTransformerWithKeySupplier<? super K, ? super V, Iterable<VR>> 
valueTransformerSupplier,
-                                                   final String... 
stateStoreNames) {
-        Objects.requireNonNull(valueTransformerSupplier, 
"valueTransformerSupplier can't be null");
-        return doFlatTransformValues(valueTransformerSupplier, 
NamedInternal.empty(), stateStoreNames);
-    }
-
-    @Override
-    @Deprecated
-    public <VR> KStream<K, VR> flatTransformValues(final 
ValueTransformerWithKeySupplier<? super K, ? super V, Iterable<VR>> 
valueTransformerSupplier,
-                                                   final Named named,
-                                                   final String... 
stateStoreNames) {
-        Objects.requireNonNull(valueTransformerSupplier, 
"valueTransformerSupplier can't be null");
-        return doFlatTransformValues(valueTransformerSupplier, named, 
stateStoreNames);
-    }
-
-    private <VR> KStream<K, VR> doFlatTransformValues(final 
ValueTransformerWithKeySupplier<? super K, ? super V, Iterable<VR>> 
valueTransformerWithKeySupplier,
-                                                      final Named named,
-                                                      final String... 
stateStoreNames) {
-        Objects.requireNonNull(stateStoreNames, "stateStoreNames can't be a 
null array");
-        for (final String stateStoreName : stateStoreNames) {
-            Objects.requireNonNull(stateStoreName, "stateStoreNames can't 
contain `null` as store name");
-        }
-        ApiUtils.checkSupplier(valueTransformerWithKeySupplier);

Review Comment:
   I think we can also remove this `checkSupplier(...)` overload from 
`ApiUtils` ?



##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java:
##########
@@ -1204,75 +1201,6 @@ private <VO, VR> KStream<K, VR> doStreamTableJoin(final 
KTable<K, VO> table,
             builder);
     }
 
-    @Override
-    @Deprecated
-    public <VR> KStream<K, VR> flatTransformValues(final 
org.apache.kafka.streams.kstream.ValueTransformerSupplier<? super V, 
Iterable<VR>> valueTransformerSupplier,
-                                                   final String... 
stateStoreNames) {
-        Objects.requireNonNull(valueTransformerSupplier, 
"valueTransformerSupplier can't be null");
-        return doFlatTransformValues(
-            toValueTransformerWithKeySupplier(valueTransformerSupplier),
-            NamedInternal.empty(),
-            stateStoreNames);
-    }
-
-    @Override
-    @Deprecated
-    public <VR> KStream<K, VR> flatTransformValues(final 
org.apache.kafka.streams.kstream.ValueTransformerSupplier<? super V, 
Iterable<VR>> valueTransformerSupplier,
-                                                   final Named named,
-                                                   final String... 
stateStoreNames) {
-        Objects.requireNonNull(valueTransformerSupplier, 
"valueTransformerSupplier can't be null");
-        return doFlatTransformValues(
-            toValueTransformerWithKeySupplier(valueTransformerSupplier),

Review Comment:
   I think we can also remove `toValueTransformerWithKeySupplier(...)` ? Should 
be unused now?



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