You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "libenchao (via GitHub)" <gi...@apache.org> on 2023/09/04 05:17:23 UTC

[GitHub] [flink] libenchao commented on a diff in pull request #22832: [FLINK-32396][jdbc-driver] Support timestamp and timestamp_ltz for jdbc driver and sql gateway

libenchao commented on code in PR #22832:
URL: https://github.com/apache/flink/pull/22832#discussion_r1314446016


##########
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/serde/ResultInfo.java:
##########
@@ -64,12 +67,21 @@ public class ResultInfo {
         this.rowFormat = rowFormat;
     }
 
-    public static ResultInfo createResultInfo(ResultSet resultSet, RowFormat rowFormat) {
+    public static ResultInfo createResultInfo(
+            ResultSet resultSet,
+            RowFormat rowFormat,
+            @Nullable RowDataLocalTimeZoneConverter timeZoneConverter) {
         Preconditions.checkArgument(resultSet.getResultType() != ResultSet.ResultType.NOT_READY);
         List<RowData> data = resultSet.getData();
 
         switch (rowFormat) {
             case JSON:
+                if (timeZoneConverter != null && timeZoneConverter.hasTimeZoneData()) {

Review Comment:
   Do we need to consider other row formats such as `PLAIN_TEXT`?



##########
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/util/RowDataLocalTimeZoneConverter.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.util;
+
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.table.api.config.TableConfigOptions;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LocalZonedTimestampType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.utils.DateTimeUtils;
+
+import java.time.ZoneId;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TimeZone;
+
+/**
+ * Convert {@link LocalZonedTimestampType} data in {@link RowData} to {@link TimestampData} based on
+ * the local time zone.
+ */
+public class RowDataLocalTimeZoneConverter {
+    private final List<RowData.FieldGetter> fieldGetterList;
+    private final List<LogicalType> logicalTypeList;
+    private final boolean hasTimeZoneData;
+    private final TimeZone timeZone;
+
+    public RowDataLocalTimeZoneConverter(List<LogicalType> logicalTypeList, ReadableConfig config) {
+        this(logicalTypeList, TimeZone.getTimeZone(getSessionTimeZone(config)));
+    }
+
+    public RowDataLocalTimeZoneConverter(List<LogicalType> logicalTypeList, TimeZone timeZone) {
+        this.logicalTypeList = logicalTypeList;
+        this.timeZone = timeZone;
+        this.fieldGetterList = new ArrayList<>(logicalTypeList.size());
+        for (int i = 0; i < logicalTypeList.size(); i++) {
+            fieldGetterList.add(RowData.createFieldGetter(logicalTypeList.get(i), i));
+        }
+        this.hasTimeZoneData = checkTimeZoneType(logicalTypeList);
+    }
+
+    private boolean checkTimeZoneType(List<LogicalType> logicalTypeList) {
+        for (LogicalType logicalType : logicalTypeList) {
+            if (logicalType.getTypeRoot() == LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE) {
+                return true;
+            } else if (logicalType.getTypeRoot() == LogicalTypeRoot.MAP) {
+                MapType mapType = (MapType) logicalType;
+                if (checkTimeZoneType(
+                        Arrays.asList(mapType.getKeyType(), mapType.getValueType()))) {
+                    return true;
+                }
+            } else if (logicalType.getTypeRoot() == LogicalTypeRoot.ARRAY) {

Review Comment:
   There are also various type roots such as `MULTISET`, `ROW`, do you think we also need to handle them too?



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