wuchong commented on a change in pull request #13090:
URL: https://github.com/apache/flink/pull/13090#discussion_r467776390



##########
File path: 
flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.formats.json.maxwell;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.formats.json.JsonRowDataDeserializationSchema;
+import org.apache.flink.formats.json.TimestampFormat;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.Collector;
+
+import java.io.IOException;
+import java.util.Objects;
+
+import static java.lang.String.format;
+import static 
org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType;
+
+/**
+ * Deserialization schema from Maxwell JSON to Flink Table/SQL internal data 
structure {@link RowData}.
+ * The deserialization schema knows Maxwell's schema definition and can 
extract the database data
+ * and convert into {@link RowData} with {@link RowKind}.
+ *
+ * <p>Deserializes a <code>byte[]</code> message as a JSON object and reads
+ * the specified fields.
+ *
+ * <p>Failures during deserialization are forwarded as wrapped IOExceptions.
+ *
+ * @see <a href="http://maxwells-daemon.io/";>Maxwell</a>
+ */
+public class MaxwellJsonDeserializationSchema implements 
DeserializationSchema<RowData> {
+       private static final long serialVersionUID = 1L;
+
+       private static final String OP_INSERT = "insert";
+       private static final String OP_UPDATE = "update";
+       private static final String OP_DELETE = "delete";
+
+       /** The deserializer to deserialize Maxwell JSON data. */
+       private final JsonRowDataDeserializationSchema jsonDeserializer;
+
+       /** TypeInformation of the produced {@link RowData}. **/
+       private final TypeInformation<RowData> resultTypeInfo;
+
+       /** Flag indicating whether to ignore invalid fields/rows (default: 
throw an exception). */
+       private final boolean ignoreParseErrors;
+
+       /** Number of fields. */
+       private final int fieldCount;
+
+       public MaxwellJsonDeserializationSchema(RowType rowType,
+                                                                               
        TypeInformation<RowData> resultTypeInfo,
+                                                                               
        boolean ignoreParseErrors,
+                                                                               
        TimestampFormat timestampFormatOption) {

Review comment:
       ```suggestion
        public MaxwellJsonDeserializationSchema(
                        RowType rowType,
                        TypeInformation<RowData> resultTypeInfo,
                        boolean ignoreParseErrors,
                        TimestampFormat timestampFormatOption) {
   ```
   
   Indent.

##########
File path: 
flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.formats.json.maxwell;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.formats.json.JsonRowDataDeserializationSchema;
+import org.apache.flink.formats.json.TimestampFormat;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.Collector;
+
+import java.io.IOException;
+import java.util.Objects;
+
+import static java.lang.String.format;
+import static 
org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType;
+
+/**
+ * Deserialization schema from Maxwell JSON to Flink Table/SQL internal data 
structure {@link RowData}.
+ * The deserialization schema knows Maxwell's schema definition and can 
extract the database data
+ * and convert into {@link RowData} with {@link RowKind}.
+ *
+ * <p>Deserializes a <code>byte[]</code> message as a JSON object and reads
+ * the specified fields.
+ *
+ * <p>Failures during deserialization are forwarded as wrapped IOExceptions.
+ *
+ * @see <a href="http://maxwells-daemon.io/";>Maxwell</a>
+ */
+public class MaxwellJsonDeserializationSchema implements 
DeserializationSchema<RowData> {
+       private static final long serialVersionUID = 1L;
+
+       private static final String OP_INSERT = "insert";
+       private static final String OP_UPDATE = "update";
+       private static final String OP_DELETE = "delete";
+
+       /** The deserializer to deserialize Maxwell JSON data. */
+       private final JsonRowDataDeserializationSchema jsonDeserializer;
+
+       /** TypeInformation of the produced {@link RowData}. **/
+       private final TypeInformation<RowData> resultTypeInfo;
+
+       /** Flag indicating whether to ignore invalid fields/rows (default: 
throw an exception). */
+       private final boolean ignoreParseErrors;
+
+       /** Number of fields. */
+       private final int fieldCount;
+
+       public MaxwellJsonDeserializationSchema(RowType rowType,
+                                                                               
        TypeInformation<RowData> resultTypeInfo,
+                                                                               
        boolean ignoreParseErrors,
+                                                                               
        TimestampFormat timestampFormatOption) {
+               this.resultTypeInfo = resultTypeInfo;
+               this.ignoreParseErrors = ignoreParseErrors;
+               this.fieldCount = rowType.getFieldCount();
+               this.jsonDeserializer = new JsonRowDataDeserializationSchema(
+                       createJsonRowType(fromLogicalToDataType(rowType)),
+                       // the result type is never used, so it's fine to pass 
in Canal's result type
+                       resultTypeInfo,
+                       false, // ignoreParseErrors already contains the 
functionality of failOnMissingField
+                       ignoreParseErrors,
+                       timestampFormatOption);
+       }
+
+       @Override
+       public RowData deserialize(byte[] message) throws IOException {
+               throw new RuntimeException(
+                       "Please invoke 
DeserializationSchema#deserialize(byte[], Collector<RowData>) instead.");
+       }
+
+       @Override
+       public void deserialize(byte[] message, Collector<RowData> out) throws 
IOException {
+               try {
+                       RowData row = jsonDeserializer.deserialize(message);
+                       String type = row.getString(2).toString(); // "type" 
field
+                       if (OP_INSERT.equals(type)) {
+                               // difference with canal-json is that the 
"data" section in canal-json contains Array <Row <>>,
+                               // and maxwell-json is Row <>
+                               // So just take out Row<> in row and collect it

Review comment:
       We don't need to compare with canal-json in the comment. From my point 
of view, canal-json is a special case in encoding. You can just mention the 
"data" is the new image of insertion. 

##########
File path: 
flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.formats.json.maxwell;
+
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.formats.json.JsonRowDataSerializationSchema;
+import org.apache.flink.formats.json.TimestampFormat;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.types.RowKind;
+
+import java.util.Objects;
+
+import static 
org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType;
+
+/**
+ * Serialization schema from Flink Table/SQL internal data structure {@link 
RowData} to maxwell-Json.
+ */
+public class MaxwellJsonSerializationSchema implements 
SerializationSchema<RowData> {
+       private static final long serialVersionUID = 1L;
+
+       private static final StringData OP_INSERT = 
StringData.fromString("insert");
+       private static final StringData OP_DELETE = 
StringData.fromString("delete");
+
+       private final JsonRowDataSerializationSchema jsonSerializer;
+
+       /**
+        * Timestamp format specification which is used to parse timestamp.
+        */
+       private final TimestampFormat timestampFormat;
+
+       private transient GenericRowData reuse;
+
+       public MaxwellJsonSerializationSchema(RowType rowType, TimestampFormat 
timestampFormat) {
+               this.jsonSerializer = new JsonRowDataSerializationSchema(
+                       createJsonRowType(fromLogicalToDataType(rowType)),
+                       timestampFormat);
+               this.timestampFormat = timestampFormat;
+       }
+
+       @Override
+       public void open(InitializationContext context) throws Exception {
+               this.reuse = new GenericRowData(3);
+       }
+
+       @Override
+       public byte[] serialize(RowData element) {
+               reuse.setField(0, element);
+               reuse.setField(1, null);
+               reuse.setField(2, rowKind2String(element.getRowKind()));
+               return jsonSerializer.serialize(reuse);
+       }
+
+       private StringData rowKind2String(RowKind rowKind) {
+               switch (rowKind) {
+                       case INSERT:
+                       case UPDATE_AFTER:
+                               return OP_INSERT;
+                       case UPDATE_BEFORE:
+                       case DELETE:
+                               return OP_DELETE;
+                       default:
+                               throw new 
UnsupportedOperationException("Unsupported operation '" + rowKind + "' for row 
kind.");
+               }
+       }
+
+       @Override
+       public boolean equals(Object o) {
+               if (this == o) {
+                       return true;
+               }
+               if (o == null || getClass() != o.getClass()) {
+                       return false;
+               }
+               MaxwellJsonSerializationSchema that = 
(MaxwellJsonSerializationSchema) o;
+               return Objects.equals(jsonSerializer, that.jsonSerializer) &&
+                       timestampFormat == that.timestampFormat;
+       }
+
+       @Override
+       public int hashCode() {
+               return Objects.hash(jsonSerializer, timestampFormat);
+       }
+
+       private RowType createJsonRowType(DataType databaseSchema) {
+               DataType payload = DataTypes.ROW(
+                       DataTypes.FIELD("data", databaseSchema),
+                       DataTypes.FIELD("old", databaseSchema),

Review comment:
       From the `maxwell-data.txt`, it seems that insert and delete doesn't 
produce `old` field. Shall we remove this?




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

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


Reply via email to