lvyanquan commented on code in PR #3784:
URL: https://github.com/apache/flink-cdc/pull/3784#discussion_r1878042149


##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/utils/JsonRowDataSerializationSchemaUtils.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.kafka.utils;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.formats.common.TimestampFormat;
+import org.apache.flink.formats.json.JsonFormatOptions;
+import org.apache.flink.formats.json.JsonRowDataSerializationSchema;
+import org.apache.flink.table.types.logical.RowType;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.util.Arrays;
+
+/** Utils for creating JsonRowDataSerializationSchema. */
+public class JsonRowDataSerializationSchemaUtils {
+
+    /**
+     * In flink>=1.20, the constructor of JsonRowDataSerializationSchema has 6 
parameters, and in
+     * flink<1.20, the constructor of JsonRowDataSerializationSchema has 5 
parameters.
+     */
+    public static JsonRowDataSerializationSchema createSerializationSchema(
+            RowType rowType,
+            TimestampFormat timestampFormat,
+            JsonFormatOptions.MapNullKeyMode mapNullKeyMode,
+            String mapNullKeyLiteral,
+            boolean encodeDecimalAsPlainNumber,
+            boolean ignoreNullFields) {
+        try {
+            Class<?>[] fullParams =
+                    new Class[] {
+                        RowType.class,
+                        TimestampFormat.class,
+                        JsonFormatOptions.MapNullKeyMode.class,
+                        String.class,
+                        boolean.class,
+                        boolean.class
+                    };
+
+            Object[] fullParamValues =
+                    new Object[] {
+                        rowType,
+                        timestampFormat,
+                        mapNullKeyMode,
+                        mapNullKeyLiteral,
+                        encodeDecimalAsPlainNumber,
+                        ignoreNullFields
+                    };
+
+            for (int i = fullParams.length; i >= 5; i--) {
+                try {
+                    Constructor<?> constructor =
+                            
JsonRowDataSerializationSchema.class.getConstructor(
+                                    Arrays.copyOfRange(fullParams, 0, i));
+
+                    return (JsonRowDataSerializationSchema)
+                            
constructor.newInstance(Arrays.copyOfRange(fullParamValues, 0, i));
+                } catch (NoSuchMethodException ignored) {
+                }
+            }
+        } catch (Exception e) {
+            throw new RuntimeException("Failed to create 
JsonRowDataSerializationSchema", e);
+        }
+        throw new RuntimeException(
+                "Failed to find appropriate constructor for 
JsonRowDataSerializationSchema");

Review Comment:
   Add some comments like "Please check that your Flink version is 1.19 or 
1.20" to guide user?



##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/utils/JsonRowDataSerializationSchemaUtils.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.kafka.utils;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.formats.common.TimestampFormat;
+import org.apache.flink.formats.json.JsonFormatOptions;
+import org.apache.flink.formats.json.JsonRowDataSerializationSchema;
+import org.apache.flink.table.types.logical.RowType;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.util.Arrays;
+
+/** Utils for creating JsonRowDataSerializationSchema. */
+public class JsonRowDataSerializationSchemaUtils {
+
+    /**
+     * In flink>=1.20, the constructor of JsonRowDataSerializationSchema has 6 
parameters, and in
+     * flink<1.20, the constructor of JsonRowDataSerializationSchema has 5 
parameters.

Review Comment:
   Add todo to remove this class after bump to 1.20



##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/utils/JsonRowDataSerializationSchemaUtils.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.kafka.utils;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.formats.common.TimestampFormat;
+import org.apache.flink.formats.json.JsonFormatOptions;
+import org.apache.flink.formats.json.JsonRowDataSerializationSchema;
+import org.apache.flink.table.types.logical.RowType;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.util.Arrays;
+
+/** Utils for creating JsonRowDataSerializationSchema. */
+public class JsonRowDataSerializationSchemaUtils {
+
+    /**
+     * In flink>=1.20, the constructor of JsonRowDataSerializationSchema has 6 
parameters, and in
+     * flink<1.20, the constructor of JsonRowDataSerializationSchema has 5 
parameters.
+     */
+    public static JsonRowDataSerializationSchema createSerializationSchema(
+            RowType rowType,
+            TimestampFormat timestampFormat,
+            JsonFormatOptions.MapNullKeyMode mapNullKeyMode,
+            String mapNullKeyLiteral,
+            boolean encodeDecimalAsPlainNumber,
+            boolean ignoreNullFields) {
+        try {
+            Class<?>[] fullParams =
+                    new Class[] {
+                        RowType.class,
+                        TimestampFormat.class,
+                        JsonFormatOptions.MapNullKeyMode.class,
+                        String.class,
+                        boolean.class,
+                        boolean.class
+                    };
+
+            Object[] fullParamValues =
+                    new Object[] {
+                        rowType,
+                        timestampFormat,
+                        mapNullKeyMode,
+                        mapNullKeyLiteral,
+                        encodeDecimalAsPlainNumber,
+                        ignoreNullFields
+                    };
+
+            for (int i = fullParams.length; i >= 5; i--) {
+                try {
+                    Constructor<?> constructor =
+                            
JsonRowDataSerializationSchema.class.getConstructor(
+                                    Arrays.copyOfRange(fullParams, 0, i));
+
+                    return (JsonRowDataSerializationSchema)
+                            
constructor.newInstance(Arrays.copyOfRange(fullParamValues, 0, i));
+                } catch (NoSuchMethodException ignored) {
+                }
+            }
+        } catch (Exception e) {
+            throw new RuntimeException("Failed to create 
JsonRowDataSerializationSchema", e);
+        }
+        throw new RuntimeException(
+                "Failed to find appropriate constructor for 
JsonRowDataSerializationSchema");
+    }
+
+    /** flink>=1.20 only has the ENCODE_IGNORE_NULL_FIELDS parameter. */

Review Comment:
   Add todo to remove this class after bump to 1.20



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