You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/07/09 02:24:43 UTC

[GitHub] [arrow] kou commented on a diff in pull request #13428: ARROW-16918: [Gandiva][C++] Adding UTC-local timezone conversion functions

kou commented on code in PR #13428:
URL: https://github.com/apache/arrow/pull/13428#discussion_r917208665


##########
cpp/src/gandiva/gdv_function_stubs.cc:
##########
@@ -611,6 +611,51 @@ int32_t gdv_fn_cast_intervalyear_utf8_int32(int64_t context_ptr, int64_t holder_
   auto* holder = reinterpret_cast<gandiva::IntervalYearsHolder*>(holder_ptr);
   return (*holder)(context, data, data_len, in1_validity, out_valid);
 }
+
+GANDIVA_EXPORT
+gdv_timestamp to_utc_timezone_timestamp(int64_t context, gdv_timestamp time_miliseconds,
+                                        const char* timezone, gdv_int32 length) {
+  using std::chrono::milliseconds;
+  using arrow_vendored::date::sys_time;
+  using arrow_vendored::date::time_zone;
+  using arrow_vendored::date::zoned_time;
+  using arrow_vendored::date::locate_zone;
+
+  sys_time <milliseconds> tp {milliseconds{time_miliseconds}};
+  try {
+    const time_zone* local_tz = locate_zone(std::string(timezone, length));

Review Comment:
   ```suggestion
       const auto local_tz = locate_zone(std::string(timezone, length));
   ```



##########
cpp/src/gandiva/gdv_function_stubs_test.cc:
##########
@@ -993,4 +993,57 @@ TEST(TestGdvFnStubs, TestTranslate) {
   EXPECT_EQ(expected, std::string(result, out_len));
 }
 
+TEST(TestTime, TestToUtcTimezone) {
+  ExecutionContext context;
+  auto context_ptr = reinterpret_cast<int64_t>(&context);
+  gdv_int32 len_ist = static_cast<gdv_int32>(strlen("Asia/Kolkata"));
+  gdv_int32 len_pst = static_cast<gdv_int32>(strlen("America/Los_Angeles"));
+
+  //2012-02-28 15:30:00 Asia/Kolkata
+  gdv_timestamp ts = 55800000;
+  gdv_timestamp ts2 = to_utc_timezone_timestamp(context_ptr, ts,
+                                                "Asia/Kolkata", len_ist);
+  EXPECT_EQ(36000000, ts2);
+
+  //1970-01-01 5:00:00 Asia/Kolkata
+  ts = 18000000;
+  ts2 = to_utc_timezone_timestamp(context_ptr, ts, "Asia/Kolkata", len_ist);
+  EXPECT_EQ(ts2, -1800000);
+
+  //daylight savings check
+  //2018-03-11 01:00:00 America/Los_Angeles
+  ts = 	1520730000000;
+  ts2 = to_utc_timezone_timestamp(context_ptr, ts, "America/Los_Angeles", len_pst);
+  EXPECT_EQ(ts2, 1520758800000);
+
+  //2018-03-12 01:00:00 America/Los_Angeles
+  ts = 1331712000000;
+  ts2 = to_utc_timezone_timestamp(context_ptr, ts, "America/Los_Angeles", len_pst);
+  EXPECT_EQ(ts2, 1331737200000);

Review Comment:
   How about adding a test for failure case too?



##########
cpp/src/gandiva/precompiled/time.cc:
##########
@@ -1055,4 +1055,5 @@ gdv_int32 datediff_timestamp_timestamp(gdv_timestamp start_millis,
 CAST_NULLABLE_INTERVAL_YEAR(int32)
 CAST_NULLABLE_INTERVAL_YEAR(int64)
 
+

Review Comment:
   ```suggestion
   ```



##########
cpp/src/gandiva/gdv_function_stubs.cc:
##########
@@ -611,6 +611,51 @@ int32_t gdv_fn_cast_intervalyear_utf8_int32(int64_t context_ptr, int64_t holder_
   auto* holder = reinterpret_cast<gandiva::IntervalYearsHolder*>(holder_ptr);
   return (*holder)(context, data, data_len, in1_validity, out_valid);
 }
+
+GANDIVA_EXPORT
+gdv_timestamp to_utc_timezone_timestamp(int64_t context, gdv_timestamp time_miliseconds,
+                                        const char* timezone, gdv_int32 length) {
+  using std::chrono::milliseconds;
+  using arrow_vendored::date::sys_time;
+  using arrow_vendored::date::time_zone;
+  using arrow_vendored::date::zoned_time;
+  using arrow_vendored::date::locate_zone;
+
+  sys_time <milliseconds> tp {milliseconds{time_miliseconds}};
+  try {
+    const time_zone* local_tz = locate_zone(std::string(timezone, length));
+    gdv_timestamp offset = local_tz->get_info(tp).offset.count()*1000;
+    return time_miliseconds - static_cast<gdv_timestamp>(offset);
+  } catch(...) {
+      gdv_fn_context_set_error_msg(context, "Invalid time zone");
+      return 0;

Review Comment:
   ```suggestion
       gdv_fn_context_set_error_msg(context, "Invalid time zone");
       return 0;
   ```



##########
cpp/src/gandiva/function_registry_datetime.cc:
##########
@@ -163,11 +163,19 @@ std::vector<NativeFunction> GetDateTimeFunctionRegistry() {
       NativeFunction("datediff", {}, DataTypeVector{timestamp(), timestamp()}, int32(),
                      kResultNullIfNull, "datediff_timestamp_timestamp"),
 
+      NativeFunction("to_utc_timestamp", {}, DataTypeVector{timestamp(), utf8()},
+                     timestamp(), kResultNullIfNull, "to_utc_timezone_timestamp",
+                     NativeFunction::kNeedsContext),
+
+      NativeFunction("from_utc_timestamp", {}, DataTypeVector{timestamp(), utf8()},
+                     timestamp(), kResultNullIfNull, "from_utc_timezone_timestamp",
+                     NativeFunction::kNeedsContext),
+
       DATE_TYPES(LAST_DAY_SAFE_NULL_IF_NULL, last_day, {}),
       BASE_NUMERIC_TYPES(TO_TIME_SAFE_NULL_IF_NULL, to_time, {}),
       BASE_NUMERIC_TYPES(TO_TIMESTAMP_SAFE_NULL_IF_NULL, to_timestamp, {})};
 
   return date_time_fn_registry_;
 }
 
-}  // namespace gandiva
+}  // namespace gandiva

Review Comment:
   Could you revert this change before we merge this?



##########
cpp/src/gandiva/precompiled/types.h:
##########
@@ -433,6 +433,7 @@ gdv_time32 castTIME_utf8(int64_t context, const char* input, int32_t length);
 gdv_time32 castTIME_timestamp(gdv_timestamp timestamp_in_millis);
 gdv_time32 castTIME_int32(int32_t int_val);
 const char* castVARCHAR_timestamp_int64(int64_t, gdv_timestamp, gdv_int64, gdv_int32*);
+

Review Comment:
   ```suggestion
   ```



##########
cpp/src/gandiva/gdv_function_stubs.cc:
##########
@@ -611,6 +611,51 @@ int32_t gdv_fn_cast_intervalyear_utf8_int32(int64_t context_ptr, int64_t holder_
   auto* holder = reinterpret_cast<gandiva::IntervalYearsHolder*>(holder_ptr);
   return (*holder)(context, data, data_len, in1_validity, out_valid);
 }
+
+GANDIVA_EXPORT
+gdv_timestamp to_utc_timezone_timestamp(int64_t context, gdv_timestamp time_miliseconds,
+                                        const char* timezone, gdv_int32 length) {
+  using std::chrono::milliseconds;
+  using arrow_vendored::date::sys_time;
+  using arrow_vendored::date::time_zone;
+  using arrow_vendored::date::zoned_time;
+  using arrow_vendored::date::locate_zone;
+
+  sys_time <milliseconds> tp {milliseconds{time_miliseconds}};
+  try {
+    const time_zone* local_tz = locate_zone(std::string(timezone, length));
+    gdv_timestamp offset = local_tz->get_info(tp).offset.count()*1000;
+    return time_miliseconds - static_cast<gdv_timestamp>(offset);
+  } catch(...) {
+      gdv_fn_context_set_error_msg(context, "Invalid time zone");
+      return 0;
+  }
+}
+
+GANDIVA_EXPORT
+gdv_timestamp from_utc_timezone_timestamp(gdv_int64 context,
+                                          gdv_timestamp time_miliseconds,
+                                          const char* timezone, gdv_int32 length) {
+  using std::chrono::milliseconds;
+  using arrow_vendored::date::sys_time;
+  using arrow_vendored::date::time_zone;
+  using arrow_vendored::date::zoned_time;
+  using arrow_vendored::date::make_zoned;
+
+  sys_time <milliseconds> tp {milliseconds{time_miliseconds}};
+  const zoned_time<milliseconds, const time_zone*> utc_tz =
+                      make_zoned(std::string("Etc/UTC"), tp);
+  try {
+    const zoned_time<milliseconds, const time_zone*> local_tz =
+                      make_zoned(std::string(timezone, length), utc_tz);
+    gdv_timestamp offset = local_tz.get_time_zone()->get_info(tp).offset.count()*1000;
+    return time_miliseconds + static_cast<gdv_timestamp>(offset);
+  } catch(...) {
+      gdv_fn_context_set_error_msg(context, "Invalid time zone");

Review Comment:
   How about adding the given `timezone` to error message? It will help users to fix the invalid input.



##########
cpp/src/gandiva/gdv_function_stubs.cc:
##########
@@ -611,6 +611,51 @@ int32_t gdv_fn_cast_intervalyear_utf8_int32(int64_t context_ptr, int64_t holder_
   auto* holder = reinterpret_cast<gandiva::IntervalYearsHolder*>(holder_ptr);
   return (*holder)(context, data, data_len, in1_validity, out_valid);
 }
+
+GANDIVA_EXPORT
+gdv_timestamp to_utc_timezone_timestamp(int64_t context, gdv_timestamp time_miliseconds,
+                                        const char* timezone, gdv_int32 length) {
+  using std::chrono::milliseconds;
+  using arrow_vendored::date::sys_time;
+  using arrow_vendored::date::time_zone;
+  using arrow_vendored::date::zoned_time;
+  using arrow_vendored::date::locate_zone;
+
+  sys_time <milliseconds> tp {milliseconds{time_miliseconds}};
+  try {
+    const time_zone* local_tz = locate_zone(std::string(timezone, length));
+    gdv_timestamp offset = local_tz->get_info(tp).offset.count()*1000;
+    return time_miliseconds - static_cast<gdv_timestamp>(offset);
+  } catch(...) {
+      gdv_fn_context_set_error_msg(context, "Invalid time zone");
+      return 0;
+  }
+}
+
+GANDIVA_EXPORT
+gdv_timestamp from_utc_timezone_timestamp(gdv_int64 context,
+                                          gdv_timestamp time_miliseconds,
+                                          const char* timezone, gdv_int32 length) {
+  using std::chrono::milliseconds;
+  using arrow_vendored::date::sys_time;
+  using arrow_vendored::date::time_zone;
+  using arrow_vendored::date::zoned_time;
+  using arrow_vendored::date::make_zoned;
+
+  sys_time <milliseconds> tp {milliseconds{time_miliseconds}};
+  const zoned_time<milliseconds, const time_zone*> utc_tz =

Review Comment:
   Can we use `const auto utc_tz = ...` here?



-- 
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: github-unsubscribe@arrow.apache.org

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