lindong28 commented on a change in pull request #18397:
URL: https://github.com/apache/flink/pull/18397#discussion_r788860986



##########
File path: 
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchema.java
##########
@@ -123,9 +121,8 @@ default void 
open(DeserializationSchema.InitializationContext context) throws Ex
      * @return A {@link KafkaRecordDeserializationSchema} that deserialize the 
value with the given
      *     deserializer.
      */
-    static <V, D extends Configurable & Deserializer<V>>
-            KafkaRecordDeserializationSchema<V> valueOnly(
-                    Class<D> valueDeserializerClass, Map<String, String> 
config) {
+    static <V, D extends Deserializer<V>> KafkaRecordDeserializationSchema<V> 
valueOnly(

Review comment:
       Prior to https://github.com/apache/flink/pull/15051, we have the 
following method in the class `KafkaRecordDeserializer`:
   
   ```
       /**
        * Wraps a Kafka {@link Deserializer} to a {@link 
KafkaRecordDeserializer}.
        *
        * @param valueDeserializerClass the deserializer class used to 
deserialize the value.
        * @param config the configuration of the value deserializer, only valid 
when the deserializer
        *     is an implementation of {@code Configurable}.
        * @param <V> the value type.
        * @param <D> the type of the deserializer.
        * @return A {@link KafkaRecordDeserializer} that deserialize the value 
with the given
        *     deserializer.
        */
       static <V, D extends Configurable & Deserializer<V>> 
KafkaRecordDeserializer<V> valueOnly(
               Class<D> valueDeserializerClass, Map<String, String> config) {
       ...
       }
   ```
   
   This method basically assumes that users might have defined a deserializerA 
as a subclass of both Configurable and Deserializer, where 
`deserializerA::configure(Map<String, ?>)` could have user-defined 
configuration logic. Since this subclass could be defined by user outside the 
Flink project, we want to continue to support this this user-defined 
deserializer for backward compatibility.
   
   Changing the method signature in this way could cause backward 
incompatibility. Though this method is not marked `@public`, I think this is 
because we forgot to add this tag. Given that this method is supposed to be 
used by public users, I feel it is preferred not to break it.
   
   What do you think?
   
   




-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to