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



##########
File path: 
flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/format/single/SingleValueFormatFactory.java
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.format.single;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.format.DecodingFormat;
+import org.apache.flink.table.connector.format.EncodingFormat;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.DeserializationFormatFactory;
+import org.apache.flink.table.factories.DynamicTableFactory.Context;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.SerializationFormatFactory;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+
+import java.util.Collections;
+import java.util.Set;
+
+/**
+ * SingleValueFormatFactory for single value.
+ */
+public class SingleValueFormatFactory implements DeserializationFormatFactory, 
SerializationFormatFactory {
+
+       public static final String IDENTIFIER = "single-value";
+
+       @Override
+       public DecodingFormat<DeserializationSchema<RowData>> 
createDecodingFormat(
+               Context context,
+               ReadableConfig formatOptions) {

Review comment:
       Add additional indent for the parameters. 

##########
File path: 
flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/format/single/SingleValueRowDataDeserialization.java
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.format.single;
+
+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.table.data.GenericRowData;
+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.RowType;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.util.Objects;
+
+/**
+ * Deserialization schema from SINGLE-VALUE to Flink Table/SQL internal data 
structure {@link RowData}.
+ */
+@Internal
+public class SingleValueRowDataDeserialization implements 
DeserializationSchema<RowData> {
+
+       private DeserializationRuntimeConverter converter;
+       private RowType rowType;
+       private TypeInformation<RowData> typeInfo;
+
+       public SingleValueRowDataDeserialization(RowType rowType,
+               TypeInformation<RowData> resultTypeInfo) {

Review comment:
       parameters should be in one line, or each in a new line. 

##########
File path: 
flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/format/single/SingleValueRowSerializerTest.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.format.single;
+
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.RowData.FieldGetter;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.BooleanType;
+import org.apache.flink.table.types.logical.CharType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.SmallIntType;
+import org.apache.flink.table.types.logical.TinyIntType;
+import org.apache.flink.table.types.logical.VarBinaryType;
+import org.apache.flink.table.types.logical.VarCharType;
+
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for {@link SingleValueRowDataDeserialization} {@link 
SingleValueRowDataSerialization}.
+ */
+public class SingleValueRowSerializerTest {
+
+       @Test
+       public void testDeSeSingleValue() throws IOException {

Review comment:
       Please refactor this into `TestSpec` approach which will be much 
cleaner. 
   
   You can take 
`org.apache.flink.formats.json.JsonRowDataSerDeSchemaTest.TestSpec` as an 
example. 

##########
File path: 
flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/format/single/SingleValueRowDataDeserialization.java
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.format.single;
+
+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.table.data.GenericRowData;
+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.RowType;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.util.Objects;
+
+/**
+ * Deserialization schema from SINGLE-VALUE to Flink Table/SQL internal data 
structure {@link RowData}.
+ */
+@Internal
+public class SingleValueRowDataDeserialization implements 
DeserializationSchema<RowData> {
+
+       private DeserializationRuntimeConverter converter;
+       private RowType rowType;
+       private TypeInformation<RowData> typeInfo;
+
+       public SingleValueRowDataDeserialization(RowType rowType,
+               TypeInformation<RowData> resultTypeInfo) {
+               this.rowType = rowType;
+               this.typeInfo = resultTypeInfo;
+               this.converter = createConverter(rowType.getTypeAt(0));
+       }
+
+       @Override
+       public RowData deserialize(byte[] message) throws IOException {
+               GenericRowData genericRowData = new GenericRowData(1);
+               genericRowData.setField(0, converter.convert(message));
+               return genericRowData;
+       }
+
+       @Override
+       public boolean isEndOfStream(RowData nextElement) {
+               return false;
+       }
+
+       @Override
+       public TypeInformation<RowData> getProducedType() {
+               return typeInfo;
+       }
+
+       /**
+        * Runtime converter that convert byte[] to a single value.
+        */
+       @FunctionalInterface
+       private interface DeserializationRuntimeConverter extends Serializable {
+               Object convert(byte[] message);
+       }
+
+       /**
+        *  Creates a runtime converter.
+        */
+       private DeserializationRuntimeConverter createConverter(LogicalType 
type) {
+               switch (type.getTypeRoot()) {
+                       case CHAR:
+                       case VARCHAR:
+                               return bytes -> StringData.fromBytes(bytes);
+                       case VARBINARY:
+                       case BINARY:
+                               return bytes -> bytes;
+                       case TINYINT:
+                               return bytes -> ByteBuffer.wrap(bytes).get();
+                       case SMALLINT:
+                               return bytes -> 
ByteBuffer.wrap(bytes).getShort();
+                       case INTEGER:
+                               return bytes -> ByteBuffer.wrap(bytes).getInt();
+                       case BIGINT:
+                               return bytes -> 
ByteBuffer.wrap(bytes).getLong();
+                       case FLOAT:
+                               return bytes -> 
ByteBuffer.wrap(bytes).getFloat();
+                       case DOUBLE:
+                               return bytes -> 
ByteBuffer.wrap(bytes).getDouble();
+                       case BOOLEAN:
+                               return bytes -> ByteBuffer.wrap(bytes).get() != 
0;

Review comment:
       Do we have more lightweight approach to convert the bytes?

##########
File path: 
flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/format/single/SingleValueFormatFactory.java
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.format.single;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.format.DecodingFormat;
+import org.apache.flink.table.connector.format.EncodingFormat;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.DeserializationFormatFactory;
+import org.apache.flink.table.factories.DynamicTableFactory.Context;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.SerializationFormatFactory;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+
+import java.util.Collections;
+import java.util.Set;
+
+/**
+ * SingleValueFormatFactory for single value.
+ */
+public class SingleValueFormatFactory implements DeserializationFormatFactory, 
SerializationFormatFactory {
+
+       public static final String IDENTIFIER = "single-value";
+
+       @Override
+       public DecodingFormat<DeserializationSchema<RowData>> 
createDecodingFormat(
+               Context context,
+               ReadableConfig formatOptions) {
+               FactoryUtil.validateFactoryOptions(this, formatOptions);
+
+               return new DecodingFormat<DeserializationSchema<RowData>>() {
+                       @Override
+                       public DeserializationSchema<RowData> 
createRuntimeDecoder(
+                               DynamicTableSource.Context context,
+                               DataType producedDataType) {

Review comment:
       ditto.

##########
File path: 
flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/format/single/SingleValue.java
##########
@@ -0,0 +1,45 @@
+/*
+ * 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.format.single;
+
+import org.apache.flink.table.descriptors.DescriptorProperties;
+import org.apache.flink.table.descriptors.FormatDescriptor;
+
+import java.util.Map;
+
+import static 
org.apache.flink.table.format.single.SingleValueValidator.FORMAT_TYPE_VALUE;
+
+/**
+ * Format descriptor for SINGLE-VALUE.
+ */
+public class SingleValue extends FormatDescriptor {

Review comment:
       Remove this. The legacy format descriptor interface can't work with the 
new format factory.

##########
File path: 
flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/format/single/SingleValueRowSerializerTest.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.format.single;
+
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.RowData.FieldGetter;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.BooleanType;
+import org.apache.flink.table.types.logical.CharType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.SmallIntType;
+import org.apache.flink.table.types.logical.TinyIntType;
+import org.apache.flink.table.types.logical.VarBinaryType;
+import org.apache.flink.table.types.logical.VarCharType;
+
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for {@link SingleValueRowDataDeserialization} {@link 
SingleValueRowDataSerialization}.
+ */
+public class SingleValueRowSerializerTest {
+
+       @Test
+       public void testDeSeSingleValue() throws IOException {
+               String testString = "hello,world";
+               byte[] testBytes = testString.getBytes();
+               int testInt = 1024;
+               short testShort = 1024;
+               byte testByte = (byte) 1;
+               float testFloat = 1024.0f;
+               double testDouble = 1024.0d;
+               long testLong = 1024L;
+               boolean testBoolean = false;
+               char testChar = 'T';
+
+               testSeDeSingleValue(getValueBytes(testString), 
StringData.fromString(testString), new VarCharType());

Review comment:
       I think we can use the serializer to convert the Object into byte[], and 
use the deserializer to convert byte[] back to Object, and compare the result 
Object with the original Object. In this way, we don't need the 
`getValueBytes`. 

##########
File path: 
flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/format/single/SingleValueTest.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.format.single;
+
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.descriptors.Descriptor;
+import org.apache.flink.table.descriptors.DescriptorTestBase;
+import org.apache.flink.table.descriptors.DescriptorValidator;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Tests for the {@link SingleValue} descriptor.
+ */
+public class SingleValueTest extends DescriptorTestBase {

Review comment:
       Remove this. The legacy format descriptor interface can't work with the 
new format factory.
   
   

##########
File path: 
flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/format/single/SingleValueValidator.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.format.single;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.descriptors.DescriptorProperties;
+import org.apache.flink.table.descriptors.FormatDescriptorValidator;
+
+/**
+ * Single value format validator.
+ */
+@Internal
+public class SingleValueValidator extends FormatDescriptorValidator {

Review comment:
       Remove this. The legacy format descriptor interface can't work with the 
new format factory.

##########
File path: 
flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/format/single/SingleValueFormatFactory.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.format.single;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.format.DecodingFormat;
+import org.apache.flink.table.connector.format.EncodingFormat;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.DeserializationFormatFactory;
+import org.apache.flink.table.factories.DynamicTableFactory.Context;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.SerializationFormatFactory;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+
+import java.util.Collections;
+import java.util.Set;
+
+/**
+ * SingleValueFormatFactory for single value.
+ */
+public class SingleValueFormatFactory implements DeserializationFormatFactory, 
SerializationFormatFactory {

Review comment:
       Personally, I prefer the built-in formats/connectors are located in api 
moudle, because they are API. What do you think about moving them into 
`org.apache.flink.table.factories` package in the `flink-table-api-java-bridge` 
module.




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