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 2021/07/28 11:05:33 UTC

[GitHub] [arrow] pitrou commented on a change in pull request #10647: ARROW-13174: [C++][Compute] Add strftime kernel

pitrou commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r678180207



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -29,6 +29,7 @@
 #include "arrow/result.h"
 #include "arrow/util/macros.h"
 #include "arrow/util/visibility.h"
+#include "arrow/vendored/datetime.h"

Review comment:
       This shouldn't be exposed here.

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -985,5 +999,17 @@ Result<Datum> Nanosecond(const Datum& values, ExecContext* ctx = NULLPTR);
 /// \note API not yet finalized
 ARROW_EXPORT Result<Datum> Subsecond(const Datum& values, ExecContext* ctx = NULLPTR);
 
+/// \brief Strftime

Review comment:
       Can you make this a proper docstring?

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -656,6 +764,13 @@ void RegisterScalarTemporal(FunctionRegistry* registry) {
 
   auto subsecond = MakeTemporal<Subsecond, DoubleType>("subsecond", &subsecond_doc);
   DCHECK_OK(registry->AddFunction(std::move(subsecond)));
+
+#ifndef _WIN32

Review comment:
       Why not Windows?

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -602,6 +703,13 @@ const FunctionDoc subsecond_doc{
      "Returns an error if timestamp has a defined timezone. Null values return null."),
     {"values"}};
 
+const FunctionDoc strftime_doc{
+    "Convert timestamps to a string representation with an arbitrary format",
+    ("Strftime returns a string representation with an arbitrary format.\n"
+     "The time format string can be set via options."),
+    {"values"},

Review comment:
       "timestamps"?

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -321,6 +323,70 @@ struct Nanosecond {
   }
 };
 
+// ----------------------------------------------------------------------
+// Convert timestamps to a string representation with an arbitrary format
+
+template <typename Duration>
+inline std::string get_timestamp(int64_t arg, const std::locale* locale,
+                                 const time_zone* tz, const std::string* format) {
+  auto zt =
+      arrow_vendored::date::zoned_time<Duration>{tz, sys_time<Duration>(Duration{arg})};
+  return arrow_vendored::date::format(*locale, *format, zt);
+}
+
+template <typename Duration>
+struct Strftime {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const auto& timezone = GetInputTimezone(in);
+    if (timezone.empty()) {
+      return Status::Invalid(
+          "Timestamps without a time zone cannot be reliably printed.");
+    }
+    const arrow_vendored::date::time_zone* tz =
+        arrow_vendored::date::locate_zone(timezone);
+    const StrftimeOptions options = StrftimeState::Get(ctx);
+
+    if (in.is_valid) {
+      const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+      *checked_cast<StringScalar*>(out) = StringScalar(
+          get_timestamp<Duration>(in_val, &options.loc, tz, &options.format));
+    } else {
+      out->is_valid = false;
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    const auto& timezone = GetInputTimezone(in);
+    if (timezone.empty()) {
+      return Status::Invalid("Timestamp without a time zone cannot be reliably printed.");
+    }
+    const arrow_vendored::date::time_zone* tz =
+        arrow_vendored::date::locate_zone(timezone);
+    const StrftimeOptions options = StrftimeState::Get(ctx);
+
+    std::unique_ptr<ArrayBuilder> array_builder;
+    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), utf8(), &array_builder));
+    StringBuilder* string_builder = checked_cast<StringBuilder*>(array_builder.get());

Review comment:
       You can certainly instantiate a local `StringBuilder` directly instead of going through `MakeBuilder` and then casting.

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -321,6 +323,70 @@ struct Nanosecond {
   }
 };
 
+// ----------------------------------------------------------------------
+// Convert timestamps to a string representation with an arbitrary format
+
+template <typename Duration>
+inline std::string get_timestamp(int64_t arg, const std::locale* locale,
+                                 const time_zone* tz, const std::string* format) {
+  auto zt =
+      arrow_vendored::date::zoned_time<Duration>{tz, sys_time<Duration>(Duration{arg})};
+  return arrow_vendored::date::format(*locale, *format, zt);
+}
+
+template <typename Duration>
+struct Strftime {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const auto& timezone = GetInputTimezone(in);
+    if (timezone.empty()) {
+      return Status::Invalid(
+          "Timestamps without a time zone cannot be reliably printed.");
+    }
+    const arrow_vendored::date::time_zone* tz =
+        arrow_vendored::date::locate_zone(timezone);
+    const StrftimeOptions options = StrftimeState::Get(ctx);
+
+    if (in.is_valid) {
+      const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+      *checked_cast<StringScalar*>(out) = StringScalar(
+          get_timestamp<Duration>(in_val, &options.loc, tz, &options.format));
+    } else {
+      out->is_valid = false;
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    const auto& timezone = GetInputTimezone(in);
+    if (timezone.empty()) {
+      return Status::Invalid("Timestamp without a time zone cannot be reliably printed.");
+    }
+    const arrow_vendored::date::time_zone* tz =
+        arrow_vendored::date::locate_zone(timezone);
+    const StrftimeOptions options = StrftimeState::Get(ctx);
+
+    std::unique_ptr<ArrayBuilder> array_builder;
+    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), utf8(), &array_builder));
+    StringBuilder* string_builder = checked_cast<StringBuilder*>(array_builder.get());
+    auto expected_string_size = static_cast<int64_t>(
+        ceil(get_timestamp<Duration>(0, &options.loc, tz, &options.format).size() * 1.1));
+    RETURN_NOT_OK(string_builder->Reserve(in.length * expected_string_size));

Review comment:
       `Reserve` is for the number of elements, so this should be `Reserve(in.length)`.
   For the underlying data, you should call `ReserveData`. Also, you should take nulls into account, so probably `ReserveData((in.length - in.GetNullCount()) * expected_string_size)`

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -178,6 +179,19 @@ class ARROW_EXPORT StrptimeOptions : public FunctionOptions {
   TimeUnit::type unit;
 };
 
+class ARROW_EXPORT StrftimeOptions : public FunctionOptions {
+ public:
+  explicit StrftimeOptions(std::string format, std::string locale);
+  explicit StrftimeOptions(std::string format);
+  StrftimeOptions();
+  constexpr static char const kTypeName[] = "StrftimeOptions";
+
+  /// The desired format string.
+  std::string format;
+  /// The desired output locale.
+  std::locale loc;

Review comment:
       I wouldn't put `std::locate` in the options. Just keep it a `std::string`.

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -321,6 +323,70 @@ struct Nanosecond {
   }
 };
 
+// ----------------------------------------------------------------------
+// Convert timestamps to a string representation with an arbitrary format
+
+template <typename Duration>
+inline std::string get_timestamp(int64_t arg, const std::locale* locale,

Review comment:
       Can you name this something meaningful? `get_timestamp` doesn't seem descriptive at all.
   Also, if this is only used inside `Strftime`, can you make this a `Strftime` method?

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -470,6 +536,41 @@ std::shared_ptr<ScalarFunction> MakeTemporalWithOptions(
   return func;
 }
 
+std::shared_ptr<ScalarFunction> MakeStrftime(std::string name, const FunctionDoc* doc,
+                                             const StrftimeOptions& default_options,
+                                             KernelInit init) {
+  const auto& out_type = utf8();
+  auto func =
+      std::make_shared<ScalarFunction>(name, Arity::Unary(), doc, &default_options);
+
+  for (auto unit : internal::AllTimeUnits()) {
+    InputType in_type{match::TimestampTypeUnit(unit)};
+    switch (unit) {
+      case TimeUnit::SECOND: {
+        auto exec = SimpleUnary<Strftime<std::chrono::seconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec), init));
+        break;
+      }
+      case TimeUnit::MILLI: {
+        auto exec = SimpleUnary<Strftime<std::chrono::milliseconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec), init));
+        break;
+      }
+      case TimeUnit::MICRO: {
+        auto exec = SimpleUnary<Strftime<std::chrono::microseconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec), init));
+        break;
+      }
+      case TimeUnit::NANO: {
+        auto exec = SimpleUnary<Strftime<std::chrono::nanoseconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec), init));
+        break;
+      }
+    }

Review comment:
       This must be the fourth function where we have this exact pattern of switching on a time unit and then instantiating a kernel with `std::chrono::{seconds, etc.}` based on the time unit.
   
   It seems like this should be factored out, can you try to do this?




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