You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ko...@apache.org on 2022/07/20 05:27:37 UTC

[arrow] branch master updated: ARROW-16918: [Gandiva][C++] Adding UTC-local timezone conversion functions (#13428)

This is an automated email from the ASF dual-hosted git repository.

kou pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git


The following commit(s) were added to refs/heads/master by this push:
     new 85c0db7413 ARROW-16918: [Gandiva][C++] Adding UTC-local timezone conversion functions (#13428)
85c0db7413 is described below

commit 85c0db74130394614dfdc41bece179bfd349d0dd
Author: palak-9202 <10...@users.noreply.github.com>
AuthorDate: Wed Jul 20 10:57:28 2022 +0530

    ARROW-16918: [Gandiva][C++] Adding UTC-local timezone conversion functions (#13428)
    
    Adding functions
    to_utc_timezone : Converts a timestamp from local timezone to UTC time
    from_utc_timezone : Converts a timestamp from UTC time to local time
    
    Lead-authored-by: Palak Pariawala <pa...@dremio.com>
    Co-authored-by: Sutou Kouhei <ko...@clear-code.com>
    Signed-off-by: Sutou Kouhei <ko...@clear-code.com>
---
 cpp/src/gandiva/function_registry_datetime.cc |   8 ++
 cpp/src/gandiva/gdv_function_stubs.cc         |  63 +++++++++++++++
 cpp/src/gandiva/gdv_function_stubs.h          |   8 ++
 cpp/src/gandiva/gdv_function_stubs_test.cc    |  72 +++++++++++++++++
 cpp/src/gandiva/tests/date_time_test.cc       | 108 ++++++++++++++++++++++++++
 5 files changed, 259 insertions(+)

diff --git a/cpp/src/gandiva/function_registry_datetime.cc b/cpp/src/gandiva/function_registry_datetime.cc
index a890c6458d..a2fe75c80f 100644
--- a/cpp/src/gandiva/function_registry_datetime.cc
+++ b/cpp/src/gandiva/function_registry_datetime.cc
@@ -163,6 +163,14 @@ 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, {})};
diff --git a/cpp/src/gandiva/gdv_function_stubs.cc b/cpp/src/gandiva/gdv_function_stubs.cc
index 27900c14af..f857ca6794 100644
--- a/cpp/src/gandiva/gdv_function_stubs.cc
+++ b/cpp/src/gandiva/gdv_function_stubs.cc
@@ -611,6 +611,46 @@ 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 arrow_vendored::date::locate_zone;
+  using arrow_vendored::date::sys_time;
+  using std::chrono::milliseconds;
+
+  sys_time<milliseconds> tp{milliseconds{time_miliseconds}};
+  try {
+    const auto 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 (...) {
+    std::string e_msg = std::string(timezone, length) + " is an invalid time zone name.";
+    gdv_fn_context_set_error_msg(context, e_msg.c_str());
+    return 0;
+  }
+}
+
+GANDIVA_EXPORT
+gdv_timestamp from_utc_timezone_timestamp(gdv_int64 context,
+                                          gdv_timestamp time_miliseconds,
+                                          const char* timezone, gdv_int32 length) {
+  using arrow_vendored::date::make_zoned;
+  using arrow_vendored::date::sys_time;
+  using std::chrono::milliseconds;
+
+  sys_time<milliseconds> tp{milliseconds{time_miliseconds}};
+  const auto utc_tz = make_zoned(std::string("Etc/UTC"), tp);
+  try {
+    const auto 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 (...) {
+    std::string e_msg = std::string(timezone, length) + " is an invalid time zone name.";
+    gdv_fn_context_set_error_msg(context, e_msg.c_str());
+    return 0;
+  }
+}
 }
 
 namespace gandiva {
@@ -962,5 +1002,28 @@ void ExportedStubFunctions::AddMappings(Engine* engine) const {
   engine->AddGlobalMappingForFunc(
       "gdv_fn_cast_intervalyear_utf8_int32", types->i32_type() /*return_type*/, args,
       reinterpret_cast<void*>(gdv_fn_cast_intervalyear_utf8_int32));
+
+  // to_utc_timezone_timestamp
+  args = {
+      types->i64_type(),     // context
+      types->i64_type(),     // timestamp
+      types->i8_ptr_type(),  // timezone
+      types->i32_type()      // length
+  };
+
+  engine->AddGlobalMappingForFunc("to_utc_timezone_timestamp",
+                                  types->i64_type() /*return_type*/, args,
+                                  reinterpret_cast<void*>(to_utc_timezone_timestamp));
+  // from_utc_timezone_timestamp
+  args = {
+      types->i64_type(),     // context
+      types->i64_type(),     // timestamp
+      types->i8_ptr_type(),  // timezone
+      types->i32_type()      // length
+  };
+
+  engine->AddGlobalMappingForFunc("from_utc_timezone_timestamp",
+                                  types->i64_type() /*return_type*/, args,
+                                  reinterpret_cast<void*>(from_utc_timezone_timestamp));
 }
 }  // namespace gandiva
diff --git a/cpp/src/gandiva/gdv_function_stubs.h b/cpp/src/gandiva/gdv_function_stubs.h
index 265d48fb72..c89720aa8c 100644
--- a/cpp/src/gandiva/gdv_function_stubs.h
+++ b/cpp/src/gandiva/gdv_function_stubs.h
@@ -338,4 +338,12 @@ GANDIVA_EXPORT
 const char* translate_utf8_utf8_utf8(int64_t context, const char* in, int32_t in_len,
                                      const char* from, int32_t from_len, const char* to,
                                      int32_t to_len, int32_t* out_len);
+
+GANDIVA_EXPORT
+gdv_timestamp to_utc_timezone_timestamp(int64_t context, gdv_timestamp time_miliseconds,
+                                        const char* timezone, int32_t length);
+
+GANDIVA_EXPORT
+gdv_timestamp from_utc_timezone_timestamp(int64_t context, gdv_timestamp time_miliseconds,
+                                          const char* timezone, int32_t length);
 }
diff --git a/cpp/src/gandiva/gdv_function_stubs_test.cc b/cpp/src/gandiva/gdv_function_stubs_test.cc
index 7f282c172b..f35e25d5ad 100644
--- a/cpp/src/gandiva/gdv_function_stubs_test.cc
+++ b/cpp/src/gandiva/gdv_function_stubs_test.cc
@@ -993,4 +993,76 @@ TEST(TestGdvFnStubs, TestTranslate) {
   EXPECT_EQ(expected, std::string(result, out_len));
 }
 
+TEST(TestGdvFnStubs, TestToUtcTimezone) {
+  gandiva::ExecutionContext context;
+  auto context_ptr = reinterpret_cast<int64_t>(&context);
+  auto len_ist = static_cast<gdv_int32>(strlen("Asia/Kolkata"));
+  auto len_pst = static_cast<gdv_int32>(strlen("America/Los_Angeles"));
+
+  // ts: 2012-02-28 15:30:00
+  // ts2:2012-02-28 10:00:00
+  gdv_timestamp ts = 1330443000000;
+  gdv_timestamp ts2 = to_utc_timezone_timestamp(context_ptr, ts, "Asia/Kolkata", len_ist);
+  EXPECT_EQ(1330423200000, ts2);
+
+  // ts: 1970-01-01 05:00:00
+  // ts2:1969-12-31 23:30:00
+  ts = 18000000;
+  ts2 = to_utc_timezone_timestamp(context_ptr, ts, "Asia/Kolkata", len_ist);
+  EXPECT_EQ(ts2, -1800000);
+
+  // daylight savings check
+  // ts: 2018-03-11 01:00:00
+  // ts2:2018-03-11 09:00:00
+  ts = 1520730000000;
+  ts2 = to_utc_timezone_timestamp(context_ptr, ts, "America/Los_Angeles", len_pst);
+  EXPECT_EQ(ts2, 1520758800000);
+
+  // ts: 2018-03-12 01:00:00
+  // ts2:2018-03-12 08:00:00
+  ts = 1520816400000;
+  ts2 = to_utc_timezone_timestamp(context_ptr, ts, "America/Los_Angeles", len_pst);
+  EXPECT_EQ(ts2, 1520841600000);
+
+  // Failure case
+  ts2 = to_utc_timezone_timestamp(context_ptr, ts, "America/LA", 10);
+  EXPECT_THAT(context.get_error(), "America/LA is an invalid time zone name.");
+}
+
+TEST(TestGdvFnStubs, TestFromUtcTimezone) {
+  ExecutionContext context;
+  auto context_ptr = reinterpret_cast<int64_t>(&context);
+  auto len_ist = static_cast<gdv_int32>(strlen("Asia/Kolkata"));
+  auto len_pst = static_cast<gdv_int32>(strlen("America/Los_Angeles"));
+
+  // ts: 1970-01-01 10:00:00
+  // ts2:1970-01-01 15:30:00
+  gdv_timestamp ts = 36000000;
+  gdv_timestamp ts2 =
+      from_utc_timezone_timestamp(context_ptr, ts, "Asia/Kolkata", len_ist);
+  EXPECT_EQ(ts2, 55800000);
+
+  // ts: 1969-12-31 23:30:00
+  // ts2:1970-01-01 05:00:00
+  ts = -1800000;
+  ts2 = from_utc_timezone_timestamp(context_ptr, ts, "Asia/Kolkata", len_ist);
+  EXPECT_EQ(ts2, 18000000);
+
+  // ts: 2018-03-11 09:00:00
+  // ts2:2018-03-11 01:00:00
+  ts = 1520758800000;
+  ts2 = from_utc_timezone_timestamp(context_ptr, ts, "America/Los_Angeles", len_pst);
+  EXPECT_EQ(ts2, 1520730000000);
+
+  // ts: 2018-03-12 08:00:00
+  // ts2:2018-03-12 01:00:00
+  ts = 1520841600000;
+  ts2 = from_utc_timezone_timestamp(context_ptr, ts, "America/Los_Angeles", len_pst);
+  EXPECT_EQ(ts2, 1520816400000);
+
+  // Failure case
+  ts2 = from_utc_timezone_timestamp(context_ptr, ts, "India", 5);
+  EXPECT_THAT(context.get_error(), "India is an invalid time zone name.");
+}
+
 }  // namespace gandiva
diff --git a/cpp/src/gandiva/tests/date_time_test.cc b/cpp/src/gandiva/tests/date_time_test.cc
index 4751f98414..ce1c3d05f6 100644
--- a/cpp/src/gandiva/tests/date_time_test.cc
+++ b/cpp/src/gandiva/tests/date_time_test.cc
@@ -720,4 +720,112 @@ TEST_F(TestProjector, TestToTimestampFromInt) {
   EXPECT_ARROW_ARRAY_EQUALS(exp_output1, outputs.at(2));
   EXPECT_ARROW_ARRAY_EQUALS(exp_output1, outputs.at(3));
 }
+
+TEST_F(TestProjector, TestToUtcTimestamp) {
+  auto f0 = field("f0", timestamp(arrow::TimeUnit::MILLI));
+  auto f1 = field("f1", arrow::utf8());
+
+  auto schema = arrow::schema({f0, f1});
+
+  // output fields
+  auto utc_timestamp = field("utc_time", timestamp(arrow::TimeUnit::MILLI));
+
+  auto utc_time_expr =
+      TreeExprBuilder::MakeExpression("to_utc_timestamp", {f0, f1}, utc_timestamp);
+  std::shared_ptr<Projector> projector;
+  Status status =
+      Projector::Make(schema, {utc_time_expr}, TestConfiguration(), &projector);
+
+  ASSERT_TRUE(status.ok());
+
+  time_t epoch = Epoch();
+
+  // Create a row-batch with some sample data
+  std::vector<int64_t> f0_data = {MillisSince(epoch, 1970, 1, 1, 6, 0, 0, 0),
+                                  MillisSince(epoch, 2001, 1, 5, 3, 0, 0, 0),
+                                  MillisSince(epoch, 2018, 3, 12, 1, 0, 0, 0),
+                                  MillisSince(epoch, 2018, 3, 11, 1, 0, 0, 0)};
+  int64_t num_records = f0_data.size();
+  std::vector<bool> validity(num_records, true);
+  auto array0 = MakeArrowTypeArray<arrow::TimestampType, int64_t>(
+      arrow::timestamp(arrow::TimeUnit::MILLI), f0_data, validity);
+
+  auto array1 = MakeArrowArrayUtf8(
+      {"Asia/Kolkata", "Asia/Kolkata", "America/Los_Angeles", "America/Los_Angeles"},
+      {true, true, true, true});
+
+  // expected output
+  std::vector<int64_t> exp_output_data = {MillisSince(epoch, 1970, 1, 1, 0, 30, 0, 0),
+                                          MillisSince(epoch, 2001, 1, 4, 21, 30, 0, 0),
+                                          MillisSince(epoch, 2018, 3, 12, 8, 0, 0, 0),
+                                          MillisSince(epoch, 2018, 3, 11, 9, 0, 0, 0)};
+  auto exp_output = MakeArrowTypeArray<arrow::TimestampType, int64_t>(
+      arrow::timestamp(arrow::TimeUnit::MILLI), exp_output_data, validity);
+
+  // prepare input record batch
+  auto in_batch = arrow::RecordBatch::Make(schema, num_records, {array0, array1});
+
+  // Evaluate expression
+  arrow::ArrayVector outputs;
+  status = projector->Evaluate(*in_batch, pool_, &outputs);
+  EXPECT_TRUE(status.ok());
+
+  // Validate results
+
+  EXPECT_ARROW_ARRAY_EQUALS(exp_output, outputs.at(0));
+}
+
+TEST_F(TestProjector, TestFromUtcTimestamp) {
+  auto f0 = field("f0", timestamp(arrow::TimeUnit::MILLI));
+  auto f1 = field("f1", arrow::utf8());
+
+  auto schema = arrow::schema({f0, f1});
+
+  // output fields
+  auto local_timestamp = field("local_time", timestamp(arrow::TimeUnit::MILLI));
+
+  auto local_time_expr =
+      TreeExprBuilder::MakeExpression("from_utc_timestamp", {f0, f1}, local_timestamp);
+  std::shared_ptr<Projector> projector;
+  Status status =
+      Projector::Make(schema, {local_time_expr}, TestConfiguration(), &projector);
+
+  ASSERT_TRUE(status.ok());
+
+  time_t epoch = Epoch();
+
+  // Create a row-batch with some sample data
+  std::vector<int64_t> f0_data = {MillisSince(epoch, 1970, 1, 1, 0, 30, 0, 0),
+                                  MillisSince(epoch, 2001, 1, 4, 21, 30, 0, 0),
+                                  MillisSince(epoch, 2018, 3, 12, 8, 0, 0, 0),
+                                  MillisSince(epoch, 2018, 3, 11, 9, 0, 0, 0)};
+
+  int64_t num_records = f0_data.size();
+  std::vector<bool> validity(num_records, true);
+  auto array0 = MakeArrowTypeArray<arrow::TimestampType, int64_t>(
+      arrow::timestamp(arrow::TimeUnit::MILLI), f0_data, validity);
+
+  auto array1 = MakeArrowArrayUtf8(
+      {"Asia/Kolkata", "Asia/Kolkata", "America/Los_Angeles", "America/Los_Angeles"},
+      {true, true, true, true});
+
+  // expected output
+  std::vector<int64_t> exp_output_data = {MillisSince(epoch, 1970, 1, 1, 6, 0, 0, 0),
+                                          MillisSince(epoch, 2001, 1, 5, 3, 0, 0, 0),
+                                          MillisSince(epoch, 2018, 3, 12, 1, 0, 0, 0),
+                                          MillisSince(epoch, 2018, 3, 11, 1, 0, 0, 0)};
+  auto exp_output = MakeArrowTypeArray<arrow::TimestampType, int64_t>(
+      arrow::timestamp(arrow::TimeUnit::MILLI), exp_output_data, validity);
+
+  // prepare input record batch
+  auto in_batch = arrow::RecordBatch::Make(schema, num_records, {array0, array1});
+
+  // Evaluate expression
+  arrow::ArrayVector outputs;
+  status = projector->Evaluate(*in_batch, pool_, &outputs);
+  EXPECT_TRUE(status.ok());
+
+  // Validate results
+  EXPECT_ARROW_ARRAY_EQUALS(exp_output, outputs.at(0));
+}
 }  // namespace gandiva