PHILO-HE commented on code in PR #10248:
URL: 
https://github.com/apache/incubator-gluten/pull/10248#discussion_r2235689981


##########
gluten-flink/runtime/src/main/java/org/apache/gluten/table/runtime/config/VeloxQueryConfig.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.gluten.table.runtime.config;
+
+import io.github.zhztheplayer.velox4j.config.Config;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.table.api.config.TableConfigOptions;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class VeloxQueryConfig {
+
+  public static ConfigOption<Boolean> ADJUST_TIMESTMP_TO_SESSION_TIMEZONE =
+      ConfigOptions.key("velox.adjust_timestamp_to_session_timezone")
+          .booleanType()
+          .defaultValue(false)
+          .withDescription(
+              "adjust the timestamp accroding to the given session timezone in 
the velox backend");
+
+  private static final String keyVeloxAdjustTimestampToSessionTimeZone =
+      "adjust_timestamp_to_session_timezone";
+  private static final String keyVeloxSessionTimezone = "session_timezone";
+
+  public static Config getConfig(RuntimeContext context) {
+    if (!(context instanceof StreamingRuntimeContext)) {
+      return Config.empty();
+    }
+    Configuration config = ((StreamingRuntimeContext) 
context).getJobConfiguration();
+    Map<String, String> configMap = new HashMap<>();
+    if (config.get(ADJUST_TIMESTMP_TO_SESSION_TIMEZONE)) {
+      String localTimeZone = config.get(TableConfigOptions.LOCAL_TIME_ZONE);
+      configMap.put(keyVeloxAdjustTimestampToSessionTimeZone, "true");

Review Comment:
   In Spark, we always set this to true to let velox respect configured session 
timezone. I assume we should also do this for Flink, not depending on the 
user's setting for `ADJUST_TIMESTMP_TO_SESSION_TIMEZONE`.



##########
gluten-flink/ut/src/test/java/org/apache/gluten/table/runtime/stream/custom/ScalarFunctionsTest.java:
##########
@@ -169,4 +172,28 @@ void testDecimal() {
     query = "select b + e as x from tblDecimal where a > 0";
     runAndCheck(query, Arrays.asList("+I[2.0]", "+I[5.0]", "+I[7.0]"));
   }
+
+  @Test
+  void testDateFormat() {
+    List<Row> rows =
+        Arrays.asList(Row.of(1, "2024-12-31 12:12:12"), Row.of(2, "2025-02-28 
12:12:12"));
+    createSimpleBoundedValuesTable("dateFormatTbl", "a int, b string", rows);
+    String query =
+        "select a, DATE_FORMAT(cast(b as Timestamp(3)), 'yyyy-MM-dd'), 
DATE_FORMAT(cast(b as Timestamp(3)), 'yyyy-MM-dd HH:mm:ss') from dateFormatTbl";
+    runAndCheck(
+        query,
+        Arrays.asList(
+            "+I[1, 2024-12-31, 2024-12-31 12:12:12]", "+I[2, 2025-02-28, 
2025-02-28 12:12:12]"));
+    Map<String, String> configs =
+        Map.of(
+            VeloxQueryConfig.ADJUST_TIMESTMP_TO_SESSION_TIMEZONE.key(),
+            "true",
+            TableConfigOptions.LOCAL_TIME_ZONE.key(),
+            "America/Los_Angeles");

Review Comment:
   In this designed test, the result cannot reflect the timezone is changed. As 
I mentioned, both cast & date_format are respecting the timezone. As long as 
the timezone they are using is same, the result aligns with the input date time 
string, even though `America/Los_Angeles` is not passed to velox but the 
default timezone is in use. So the test here cannot confirm this new timezone 
is passed to Velox and taking effect.
   
   I haven't come up with a good way. Do you have any idea?



-- 
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: [email protected]

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


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to