You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/12/21 08:02:48 UTC

[GitHub] [flink] fsk119 commented on a diff in pull request #21523: [FLINK-30449][sql-gateway] Introduce ResultInfo to improve the serialization and deserialization of FetchResultsResponseBody

fsk119 commented on code in PR #21523:
URL: https://github.com/apache/flink/pull/21523#discussion_r1054071300


##########
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/serde/ResultInfo.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.gateway.rest.serde;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.gateway.api.results.ResultSet;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * A {@code ResultInfo} contains information of a {@link ResultSet}. It is designed for transferring
+ * the information of ResultSet via REST. For its serialization and deserialization, See:
+ *
+ * <p>{@link ResultInfoJsonSerializer} and {@link ResultInfoJsonDeserializer}
+ */
+@Internal
+public class ResultInfo {
+
+    public static final String FIELD_NAME_COLUMN_INFOS = "columns";
+
+    public static final String FIELD_NAME_ROW_DATA_INFOS = "data";
+
+    private final List<ColumnInfo> columnInfos;
+
+    private final List<RowDataInfo> rowDataInfos;

Review Comment:
   I think we can use List<Row> to prevent introducing a new data structure.



##########
flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/rest/serde/ResultInfoJsonSerDeTest.java:
##########
@@ -72,103 +73,110 @@
 import static org.apache.flink.table.api.DataTypes.TINYINT;
 import static org.assertj.core.api.Assertions.assertThat;
 
-/** Tests for {@link JsonResultSetSerializer} and {@link JsonResultSetDeserializer}. */
-class JsonResultSetSerDeTest {
+/** Tests for {@link ResultInfoJsonSerializer} and {@link ResultInfoJsonDeserializer}. */
+public class ResultInfoJsonSerDeTest {
 
-    private static final byte tinyint = 'c';
-    private static final short smallint = 128;
-    private static final int intValue = 45536;
-    private static final float floatValue = 33.333F;
-    private static final long bigint = 1238123899121L;
-    private static final String name = "asdlkjasjkdla998y1122";
-    private static final byte[] bytes = new byte[1024];
-    private static final Double[] doubles = new Double[] {1.1, 2.2, 3.3};
-    private static final BigDecimal decimal = new BigDecimal("123.456789");
-    private static final LocalDate date = LocalDate.parse("1990-10-14");
-    private static final LocalTime time = LocalTime.parse("12:12:43");
-    private static final Timestamp timestamp3 = Timestamp.valueOf("1990-10-14 12:12:43.123");
-    private static final Timestamp timestamp9 = Timestamp.valueOf("1990-10-14 12:12:43.123456789");
-    private static final Instant timestampWithLocalZone =
+    private final byte tinyint = 'c';
+    private final short smallint = 128;
+    private final int intValue = 45536;
+    private final float floatValue = 33.333F;
+    private final long bigint = 1238123899121L;
+    private final String name = "asdlkjasjkdla998y1122";
+    private static final byte[] BYTES = new byte[1024];

Review Comment:
   why this is not object-level?



##########
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/serde/ResultInfo.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.gateway.rest.serde;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.gateway.api.results.ResultSet;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * A {@code ResultInfo} contains information of a {@link ResultSet}. It is designed for transferring
+ * the information of ResultSet via REST. For its serialization and deserialization, See:
+ *
+ * <p>{@link ResultInfoJsonSerializer} and {@link ResultInfoJsonDeserializer}
+ */
+@Internal
+public class ResultInfo {
+
+    public static final String FIELD_NAME_COLUMN_INFOS = "columns";
+
+    public static final String FIELD_NAME_ROW_DATA_INFOS = "data";
+
+    private final List<ColumnInfo> columnInfos;
+
+    private final List<RowDataInfo> rowDataInfos;
+
+    public ResultInfo(List<ColumnInfo> columnInfos, List<RowDataInfo> rowDataInfos) {
+        this.columnInfos = columnInfos;
+        this.rowDataInfos = rowDataInfos;
+    }
+
+    public List<ColumnInfo> getColumnInfos() {
+        return columnInfos;

Review Comment:
   Collections.unmodifiedList



##########
flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/rest/serde/ResultInfoJsonSerDeTest.java:
##########
@@ -72,103 +73,110 @@
 import static org.apache.flink.table.api.DataTypes.TINYINT;
 import static org.assertj.core.api.Assertions.assertThat;
 
-/** Tests for {@link JsonResultSetSerializer} and {@link JsonResultSetDeserializer}. */
-class JsonResultSetSerDeTest {
+/** Tests for {@link ResultInfoJsonSerializer} and {@link ResultInfoJsonDeserializer}. */

Review Comment:
   Add one more test case: read json from text as the input and the output should be the same with serde.



##########
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/serde/ResultInfoJsonSerializer.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.gateway.rest.serde;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.formats.common.TimestampFormat;
+import org.apache.flink.formats.json.JsonFormatOptions;
+import org.apache.flink.formats.json.RowDataToJsonConverters;
+import org.apache.flink.table.types.DataType;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.SerializerProvider;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ser.std.StdSerializer;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.apache.flink.table.gateway.rest.serde.ResultInfo.FIELD_NAME_COLUMN_INFOS;
+import static org.apache.flink.table.gateway.rest.serde.ResultInfo.FIELD_NAME_ROW_DATA_INFOS;
+import static org.apache.flink.table.gateway.rest.serde.RowDataInfo.FIELD_NAME_FIELDS;
+import static org.apache.flink.table.gateway.rest.serde.RowDataInfo.FIELD_NAME_KIND;
+
+/**
+ * Json serializer for {@link ResultInfo}.
+ *
+ * @see ResultInfoJsonDeserializer for the reverse operation.
+ */
+@Internal
+public class ResultInfoJsonSerializer extends StdSerializer<ResultInfo> {
+
+    private static final long serialVersionUID = 1L;
+
+    public ResultInfoJsonSerializer() {
+        super(ResultInfo.class);
+    }
+
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+    private static final RowDataToJsonConverters TO_JSON_CONVERTERS =
+            new RowDataToJsonConverters(
+                    TimestampFormat.ISO_8601, JsonFormatOptions.MapNullKeyMode.LITERAL, "");
+
+    @Override
+    public void serialize(
+            ResultInfo resultInfo,
+            JsonGenerator jsonGenerator,
+            SerializerProvider serializerProvider)
+            throws IOException {
+        jsonGenerator.writeStartObject();
+
+        // serialize ColumnInfos
+        serializerProvider.defaultSerializeField(
+                FIELD_NAME_COLUMN_INFOS, resultInfo.getColumnInfos(), jsonGenerator);
+
+        // serialize RowDataInfos as format:
+        // data: [{"kind": "", "fields": []}],
+        ArrayNode rowDataInfoArrayNode = OBJECT_MAPPER.createArrayNode();
+
+        // generate converters for all fields of each row
+        List<RowDataToJsonConverters.RowDataToJsonConverter> converters =
+                resultInfo.buildResultSchema().getColumnDataTypes().stream()
+                        .map(DataType::getLogicalType)
+                        .map(TO_JSON_CONVERTERS::createConverter)
+                        .collect(Collectors.toList());
+
+        // construct all element nodes for rowDataInfoArrayNode
+        List<ObjectNode> elementNodes =
+                resultInfo.getRowDataInfos().stream()
+                        .map(
+                                rowDataInfo -> {
+                                    ObjectNode elementNode = OBJECT_MAPPER.createObjectNode();
+                                    // kind
+                                    elementNode.put(FIELD_NAME_KIND, rowDataInfo.getKind());
+                                    // fields
+                                    ArrayNode fieldsArrayNode =
+                                            elementNode.putArray(FIELD_NAME_FIELDS);
+
+                                    List<Object> fields = rowDataInfo.getFields();
+                                    fieldsArrayNode.addAll(
+                                            IntStream.range(0, fields.size())
+                                                    .mapToObj(
+                                                            i ->
+                                                                    converters
+                                                                            .get(i)
+                                                                            .convert(
+                                                                                    OBJECT_MAPPER,
+                                                                                    null,
+                                                                                    fields.get(i)))
+                                                    .collect(Collectors.toList()));
+
+                                    return elementNode;

Review Comment:
   Move this to a single method named serializeRowData



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org