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



##########
File path: 
flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/formats/raw/RawFormatDeserializationSchema.java
##########
@@ -0,0 +1,343 @@
+/*
+ * 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.table.formats.raw;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.DataInputDeserializer;
+import org.apache.flink.core.memory.HeapMemorySegment;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RawType;
+import org.apache.flink.types.DeserializationException;
+import org.apache.flink.util.function.FunctionUtils;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.util.Objects;
+import java.util.function.Consumer;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Deserialization schema from raw (byte based) value to Flink Table/SQL 
internal data structure {@link RowData}.
+ */
+@Internal
+public class RawFormatDeserializationSchema implements 
DeserializationSchema<RowData> {
+
+       private static final long serialVersionUID = 1L;
+
+       private final LogicalType deserializedType;
+
+       private final DeserializationRuntimeConverter converter;
+
+       private final TypeInformation<RowData> producedTypeInfo;
+
+       private final String charsetName;
+
+       private final boolean isBigEndian;
+
+       private transient GenericRowData reuse;
+
+       public RawFormatDeserializationSchema(
+                       LogicalType deserializedType,
+                       TypeInformation<RowData> producedTypeInfo,
+                       String charsetName,
+                       boolean isBigEndian) {
+               this.deserializedType = checkNotNull(deserializedType);
+               this.producedTypeInfo = checkNotNull(producedTypeInfo);
+               this.converter = createConverter(deserializedType, charsetName, 
isBigEndian);
+               this.charsetName = charsetName;
+               this.isBigEndian = isBigEndian;
+       }
+
+       @Override
+       public void open(InitializationContext context) throws Exception {
+               reuse = new GenericRowData(1);
+               converter.open();
+       }
+
+       @Override
+       public RowData deserialize(byte[] message) throws IOException {
+               reuse.setField(0, converter.convert(message));
+               return reuse;
+       }
+
+       @Override
+       public boolean isEndOfStream(RowData nextElement) {
+               return false;
+       }
+
+       @Override
+       public TypeInformation<RowData> getProducedType() {
+               return producedTypeInfo;
+       }
+
+       @Override
+       public boolean equals(Object o) {
+               if (this == o) {
+                       return true;
+               }
+               if (o == null || getClass() != o.getClass()) {
+                       return false;
+               }
+               RawFormatDeserializationSchema that = 
(RawFormatDeserializationSchema) o;
+               return producedTypeInfo.equals(that.producedTypeInfo) &&
+                       deserializedType.equals(that.deserializedType) &&
+                       charsetName.equals(that.charsetName) &&
+                       isBigEndian == that.isBigEndian;
+       }
+
+       @Override
+       public int hashCode() {
+               return Objects.hash(producedTypeInfo, deserializedType, 
charsetName, isBigEndian);
+       }
+
+       // 
------------------------------------------------------------------------
+
+       /**
+        * Runtime converter that convert byte[] to internal data structure 
object.
+        */
+       @FunctionalInterface
+       private interface DeserializationRuntimeConverter extends Serializable {
+
+               default void open() {}
+
+               Object convert(byte[] data) throws IOException;
+       }
+
+       /**
+        * Creates a runtime converter.
+        */
+       private static DeserializationRuntimeConverter createConverter(
+                       LogicalType type, String charsetName, boolean 
isBigEndian) {
+               final DeserializationRuntimeConverter converter = 
createNotNullConverter(type, charsetName, isBigEndian);
+               final Consumer<byte[]> validator = 
createDataLengthValidator(type);
+
+               return new DeserializationRuntimeConverter() {
+                       private static final long serialVersionUID = 1L;
+
+                       @Override
+                       public void open() {
+                               converter.open();
+                       }
+
+                       @Override
+                       public Object convert(byte[] data) throws IOException {
+                               if (data == null) {

Review comment:
       Why? The Kafka message value might be null.




----------------------------------------------------------------
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:
us...@infra.apache.org


Reply via email to