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


##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/debezium/DebeziumJsonSerializationSchema.java:
##########
@@ -138,81 +185,202 @@ public byte[] serialize(Event event) {
         }
 
         DataChangeEvent dataChangeEvent = (DataChangeEvent) event;
-        reuseGenericRowData.setField(
-                3,
-                GenericRowData.of(
-                        
StringData.fromString(dataChangeEvent.tableId().getSchemaName()),
-                        
StringData.fromString(dataChangeEvent.tableId().getTableName())));
+        BiConsumer<DataChangeEvent, GenericRowData> converter;
         try {
             switch (dataChangeEvent.op()) {
                 case INSERT:
-                    reuseGenericRowData.setField(0, null);
-                    reuseGenericRowData.setField(
-                            1,
-                            jsonSerializers
-                                    .get(dataChangeEvent.tableId())
-                                    
.getRowDataFromRecordData(dataChangeEvent.after(), false));
-                    reuseGenericRowData.setField(2, OP_INSERT);
-                    return jsonSerializers
-                            .get(dataChangeEvent.tableId())
-                            .getSerializationSchema()
-                            .serialize(reuseGenericRowData);
+                    converter = this::convertInsertEventToRowData;
+                    break;
                 case DELETE:
-                    reuseGenericRowData.setField(
-                            0,
-                            jsonSerializers
-                                    .get(dataChangeEvent.tableId())
-                                    
.getRowDataFromRecordData(dataChangeEvent.before(), false));
-                    reuseGenericRowData.setField(1, null);
-                    reuseGenericRowData.setField(2, OP_DELETE);
-                    return jsonSerializers
-                            .get(dataChangeEvent.tableId())
-                            .getSerializationSchema()
-                            .serialize(reuseGenericRowData);
+                    converter = this::convertDeleteEventToRowData;
+                    break;
                 case UPDATE:
                 case REPLACE:
-                    reuseGenericRowData.setField(
-                            0,
-                            jsonSerializers
-                                    .get(dataChangeEvent.tableId())
-                                    
.getRowDataFromRecordData(dataChangeEvent.before(), false));
-                    reuseGenericRowData.setField(
-                            1,
-                            jsonSerializers
-                                    .get(dataChangeEvent.tableId())
-                                    
.getRowDataFromRecordData(dataChangeEvent.after(), false));
-                    reuseGenericRowData.setField(2, OP_UPDATE);
-                    return jsonSerializers
-                            .get(dataChangeEvent.tableId())
-                            .getSerializationSchema()
-                            .serialize(reuseGenericRowData);
+                    converter = this::convertUpdateEventToRowData;
+                    break;
                 default:
                     throw new UnsupportedOperationException(
                             format(
                                     "Unsupported operation '%s' for 
OperationType.",
                                     dataChangeEvent.op()));
             }
+
+            if (isIncludedDebeziumSchema) {
+                converter.accept(dataChangeEvent, payloadGenericRowData);
+                reuseGenericRowData.setField(
+                        SCHEMA.getPosition(),
+                        
StringData.fromString(schemaMap.get(dataChangeEvent.tableId())));
+            } else {
+                converter.accept(dataChangeEvent, reuseGenericRowData);
+            }
+            return jsonSerializers
+                    .get(dataChangeEvent.tableId())
+                    .getSerializationSchema()
+                    .serialize(reuseGenericRowData);
         } catch (Throwable t) {
             throw new RuntimeException(format("Could not serialize event 
'%s'.", event), t);
         }
     }
 
+    public String convertSchemaToDebeziumSchema(Schema schema) {

Review Comment:
   It's better to add a example for the output in Java Doc to help developers 
understanding.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to