hailin0 commented on code in PR #3950:
URL: 
https://github.com/apache/incubator-seatunnel/pull/3950#discussion_r1072048360


##########
seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java:
##########
@@ -0,0 +1,282 @@
+/*
+ * 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.seatunnel.format.json.canal;
+
+import static org.apache.seatunnel.api.table.type.BasicType.STRING_TYPE;
+import static java.lang.String.format;
+
+import org.apache.seatunnel.api.serialization.DeserializationSchema;
+import org.apache.seatunnel.api.source.Collector;
+import org.apache.seatunnel.api.table.type.RowKind;
+import org.apache.seatunnel.api.table.type.SeaTunnelDataType;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+import org.apache.seatunnel.common.exception.CommonErrorCode;
+import org.apache.seatunnel.format.json.JsonDeserializationSchema;
+import org.apache.seatunnel.format.json.exception.SeaTunnelJsonFormatException;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.google.common.collect.Lists;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.regex.Pattern;
+
+public class CanalJsonDeserializationSchema implements 
DeserializationSchema<SeaTunnelRow> {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final String FIELD_OLD = "old";
+
+    private static final String FIELD_DATA = "data";
+
+    private static final String FIELD_TYPE = "type";
+
+    private static final String OP_INSERT = "INSERT";
+
+    private static final String OP_UPDATE = "UPDATE";
+
+    private static final String OP_DELETE = "DELETE";
+
+    private static final String OP_CREATE = "CREATE";
+
+    private String database;
+
+    private String table;
+
+    /** Names of fields. */
+    private final String[] fieldNames;
+
+    /** Number of fields. */
+    private final int fieldCount;
+
+    private boolean ignoreParseErrors;
+
+    /** Pattern of the specific database. */
+    private final Pattern databasePattern;
+
+    /** Pattern of the specific table. */
+    private final Pattern tablePattern;
+
+    private final JsonDeserializationSchema jsonDeserializer;
+
+    private final SeaTunnelRowType physicalRowType;
+
+    public CanalJsonDeserializationSchema(SeaTunnelRowType physicalRowType,
+                                          String database,
+                                          String table,
+                                          boolean ignoreParseErrors
+                                          ) {
+        this.physicalRowType = physicalRowType;
+        final SeaTunnelRowType jsonRowType = 
createJsonRowType(physicalRowType);
+        this.jsonDeserializer = new JsonDeserializationSchema(
+            false,
+            ignoreParseErrors,
+            jsonRowType
+        );
+        this.database = database;
+        this.table = table;
+        this.fieldNames = physicalRowType.getFieldNames();
+        this.fieldCount = physicalRowType.getTotalFields();
+        this.ignoreParseErrors = ignoreParseErrors;
+        this.databasePattern = database == null ? null : 
Pattern.compile(database);
+        this.tablePattern = table == null ? null : Pattern.compile(table);
+    }
+
+    @Override
+    public SeaTunnelRow deserialize(byte[] message) throws IOException {
+        if (message == null) {
+            return null;
+        }
+        return convertJsonNode(convertBytes(message));
+    }
+
+    @Override
+    public SeaTunnelDataType<SeaTunnelRow> getProducedType() {
+        return this.physicalRowType;
+    }
+
+    public void deserialize(byte[] message, Collector<SeaTunnelRow> out) 
throws IOException {

Review Comment:
   example
   
   ```java
   public void deserialize(byte[] message, Collector<SeaTunnelRow> out) throws 
IOException {
           JsonNode jsonNode = convert(message);
           ArrayNode dataNode = (ArrayNode) jsonNode.get("data");
           switch (jsonNode.get("type")) {
               case "update":
                   ArrayNode oldNode = (ArrayNode) jsonNode.get("old");
                   for (int i = 0; i < dataNode.size(); i++) {
                       JsonNode data = dataNode.get(i);
                       JsonNode old = oldNode.get(i);
   
                       SeaTunnelRow updateBefore = convert(jsonNode, data);
                       SeaTunnelRow updateAfter = convert(jsonNode, old);
   
                       out.collect(updateBefore);
                       out.collect(updateAfter);
                   }
                   break;
               case "create":
                   for (int i = 0; i < dataNode.size(); i++) {
                       JsonNode data = dataNode.get(i);
                       SeaTunnelRow insert = convert(jsonNode, data);
                       out.collect(insert);
                   }
                   break;
               case "delete":
                   for (int i = 0; i < dataNode.size(); i++) {
                       JsonNode data = dataNode.get(i);
                       SeaTunnelRow delete = convert(jsonNode, data);
                       out.collect(delete);
                   }
                   break;
           }
       }
   ```



##########
seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java:
##########
@@ -0,0 +1,282 @@
+/*
+ * 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.seatunnel.format.json.canal;
+
+import static org.apache.seatunnel.api.table.type.BasicType.STRING_TYPE;
+import static java.lang.String.format;
+
+import org.apache.seatunnel.api.serialization.DeserializationSchema;
+import org.apache.seatunnel.api.source.Collector;
+import org.apache.seatunnel.api.table.type.RowKind;
+import org.apache.seatunnel.api.table.type.SeaTunnelDataType;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+import org.apache.seatunnel.common.exception.CommonErrorCode;
+import org.apache.seatunnel.format.json.JsonDeserializationSchema;
+import org.apache.seatunnel.format.json.exception.SeaTunnelJsonFormatException;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.google.common.collect.Lists;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.regex.Pattern;
+
+public class CanalJsonDeserializationSchema implements 
DeserializationSchema<SeaTunnelRow> {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final String FIELD_OLD = "old";
+
+    private static final String FIELD_DATA = "data";
+
+    private static final String FIELD_TYPE = "type";
+
+    private static final String OP_INSERT = "INSERT";
+
+    private static final String OP_UPDATE = "UPDATE";
+
+    private static final String OP_DELETE = "DELETE";
+
+    private static final String OP_CREATE = "CREATE";
+
+    private String database;
+
+    private String table;
+
+    /** Names of fields. */
+    private final String[] fieldNames;
+
+    /** Number of fields. */
+    private final int fieldCount;
+
+    private boolean ignoreParseErrors;
+
+    /** Pattern of the specific database. */
+    private final Pattern databasePattern;
+
+    /** Pattern of the specific table. */
+    private final Pattern tablePattern;
+
+    private final JsonDeserializationSchema jsonDeserializer;
+
+    private final SeaTunnelRowType physicalRowType;
+
+    public CanalJsonDeserializationSchema(SeaTunnelRowType physicalRowType,
+                                          String database,
+                                          String table,
+                                          boolean ignoreParseErrors
+                                          ) {
+        this.physicalRowType = physicalRowType;
+        final SeaTunnelRowType jsonRowType = 
createJsonRowType(physicalRowType);
+        this.jsonDeserializer = new JsonDeserializationSchema(
+            false,
+            ignoreParseErrors,
+            jsonRowType
+        );
+        this.database = database;
+        this.table = table;
+        this.fieldNames = physicalRowType.getFieldNames();
+        this.fieldCount = physicalRowType.getTotalFields();
+        this.ignoreParseErrors = ignoreParseErrors;
+        this.databasePattern = database == null ? null : 
Pattern.compile(database);
+        this.tablePattern = table == null ? null : Pattern.compile(table);
+    }
+
+    @Override
+    public SeaTunnelRow deserialize(byte[] message) throws IOException {
+        if (message == null) {
+            return null;
+        }
+        return convertJsonNode(convertBytes(message));
+    }

Review Comment:
   ```suggestion
       public SeaTunnelRow deserialize(byte[] message) throws IOException {
           throw new UnsupportedOperationException();
       }
   ```



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