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/02 00:52:30 UTC

[GitHub] [arrow] rok opened a new pull request #10647: ARROW-13174: [C++][Compute] Add strftime kernel

rok opened a new pull request #10647:
URL: https://github.com/apache/arrow/pull/10647


   This is to resolve [ARROW-13174](https://issues.apache.org/jira/browse/ARROW-13174).


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



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

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r668370637



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -602,6 +685,15 @@ 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"
+     "Returns an error if timestamp has a defined timezone.\n"

Review comment:
       What does this mean?  I don't see code anywhere that errors if the timestamp has a timezone but I might just not be seeing it.

##########
File path: cpp/src/arrow/compute/api_scalar.cc
##########
@@ -240,6 +243,15 @@ StrptimeOptions::StrptimeOptions(std::string format, TimeUnit::type unit)
 StrptimeOptions::StrptimeOptions() : StrptimeOptions("", TimeUnit::SECOND) {}
 constexpr char StrptimeOptions::kTypeName[];
 
+StrftimeOptions::StrftimeOptions(std::string format, std::string timezone)
+    : FunctionOptions(internal::kStrftimeOptionsType),
+      format(std::move(format)),
+      timezone(std::move(timezone)) {
+  tz = arrow_vendored::date::locate_zone(this->timezone);
+}
+StrftimeOptions::StrftimeOptions() : StrftimeOptions("%Y-%m-%dT%H:%M:%S", "UTC") {}

Review comment:
       Should the default format string include the trailing `%z` so that it is ISO-8601 compliant?

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -321,6 +322,53 @@ struct Nanosecond {
   }
 };
 
+// ----------------------------------------------------------------------
+// Convert timestamps to a string representation with an arbitrary format
+
+template <typename Duration>
+inline std::string get_timestamp(int64_t arg, const StrftimeOptions* options) {
+  auto zt = arrow_vendored::date::zoned_time<Duration>{options->tz,
+                                                       sys_time<Duration>(Duration{arg})};
+  return arrow_vendored::date::format(options->format, zt.get_local_time());
+}
+
+template <typename Duration>
+struct Strftime {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    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));
+    } else {
+      out->is_valid = false;
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    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());
+    RETURN_NOT_OK(string_builder->Reserve(in.length * 30));

Review comment:
       Nit: How precise is this 30?  Is it based on the default format string?  If I look at 2021-07-12T23:55:41+00:00 I would think you only need ~25, maybe more if subsecond resolution.  Could we base this on the resolution?  It's not a big deal so if it seems like too much don't worry.

##########
File path: python/pyarrow/tests/test_compute.py
##########
@@ -1347,6 +1351,32 @@ def test_strptime():
     assert got == expected
 
 
+# TODO: We should test on windows once ARROW-13168 is resolved.
+@pytest.mark.pandas
+@pytest.mark.skipif(sys.platform == 'win32',
+                    reason="Timezone database is not available on Windows yet")
+def test_strftime():

Review comment:
       Can you add some examples with `%z`/`%Z` on both naive/local timestamps and UTC timestamps.  It looks like python emits an empty string for `%z`/`%Z` if the timestamp is naive but I don't know what the vendored tz lib will do.




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



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

Posted by GitBox <gi...@apache.org>.
jorisvandenbossche commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r670405431



##########
File path: cpp/src/arrow/compute/api_scalar.cc
##########
@@ -240,6 +243,15 @@ StrptimeOptions::StrptimeOptions(std::string format, TimeUnit::type unit)
 StrptimeOptions::StrptimeOptions() : StrptimeOptions("", TimeUnit::SECOND) {}
 constexpr char StrptimeOptions::kTypeName[];
 
+StrftimeOptions::StrftimeOptions(std::string format, std::string timezone)
+    : FunctionOptions(internal::kStrftimeOptionsType),
+      format(std::move(format)),
+      timezone(std::move(timezone)) {
+  tz = arrow_vendored::date::locate_zone(this->timezone);
+}
+StrftimeOptions::StrftimeOptions() : StrftimeOptions("%Y-%m-%dT%H:%M:%S", "UTC") {}

Review comment:
       I don't think that `%z` adds a trailing z as in the ISO format. It adds a numeric offset (eg https://www.cplusplus.com/reference/ctime/strftime/):
   
   ```
   In [44]: datetime.datetime(2012, 1, 1, tzinfo=datetime.timezone.utc).strftime("%Y-%m-%dT%H:%M:%S%z")
   Out[44]: '2012-01-01T00:00:00+0000'
   ```




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-879143370


   Thanks for the review @westonpace!
   I've gone through it and pushed appropriate changes.
   
   One thing I'm not sure about is if we should be using `date.h` for formatting here or would it be better to use system / c++ equivalents.


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



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

Posted by GitBox <gi...@apache.org>.
jorisvandenbossche commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r670406295



##########
File path: cpp/src/arrow/compute/api_scalar.cc
##########
@@ -240,6 +243,15 @@ StrptimeOptions::StrptimeOptions(std::string format, TimeUnit::type unit)
 StrptimeOptions::StrptimeOptions() : StrptimeOptions("", TimeUnit::SECOND) {}
 constexpr char StrptimeOptions::kTypeName[];
 
+StrftimeOptions::StrftimeOptions(std::string format, std::string timezone)
+    : FunctionOptions(internal::kStrftimeOptionsType),
+      format(std::move(format)),
+      timezone(std::move(timezone)) {
+  tz = arrow_vendored::date::locate_zone(this->timezone);
+}
+StrftimeOptions::StrftimeOptions() : StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "UTC") {}

Review comment:
       The default is set to UTC, but for timestamp with timezone input, can we default to using the type's timezone?
   
   Also for timestamp without timezone, it should probably not use `%z` by default?
   
   And see my other comment, I suppose the default should rather be a literal `Z` instead of `%z` (or `%Z`) ?

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

Review comment:
       How does this keyword behave if the type already has a timezone (then a simple conversion I assume) or if the type has no time zone?
   
   (alternative could be to leave out this option and require users first to explicitly convert from one timezone to another (which is a cheap metadata only change) or localize timezone-naive data)

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

Review comment:
       Although for local timestamp without timezone, requiring the user to first localize it to a timezone-aware type might give additional overhead, as `strftime` would then convert it back to a local timestamp?

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -216,5 +216,40 @@ TEST_F(ScalarTemporalTest, DayOfWeek) {
                 DayOfWeek(timestamps, DayOfWeekOptions(/*one_based_numbering=*/false,
                                                        /*week_start=*/8)));
 }
+
+#ifndef _WIN32
+TEST_F(ScalarTemporalTest, Strftime) {
+  auto options_seconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "UTC");
+  auto options_milliseconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "GMT");
+  auto options_microseconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "Asia/Kolkata");
+  auto options_nanoseconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "US/Hawaii");
+
+  const char* times_seconds = R"(["1970-01-01T00:00:59", null])";
+  const char* times_milliseconds = R"(["1970-01-01T00:00:59.123", null])";
+  const char* times_microseconds = R"(["1970-01-01T00:00:59.123456", null])";
+  const char* times_nanoseconds = R"(["1970-01-01T00:00:59.123456789", null])";
+
+  const char* zoned_seconds = R"(["1970-01-01T00:00:59+0000", null])";
+  const char* zoned_milliseconds = R"(["1970-01-01T00:00:59.123+0000", null])";

Review comment:
       With a default format of `%S`, I don't expect sub-second values to be shown (since there is a `%f` for this) ?

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -321,6 +322,82 @@ struct Nanosecond {
   }
 };
 
+// ----------------------------------------------------------------------
+// Convert timestamps to a string representation with an arbitrary format
+
+template <typename Duration>
+inline std::string get_timestamp(int64_t arg, const StrftimeOptions* options) {
+  auto zt = arrow_vendored::date::zoned_time<Duration>{options->tz,
+                                                       sys_time<Duration>(Duration{arg})};
+  return arrow_vendored::date::format(options->format, zt);
+}
+
+// If source timestamp is timezone naive we do not print timezone info
+inline StrftimeOptions get_options(KernelContext* ctx, const bool timezone_known) {
+  const StrftimeOptions options = StrftimeState::Get(ctx);
+  if (timezone_known) {
+    return options;
+  }
+
+  std::string new_format = std::move(options.format);
+  for (std::string str : {"%z", "%Z"}) {
+    size_t pos = std::string::npos;
+    while ((pos = new_format.find(str)) != std::string::npos) {
+      new_format.erase(pos, str.length());
+    }
+  }
+  return StrftimeOptions(new_format, options.timezone);
+}
+
+template <typename Duration>
+struct Strftime {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const auto& timezone = GetInputTimezone(in);
+    const StrftimeOptions options = get_options(ctx, !timezone.empty());
+    if (timezone.empty() && options.timezone != "UTC") {
+      return Status::Invalid("Timezone naive timestamp can only be printed in UTC. Got: ",

Review comment:
       Why allow UTC here (and not other timezones, or disallow any timezone including UTC) ?

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -216,5 +216,40 @@ TEST_F(ScalarTemporalTest, DayOfWeek) {
                 DayOfWeek(timestamps, DayOfWeekOptions(/*one_based_numbering=*/false,
                                                        /*week_start=*/8)));
 }
+
+#ifndef _WIN32
+TEST_F(ScalarTemporalTest, Strftime) {
+  auto options_seconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "UTC");
+  auto options_milliseconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "GMT");
+  auto options_microseconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "Asia/Kolkata");
+  auto options_nanoseconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "US/Hawaii");
+
+  const char* times_seconds = R"(["1970-01-01T00:00:59", null])";
+  const char* times_milliseconds = R"(["1970-01-01T00:00:59.123", null])";
+  const char* times_microseconds = R"(["1970-01-01T00:00:59.123456", null])";
+  const char* times_nanoseconds = R"(["1970-01-01T00:00:59.123456789", null])";
+
+  const char* zoned_seconds = R"(["1970-01-01T00:00:59+0000", null])";
+  const char* zoned_milliseconds = R"(["1970-01-01T00:00:59.123+0000", null])";

Review comment:
       I see you commented about this in the python/pandas tests below as well. 
   Is this the default behaviour of `date.h` (which seems a strange deviation from the standard?)




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r668588512



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -602,6 +685,15 @@ 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"
+     "Returns an error if timestamp has a defined timezone.\n"

Review comment:
       Oh, sorry forgot to delete that.




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-880628622


   @jorisvandenbossche Could you take a look if this PR makes sense?
   
   @thisisnic this should unblock [ARROW-13133](https://issues.apache.org/jira/browse/ARROW-13133). Does the [R wrapper look ok](https://github.com/apache/arrow/pull/10647/files#diff-a138b87f0da58d824c72293eab62d9f352718dcb7f41a47ea7e1c3c84bbe27dd)? Note that `StrftimeOptions()` returns by default `StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "UTC")`.


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



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

Posted by GitBox <gi...@apache.org>.
nealrichardson commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r676641413



##########
File path: r/src/compute.cpp
##########
@@ -288,6 +288,19 @@ std::shared_ptr<arrow::compute::FunctionOptions> make_compute_options(
         cpp11::as_cpp<arrow::TimeUnit::type>(options["unit"]));
   }
 
+  if (func_name == "strftime") {
+    using Options = arrow::compute::StrftimeOptions;
+    std::string format = "%Y-%m-%dT%H:%M:%SZ";
+    std::string locale = "C";
+    if (!Rf_isNull(options["format"])) {
+      format = cpp11::as_cpp<std::string>(options["format"]);
+    }
+    if (!Rf_isNull(options["locale"])) {
+      locale = cpp11::as_cpp<std::string>(options["locale"]);
+    }
+    return std::make_shared<Options>(Options(format, locale));
+  }
+

Review comment:
       Thanks for adding this, but since there aren't any tests that exercise this I don't think we're done with the R bindings. I've made ARROW-13448 for that.

##########
File path: r/configure.win
##########
@@ -44,7 +44,7 @@ else
   RWINLIB="../windows/$(ls windows/ | grep ^arrow- | tail -n 1)"
 fi
 OPENSSL_LIBS="-lcrypto -lcrypt32"
-MIMALLOC_LIBS="-lbcrypt -lpsapi"
+MIMALLOC_LIBS="-lbcrypt -lpsapi -lole32"

Review comment:
       This isn't the right place to put it because it's not about mimalloc. I think L53 is where it belongs




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-891075094


   > @rok Is this ready for review again?
   
   Sorry I still need to refactor `MakeStrftime`. I'll try to do it tonight.


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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r678359188



##########
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:
       Done.




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



[GitHub] [arrow] pitrou closed pull request #10647: ARROW-13174: [C++][Compute] Add strftime kernel

Posted by GitBox <gi...@apache.org>.
pitrou closed pull request #10647:
URL: https://github.com/apache/arrow/pull/10647


   


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



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

Posted by GitBox <gi...@apache.org>.
rok commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-893973246


   @pitrou I refactored to have two kernel generators now - `MakeTemporal` and `MakeSimpleUnaryTemporal`. Could you please review if this is dry enough now?


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



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

Posted by GitBox <gi...@apache.org>.
rok commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-900263546


   Also I'm noticing we're doing this a lot:
   ```
   const time_zone* tz;
   try {
     tz = locate_zone(timezone);
   } catch (const std::runtime_error& ex) {
     return Status::Invalid(ex.what());
   }
   ```
   
   Can we do something like:
   ```
   Status LocateZone(std::string timezone, const time_zone* tz) {
     try {
       tz = locate_zone(timezone);
     } catch (const std::runtime_error& ex) {
       return Status::Invalid(ex.what());
     }
     return Status::OK();
   }
   
   const time_zone* tz;
   RETURN_NOT_OK(LocateZone(timezone, tz));
   ```


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



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

Posted by GitBox <gi...@apache.org>.
rok commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-887629308


   Looks like a DuckDB issue ([ARROW-13453](https://github.com/apache/arrow/pull/10807)) now.


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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r668820043



##########
File path: python/pyarrow/tests/test_compute.py
##########
@@ -1347,6 +1351,32 @@ def test_strptime():
     assert got == expected
 
 
+# TODO: We should test on windows once ARROW-13168 is resolved.
+@pytest.mark.pandas
+@pytest.mark.skipif(sys.platform == 'win32',
+                    reason="Timezone database is not available on Windows yet")
+def test_strftime():

Review comment:
       Done.




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-901241662


   Nice! Thanks @westonpace @jorisvandenbossche @pitrou 


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



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

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r670695787



##########
File path: cpp/src/arrow/compute/api_scalar.cc
##########
@@ -240,6 +243,15 @@ StrptimeOptions::StrptimeOptions(std::string format, TimeUnit::type unit)
 StrptimeOptions::StrptimeOptions() : StrptimeOptions("", TimeUnit::SECOND) {}
 constexpr char StrptimeOptions::kTypeName[];
 
+StrftimeOptions::StrftimeOptions(std::string format, std::string timezone)
+    : FunctionOptions(internal::kStrftimeOptionsType),
+      format(std::move(format)),
+      timezone(std::move(timezone)) {
+  tz = arrow_vendored::date::locate_zone(this->timezone);
+}
+StrftimeOptions::StrftimeOptions() : StrftimeOptions("%Y-%m-%dT%H:%M:%S", "UTC") {}

Review comment:
       If it is significant effort I agree that `+00:00` is good enough but I personally prefer `Z` if there is a quick option to enable it when the time zone is UTC.




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



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

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-901150972


   Will merge if green.


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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r678369688



##########
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:
       Ok, I'll give it a spin.




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-900367148


   > Yes, you can. Make it return a `Result<const time_zone*>`
   
   @pitrou Done. Please review.


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



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

Posted by GitBox <gi...@apache.org>.
rok commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-901241662


   Nice! Thanks @westonpace @jorisvandenbossche @pitrou 


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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r670873848



##########
File path: cpp/src/arrow/compute/api_scalar.cc
##########
@@ -240,6 +243,15 @@ StrptimeOptions::StrptimeOptions(std::string format, TimeUnit::type unit)
 StrptimeOptions::StrptimeOptions() : StrptimeOptions("", TimeUnit::SECOND) {}
 constexpr char StrptimeOptions::kTypeName[];
 
+StrftimeOptions::StrftimeOptions(std::string format, std::string timezone)
+    : FunctionOptions(internal::kStrftimeOptionsType),
+      format(std::move(format)),
+      timezone(std::move(timezone)) {
+  tz = arrow_vendored::date::locate_zone(this->timezone);
+}
+StrftimeOptions::StrftimeOptions() : StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "UTC") {}

Review comment:
       Switched to `Z` and removed `options.timezone` parameter to default  to type's timezone.




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



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

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r678351566



##########
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" plural then?




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-877493031


   @westonpace [strftime kernel](https://github.com/apache/arrow/pull/10647/files#diff-53850537a2f3f7977473b30241fdd3ddb35aa97269284c06471d9b0c599e2c59R347) is almost ready for review.
   I do need to take care of the string encoding.


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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r678366291



##########
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:
       Got it. Added.

##########
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:
       Done.




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r678358843



##########
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:
       Sounds good to me either way.




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



[GitHub] [arrow] rok edited a comment on pull request #10647: ARROW-13174: [C++][Compute] Add strftime kernel

Posted by GitBox <gi...@apache.org>.
rok edited a comment on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-883731080


   I've added a [`locale` parameter](https://github.com/apache/arrow/pull/10647/files#diff-53850537a2f3f7977473b30241fdd3ddb35aa97269284c06471d9b0c599e2c59R334) to enable things like printing weekday and month names in a specific languages. Only `"C"` locale is tested now (as it's the only one always available) but we could add more locales to docker images to test.


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



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

Posted by GitBox <gi...@apache.org>.
rok commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-887636424


   > Builds are green now so I'm good with this (though to be clear I did not review the C++ changes)
   
   In my mind there was also the [open question of the Python style `%S.%f` vs c++ style `%S`](https://github.com/apache/arrow/pull/10647#discussion_r671518524). @jorisvandenbossche


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



[GitHub] [arrow] rok edited a comment on pull request #10647: ARROW-13174: [C++][Compute] Add strftime kernel

Posted by GitBox <gi...@apache.org>.
rok edited a comment on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-900263119


   @pitrou I rebased on ARROW-12980 and I think this is ready for review.
   
   I've also included a [comment @nealrichardson requested](https://github.com/apache/arrow/pull/10457#pullrequestreview-731074043).


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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r678350468



##########
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:
       Done.




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



[GitHub] [arrow] rok edited a comment on pull request #10647: ARROW-13174: [C++][Compute] Add strftime kernel

Posted by GitBox <gi...@apache.org>.
rok edited a comment on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-881080829


   @jorisvandenbossche & @westonpace Thanks for the review! Removing `options.timezone` simplifies things a lot. I think now the main thing left open is the `%S` style question


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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r690452017



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -382,5 +382,75 @@ TEST_F(ScalarTemporalTest, DayOfWeek) {
                 DayOfWeek(timestamps, DayOfWeekOptions(/*one_based_numbering=*/false,
                                                        /*week_start=*/8)));
 }
+
+#ifndef _WIN32
+TEST_F(ScalarTemporalTest, Strftime) {
+  auto options_default = StrftimeOptions();
+  auto options = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z");
+
+  const char* seconds = R"(["1970-01-01T00:00:59", null])";
+  const char* milliseconds = R"(["1970-01-01T00:00:59.123", null])";
+  const char* microseconds = R"(["1970-01-01T00:00:59.123456", null])";
+  const char* nanoseconds = R"(["1970-01-01T00:00:59.123456789", null])";
+
+  const char* default_seconds = R"(["1970-01-01T00:00:59Z", null])";
+  const char* string_seconds = R"(["1970-01-01T00:00:59+0000", null])";
+  const char* string_milliseconds = R"(["1970-01-01T00:00:59.123+0000", null])";
+  const char* string_microseconds = R"(["1970-01-01T05:30:59.123456+0530", null])";
+  const char* string_nanoseconds = R"(["1969-12-31T14:00:59.123456789-1000", null])";
+
+  CheckScalarUnary("strftime", timestamp(TimeUnit::SECOND, "UTC"), seconds, utf8(),
+                   default_seconds, &options_default);
+  CheckScalarUnary("strftime", timestamp(TimeUnit::SECOND, "UTC"), seconds, utf8(),
+                   string_seconds, &options);
+  CheckScalarUnary("strftime", timestamp(TimeUnit::MILLI, "GMT"), milliseconds, utf8(),
+                   string_milliseconds, &options);
+  CheckScalarUnary("strftime", timestamp(TimeUnit::MICRO, "Asia/Kolkata"), microseconds,
+                   utf8(), string_microseconds, &options);
+  CheckScalarUnary("strftime", timestamp(TimeUnit::NANO, "US/Hawaii"), nanoseconds,
+                   utf8(), string_nanoseconds, &options);
+
+  auto naive_times = ArrayFromJSON(timestamp(TimeUnit::SECOND), seconds);
+  ASSERT_RAISES(Invalid, Strftime(naive_times, options_default));
+  ASSERT_RAISES(Invalid, Strftime(naive_times, options));
+}
+
+TEST_F(ScalarTemporalTest, StrftimeWithLocale) {
+  auto options_default = StrftimeOptions();
+  auto options = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "C");
+  auto options_locale_specific = StrftimeOptions("%a", "C");

Review comment:
       Please note this is only testing for `C` locale because that is all that is available in CI at the moment. Ideally we would cover several.




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-887557250


   > It appears that some change about `locale` breaks things on gcc 4.8, as revealed by the R centos7 job: https://github.com/apache/arrow/pull/10647/checks?check_run_id=3162539131#step:9:584
   
   Huh, I completely missed that message. Seems [related to this](https://stackoverflow.com/questions/20083849/why-am-i-getting-an-error-using-stdlocalelocaleconst-stdstring). Changed the instantiation, let's see what happens.


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



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

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r671500143



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -216,5 +216,40 @@ TEST_F(ScalarTemporalTest, DayOfWeek) {
                 DayOfWeek(timestamps, DayOfWeekOptions(/*one_based_numbering=*/false,
                                                        /*week_start=*/8)));
 }
+
+#ifndef _WIN32
+TEST_F(ScalarTemporalTest, Strftime) {
+  auto options_seconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "UTC");
+  auto options_milliseconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "GMT");
+  auto options_microseconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "Asia/Kolkata");
+  auto options_nanoseconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "US/Hawaii");
+
+  const char* times_seconds = R"(["1970-01-01T00:00:59", null])";
+  const char* times_milliseconds = R"(["1970-01-01T00:00:59.123", null])";
+  const char* times_microseconds = R"(["1970-01-01T00:00:59.123456", null])";
+  const char* times_nanoseconds = R"(["1970-01-01T00:00:59.123456789", null])";
+
+  const char* zoned_seconds = R"(["1970-01-01T00:00:59+0000", null])";
+  const char* zoned_milliseconds = R"(["1970-01-01T00:00:59.123+0000", null])";

Review comment:
       Possible relevant: Related: https://github.com/HowardHinnant/date/issues/562

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -216,5 +216,40 @@ TEST_F(ScalarTemporalTest, DayOfWeek) {
                 DayOfWeek(timestamps, DayOfWeekOptions(/*one_based_numbering=*/false,
                                                        /*week_start=*/8)));
 }
+
+#ifndef _WIN32
+TEST_F(ScalarTemporalTest, Strftime) {
+  auto options_seconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "UTC");
+  auto options_milliseconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "GMT");
+  auto options_microseconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "Asia/Kolkata");
+  auto options_nanoseconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "US/Hawaii");
+
+  const char* times_seconds = R"(["1970-01-01T00:00:59", null])";
+  const char* times_milliseconds = R"(["1970-01-01T00:00:59.123", null])";
+  const char* times_microseconds = R"(["1970-01-01T00:00:59.123456", null])";
+  const char* times_nanoseconds = R"(["1970-01-01T00:00:59.123456789", null])";
+
+  const char* zoned_seconds = R"(["1970-01-01T00:00:59+0000", null])";
+  const char* zoned_milliseconds = R"(["1970-01-01T00:00:59.123+0000", null])";

Review comment:
       Possible relevant: https://github.com/HowardHinnant/date/issues/562




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r671494734



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -216,5 +216,40 @@ TEST_F(ScalarTemporalTest, DayOfWeek) {
                 DayOfWeek(timestamps, DayOfWeekOptions(/*one_based_numbering=*/false,
                                                        /*week_start=*/8)));
 }
+
+#ifndef _WIN32
+TEST_F(ScalarTemporalTest, Strftime) {
+  auto options_seconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "UTC");
+  auto options_milliseconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "GMT");
+  auto options_microseconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "Asia/Kolkata");
+  auto options_nanoseconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "US/Hawaii");
+
+  const char* times_seconds = R"(["1970-01-01T00:00:59", null])";
+  const char* times_milliseconds = R"(["1970-01-01T00:00:59.123", null])";
+  const char* times_microseconds = R"(["1970-01-01T00:00:59.123456", null])";
+  const char* times_nanoseconds = R"(["1970-01-01T00:00:59.123456789", null])";
+
+  const char* zoned_seconds = R"(["1970-01-01T00:00:59+0000", null])";
+  const char* zoned_milliseconds = R"(["1970-01-01T00:00:59.123+0000", null])";

Review comment:
       It would be nice to have an option to match Python here. Perhaps we could try to add `%f` to `date.h`.




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-881080829


   @jorisvandenbossche Thanks for the review! Removing `options.timezone` simplifies things a lot. I think now the main thing left open is the `%S` style question


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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r671518524



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -216,5 +216,40 @@ TEST_F(ScalarTemporalTest, DayOfWeek) {
                 DayOfWeek(timestamps, DayOfWeekOptions(/*one_based_numbering=*/false,
                                                        /*week_start=*/8)));
 }
+
+#ifndef _WIN32
+TEST_F(ScalarTemporalTest, Strftime) {
+  auto options_seconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "UTC");
+  auto options_milliseconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "GMT");
+  auto options_microseconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "Asia/Kolkata");
+  auto options_nanoseconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "US/Hawaii");
+
+  const char* times_seconds = R"(["1970-01-01T00:00:59", null])";
+  const char* times_milliseconds = R"(["1970-01-01T00:00:59.123", null])";
+  const char* times_microseconds = R"(["1970-01-01T00:00:59.123456", null])";
+  const char* times_nanoseconds = R"(["1970-01-01T00:00:59.123456789", null])";
+
+  const char* zoned_seconds = R"(["1970-01-01T00:00:59+0000", null])";
+  const char* zoned_milliseconds = R"(["1970-01-01T00:00:59.123+0000", null])";

Review comment:
       [Looks `date.h` is going a different way than `%f`](https://github.com/HowardHinnant/date/issues/562#issuecomment-608406611). We could mimic that to a degree and add a `precision` parameter to add a possibility to round to `s, ms, us, ns` precisions `(1, 1.123, 1.123456, 1.123456789)`. Unfortunately that only allows decimal place setting in increments of 3.




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r668593700



##########
File path: python/pyarrow/tests/test_compute.py
##########
@@ -1347,6 +1351,32 @@ def test_strptime():
     assert got == expected
 
 
+# TODO: We should test on windows once ARROW-13168 is resolved.
+@pytest.mark.pandas
+@pytest.mark.skipif(sys.platform == 'win32',
+                    reason="Timezone database is not available on Windows yet")
+def test_strftime():

Review comment:
       date.h will fail on naive timestamp:
   > ***%z*** | The offset from UTC in the ISO 8601 format. For example -0430 refers to 4 hours 30 minutes behind UTC. If the offset is zero, +0000 is used. The modified commands %Ez and %Oz insert a : between the hours and minutes: -04:30. If the offset information is not available, failbit will be set.
   
   I'll see if I can make it pandas-like.




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r678350597



##########
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:
       Done.




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



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

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r669098242



##########
File path: r/configure.win
##########
@@ -44,7 +44,7 @@ else
   RWINLIB="../windows/$(ls windows/ | grep ^arrow- | tail -n 1)"
 fi
 OPENSSL_LIBS="-lcrypto -lcrypt32"
-MIMALLOC_LIBS="-lbcrypt -lpsapi"
+MIMALLOC_LIBS="-lbcrypt -lpsapi -lole32"

Review comment:
       Is this related?




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



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

Posted by GitBox <gi...@apache.org>.
jorisvandenbossche commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-901783928


   Sorry for the slow response here, but I think there are still a few behavioural aspects to fix/clarify:
   
   * Related to @westonpace's comment above (https://github.com/apache/arrow/pull/10647#discussion_r668364905), you added a "Z" to the default format. However, this is only correct if you have a UTC timezone, and not for any other timezone. For example:
       ```python
       >>> ts = pd.to_datetime(["2018-03-10 09:00"]).tz_localize("US/Eastern")
       >>> ts
       DatetimeIndex(['2018-03-10 09:00:00-05:00'], dtype='datetime64[ns, US/Eastern]', freq=None)
       >>> tsa = pa.array(ts)
       >>> tsa
       <pyarrow.lib.TimestampArray object at 0x7f7350b087c0>
       [
         2018-03-10 14:00:00.000000000
       ]
   
       >>> pc.strftime(tsa)
       <pyarrow.lib.StringArray object at 0x7f7350b74a60>
       [
         "2018-03-10T09:00:00.000000000Z"
       ]
       ```
     So it's correctly showing the timestamp in the timezone's local time, but thus the "Z" indicator for UTC is wrong (the correct UTC time is 14:00, not 09:00). 
     I think we should only add the "Z" indicator if the timezone is UTC. I am not fully sure what we should then use as default format for non-UTC timezones though: don't show any timezone information, include a numeric offset, or error. 
     That would also mean that the "default" format string would depend on the input type of the data, which might not be easy / desirable.
   
   - I commented about the timezone handling when the initial PR had a keyword for this, but I forgot to reply after you removed that keyword (and support for local timestamps) altogether. But, what's the reasoning for disallowing local timestamps without timezone? I don't think there is any ambiguity in how they would be formatted? (after all, they represent "clock" time, which in the end is kind of a formatted string)
   
   - There was some discussion above about the behaviour of `%S` (https://github.com/apache/arrow/pull/10647#discussion_r670410876), where `date.h` / C++ handles it differently as Python or R (i.e. we are including the fractional sub-second decimals, and there is no easy way to only show integer seconds apart from casting to `timestamp("s")` first AFAIK). 
     Since there are conflicting standards vs language implementations, there is no easy way to solve this. But I think it would be good to at least document this difference (it will be surprising for Python/R users) and how to work-around it.


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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r670872947



##########
File path: cpp/src/arrow/compute/api_scalar.cc
##########
@@ -240,6 +243,15 @@ StrptimeOptions::StrptimeOptions(std::string format, TimeUnit::type unit)
 StrptimeOptions::StrptimeOptions() : StrptimeOptions("", TimeUnit::SECOND) {}
 constexpr char StrptimeOptions::kTypeName[];
 
+StrftimeOptions::StrftimeOptions(std::string format, std::string timezone)
+    : FunctionOptions(internal::kStrftimeOptionsType),
+      format(std::move(format)),
+      timezone(std::move(timezone)) {
+  tz = arrow_vendored::date::locate_zone(this->timezone);
+}
+StrftimeOptions::StrftimeOptions() : StrftimeOptions("%Y-%m-%dT%H:%M:%S", "UTC") {}

Review comment:
       Switched to `%Y-%m-%dT%H:%M:%SZ` (2.) as default.




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r678349121



##########
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:
       And there are two more coming (https://github.com/apache/arrow/pull/10610, https://github.com/apache/arrow/pull/10457).
   Would it make more sense to rather do this in the last merge?




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



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

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-900275343


   Yes, you can. Make it return a `Result<const time_zone*>`


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



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

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r678350853



##########
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:
       Ok, then I have think the function should still be exposed but return `Status::NotImplemented`.




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



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

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r671470682



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -321,6 +323,68 @@ struct Nanosecond {
   }
 };
 
+// ----------------------------------------------------------------------
+// Convert timestamps to a string representation with an arbitrary format
+
+template <typename Duration>
+inline std::string get_timestamp(int64_t arg, 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(*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("Timezone naive timestamp can not be reliably printed.");

Review comment:
       ```suggestion
         return Status::Invalid("Timestamps without a time zone cannot be reliably printed.");
   ```
   Nit: We don't formally use "naive" in the docs that I'm aware of as it is a python-only concept. 

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -957,5 +968,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
+///
+/// \param[in] values input to print time string from
+/// \param[in] options for setting time format and timezone

Review comment:
       ```suggestion
   /// \param[in] options for setting time format
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -321,6 +323,68 @@ struct Nanosecond {
   }
 };
 
+// ----------------------------------------------------------------------
+// Convert timestamps to a string representation with an arbitrary format
+
+template <typename Duration>
+inline std::string get_timestamp(int64_t arg, 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(*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("Timezone naive timestamp can not 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, 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("Timezone naive timestamp can not be reliably printed.");

Review comment:
       ```suggestion
         return Status::Invalid("Timestamp without a time zone cannot be reliably printed.");
   ```
   ditto




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r670801579



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -216,5 +216,40 @@ TEST_F(ScalarTemporalTest, DayOfWeek) {
                 DayOfWeek(timestamps, DayOfWeekOptions(/*one_based_numbering=*/false,
                                                        /*week_start=*/8)));
 }
+
+#ifndef _WIN32
+TEST_F(ScalarTemporalTest, Strftime) {
+  auto options_seconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "UTC");
+  auto options_milliseconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "GMT");
+  auto options_microseconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "Asia/Kolkata");
+  auto options_nanoseconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "US/Hawaii");
+
+  const char* times_seconds = R"(["1970-01-01T00:00:59", null])";
+  const char* times_milliseconds = R"(["1970-01-01T00:00:59.123", null])";
+  const char* times_microseconds = R"(["1970-01-01T00:00:59.123456", null])";
+  const char* times_nanoseconds = R"(["1970-01-01T00:00:59.123456789", null])";
+
+  const char* zoned_seconds = R"(["1970-01-01T00:00:59+0000", null])";
+  const char* zoned_milliseconds = R"(["1970-01-01T00:00:59.123+0000", null])";

Review comment:
       `date.h` doesn't appear to have a `%f` flag.
   
   [date.h](https://howardhinnant.github.io/date/date.html) doc states:
   
   > ***%S*** | Seconds as a decimal number. If the number of seconds is less than 10, the result is prefixed with 0. If the precision of the input can not be exactly represented with seconds, then the format is a decimal floating point number with a fixed format and a precision matching that of the precision of the input (or to a microseconds precision if the conversion to floating point decimal seconds can not be made within 18 fractional digits). The character for the decimal point is localized according to the locale. The modified command %OS produces the locale's alternative representation.
   
   Which almost exactly matches [c++ formatter](https://en.cppreference.com/w/cpp/chrono/duration/formatter).
   
   Implementing [python-like](https://strftime.org/) `%f` and `%S` would require additional logic here.
   
   R appears to have a [similar convention](https://stat.ethz.ch/R-manual/R-devel/library/base/html/strptime.html) to Python.
   
   [SQLite](https://sqlite.org/lang_datefunc.html):
   ```
   %f |   | fractional seconds: SS.SSS
   %S |   | seconds: 00-59
   ```
   [Postgres](https://www.postgresql.org/docs/9.1/ecpg-pgtypes.html):
   ```
   %S - is replaced by the second as a decimal number (00-60).
   No fractions.
   ```
   [Ruby](https://apidock.com/ruby/DateTime/strftime):
   ```
   %S - Second of the minute (00..59)
   No fractions.
   ```
   
   I kind of like the current behavior but it could would of course not cover all cases.




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



[GitHub] [arrow] github-actions[bot] commented on pull request #10647: ARROW-13174: [C++][Compute] Add strftime kernel

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-872637340


   https://issues.apache.org/jira/browse/ARROW-13174


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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r670874881



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -321,6 +322,82 @@ struct Nanosecond {
   }
 };
 
+// ----------------------------------------------------------------------
+// Convert timestamps to a string representation with an arbitrary format
+
+template <typename Duration>
+inline std::string get_timestamp(int64_t arg, const StrftimeOptions* options) {
+  auto zt = arrow_vendored::date::zoned_time<Duration>{options->tz,
+                                                       sys_time<Duration>(Duration{arg})};
+  return arrow_vendored::date::format(options->format, zt);
+}
+
+// If source timestamp is timezone naive we do not print timezone info
+inline StrftimeOptions get_options(KernelContext* ctx, const bool timezone_known) {
+  const StrftimeOptions options = StrftimeState::Get(ctx);
+  if (timezone_known) {
+    return options;
+  }
+
+  std::string new_format = std::move(options.format);
+  for (std::string str : {"%z", "%Z"}) {
+    size_t pos = std::string::npos;
+    while ((pos = new_format.find(str)) != std::string::npos) {
+      new_format.erase(pos, str.length());
+    }
+  }
+  return StrftimeOptions(new_format, options.timezone);
+}
+
+template <typename Duration>
+struct Strftime {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const auto& timezone = GetInputTimezone(in);
+    const StrftimeOptions options = get_options(ctx, !timezone.empty());
+    if (timezone.empty() && options.timezone != "UTC") {
+      return Status::Invalid("Timezone naive timestamp can only be printed in UTC. Got: ",

Review comment:
       Removed `options.timezone` so this logic was removed.




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



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

Posted by GitBox <gi...@apache.org>.
jorisvandenbossche commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-903621489


   Thanks!


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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r678349808



##########
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:
       I took your `ReserveData` suggestion and removed `Reserve`.

##########
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:
       Done.




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-905523497


   Follow up PR https://github.com/apache/arrow/pull/10998


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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r678344340



##########
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:
       This currently uses timezones (it could probably temporarily be done without for UTC only). We don't yet have timezones solved on windows [ARROW-13168](https://issues.apache.org/jira/browse/ARROW-13168), hence the exclusion.




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r670874666



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

Review comment:
       I'm not sure we can print timezone naive timestamps without "understanding" them as UTC first anyway (in the `date.h` logic) so I don't think we can get rid of the overhead. But we can choose to do it with one kernel or two.




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r669107412



##########
File path: r/configure.win
##########
@@ -44,7 +44,7 @@ else
   RWINLIB="../windows/$(ls windows/ | grep ^arrow- | tail -n 1)"
 fi
 OPENSSL_LIBS="-lcrypto -lcrypt32"
-MIMALLOC_LIBS="-lbcrypt -lpsapi"
+MIMALLOC_LIBS="-lbcrypt -lpsapi -lole32"

Review comment:
       Yeah, without it RTools40 fails on mingw32 build:
   ```
   C:/rtools40/mingw32/bin/../lib/gcc/i686-w64-mingw32/8.3.0/../../../../i686-w64-mingw32/bin/ld.exe: ../windows/arrow-4.0.1.9000/lib/i386/libarrow.a(tz.cpp.obj):(.text+0x725): undefined reference to `_imp__CoTaskMemFree@4'
   C:/rtools40/mingw32/bin/../lib/gcc/i686-w64-mingw32/8.3.0/../../../../i686-w64-mingw32/bin/ld.exe: ../windows/arrow-4.0.1.9000/lib/i386/libarrow.a(tz.cpp.obj):(.text.unlikely+0x3e): undefined reference to `_imp__CoTaskMemFree@4'
   ```
    [Fix source here](https://github.com/HowardHinnant/date/issues/272). I don't really know what it really does. I'm also not sure if this is the place to put it. :)




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-880656288


   > Naive question: how does strftime return the correct local-specific time when using the format that outputs days of week? I see that in the base R implementation of strftime, it's not a parameter that's passed in; is this something that here is detected by the vendored date libraries that we're using?
   
   Good question! I've tried adding a locale parameter to `StrftimeOptions` to be able to choose locale but locale instantiation was slightly different for different OS-es so I've opted to take it out of scope. I've not properly tested but I believe you will currently get the result in your systems locale.
   
   I would like to add the locale parameter to this but it might be good to tackle it in a separate PR. I'm open to adding it here too though.


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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r676661819



##########
File path: r/configure.win
##########
@@ -44,7 +44,7 @@ else
   RWINLIB="../windows/$(ls windows/ | grep ^arrow- | tail -n 1)"
 fi
 OPENSSL_LIBS="-lcrypto -lcrypt32"
-MIMALLOC_LIBS="-lbcrypt -lpsapi"
+MIMALLOC_LIBS="-lbcrypt -lpsapi -lole32"

Review comment:
       Thanks! Moved.




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



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

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-900080615


   @rok Please tell me when this is ready for review.


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



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

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r678351868



##########
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:
       No, it would make more sense to refactor now.




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r668589540



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -321,6 +322,53 @@ struct Nanosecond {
   }
 };
 
+// ----------------------------------------------------------------------
+// Convert timestamps to a string representation with an arbitrary format
+
+template <typename Duration>
+inline std::string get_timestamp(int64_t arg, const StrftimeOptions* options) {
+  auto zt = arrow_vendored::date::zoned_time<Duration>{options->tz,
+                                                       sys_time<Duration>(Duration{arg})};
+  return arrow_vendored::date::format(options->format, zt.get_local_time());
+}
+
+template <typename Duration>
+struct Strftime {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    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));
+    } else {
+      out->is_valid = false;
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    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());
+    RETURN_NOT_OK(string_builder->Reserve(in.length * 30));

Review comment:
       How about a random date's string size times a fudge factor? :)
   ```
   int expected_string_size = round(get_timestamp<Duration>(0, &options).size() * 1.1);
   RETURN_NOT_OK(string_builder->Reserve(in.length * expected_string_size));
   ```




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r668589540



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -321,6 +322,53 @@ struct Nanosecond {
   }
 };
 
+// ----------------------------------------------------------------------
+// Convert timestamps to a string representation with an arbitrary format
+
+template <typename Duration>
+inline std::string get_timestamp(int64_t arg, const StrftimeOptions* options) {
+  auto zt = arrow_vendored::date::zoned_time<Duration>{options->tz,
+                                                       sys_time<Duration>(Duration{arg})};
+  return arrow_vendored::date::format(options->format, zt.get_local_time());
+}
+
+template <typename Duration>
+struct Strftime {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    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));
+    } else {
+      out->is_valid = false;
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    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());
+    RETURN_NOT_OK(string_builder->Reserve(in.length * 30));

Review comment:
       How about a random string size times a fudge factor? :)
   ```
   int expected_string_size = round(get_timestamp<Duration>(0, &options).size() * 1.2);
   RETURN_NOT_OK(string_builder->Reserve(in.length * expected_string_size));
   ```




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-879403748


   > Thanks for the changes. This looks good. I'm afraid I don't know the differences between `date.h` and the system equivalents. By system equivalents do you mean `ctime`? In that case does strftime support time zones?
   
   I meant like [cpp](https://en.cppreference.com/w/cpp/chrono/c/strftime) or [linux](https://man7.org/linux/man-pages/man3/strftime.3.html) strftimes. It seems they support timezones but one still needs to go from UTC to local time etc.
   
   > Since we already have the vendored lib in I suppose it comes down to readability, simplicity, and performance. What you have here seems plenty readable and simple. We can always benchmark the two approaches later.
   
   Fully agreed. Without benchmarking alternatives it's a moot discussion anyway. I just wanted to check if I missed something obvious.


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



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

Posted by GitBox <gi...@apache.org>.
rok commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-883731080


   I've added a [`locale` parameter](https://github.com/apache/arrow/pull/10647/files#diff-53850537a2f3f7977473b30241fdd3ddb35aa97269284c06471d9b0c599e2c59R334) to enable things like printing weekday and month names in a specific languages.


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



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

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-891069189


   @rok Is this ready for review again?


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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r678350232



##########
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:
       Made a private `strftime` method.




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



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

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-901150972


   Will merge if green.


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



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

Posted by GitBox <gi...@apache.org>.
thisisnic commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-880657927


   > > Naive question: how does strftime return the correct local-specific time when using the format that outputs days of week? I see that in the base R implementation of strftime, it's not a parameter that's passed in; is this something that here is detected by the vendored date libraries that we're using?
   > 
   > Good question! I've tried adding a locale parameter to `StrftimeOptions` to be able to choose locale but locale instantiation was slightly different for different OS-es so I've opted to take it out of scope. I've not properly tested but I believe you will currently get the result in your systems locale.
   > 
   > I would like to add the locale parameter to this but it might be good to tackle it in a separate PR. I'm open to adding it here too though.
   
   Thanks, that's useful to know!
   
   I don't think anyone is planning on doing ARROW-13133 ahead of the upcoming release, so no great urgency to do it in this PR, as long as there's a ticket created for implementing it.


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



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

Posted by GitBox <gi...@apache.org>.
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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r671492724



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -321,6 +323,68 @@ struct Nanosecond {
   }
 };
 
+// ----------------------------------------------------------------------
+// Convert timestamps to a string representation with an arbitrary format
+
+template <typename Duration>
+inline std::string get_timestamp(int64_t arg, 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(*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("Timezone naive timestamp can not be reliably printed.");

Review comment:
       Agreed.




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r668589894



##########
File path: cpp/src/arrow/compute/api_scalar.cc
##########
@@ -240,6 +243,15 @@ StrptimeOptions::StrptimeOptions(std::string format, TimeUnit::type unit)
 StrptimeOptions::StrptimeOptions() : StrptimeOptions("", TimeUnit::SECOND) {}
 constexpr char StrptimeOptions::kTypeName[];
 
+StrftimeOptions::StrftimeOptions(std::string format, std::string timezone)
+    : FunctionOptions(internal::kStrftimeOptionsType),
+      format(std::move(format)),
+      timezone(std::move(timezone)) {
+  tz = arrow_vendored::date::locate_zone(this->timezone);
+}
+StrftimeOptions::StrftimeOptions() : StrftimeOptions("%Y-%m-%dT%H:%M:%S", "UTC") {}

Review comment:
       Indeed. Will add.




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r670410454



##########
File path: cpp/src/arrow/compute/api_scalar.cc
##########
@@ -240,6 +243,15 @@ StrptimeOptions::StrptimeOptions(std::string format, TimeUnit::type unit)
 StrptimeOptions::StrptimeOptions() : StrptimeOptions("", TimeUnit::SECOND) {}
 constexpr char StrptimeOptions::kTypeName[];
 
+StrftimeOptions::StrftimeOptions(std::string format, std::string timezone)
+    : FunctionOptions(internal::kStrftimeOptionsType),
+      format(std::move(format)),
+      timezone(std::move(timezone)) {
+  tz = arrow_vendored::date::locate_zone(this->timezone);
+}
+StrftimeOptions::StrftimeOptions() : StrftimeOptions("%Y-%m-%dT%H:%M:%S", "UTC") {}

Review comment:
       According to [wikipedia](https://en.wikipedia.org/wiki/ISO_8601) these are valid formats: 
   1. 2021-07-15T11:24:54+00:00
   1. 2021-07-15T11:24:54Z
   1. 20210715T112454Z
   
   `%z` will give us option 1. I don't have a strong preference and pandas doesn't seem to have a default.




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r676662216



##########
File path: r/src/compute.cpp
##########
@@ -288,6 +288,19 @@ std::shared_ptr<arrow::compute::FunctionOptions> make_compute_options(
         cpp11::as_cpp<arrow::TimeUnit::type>(options["unit"]));
   }
 
+  if (func_name == "strftime") {
+    using Options = arrow::compute::StrftimeOptions;
+    std::string format = "%Y-%m-%dT%H:%M:%SZ";
+    std::string locale = "C";
+    if (!Rf_isNull(options["format"])) {
+      format = cpp11::as_cpp<std::string>(options["format"]);
+    }
+    if (!Rf_isNull(options["locale"])) {
+      locale = cpp11::as_cpp<std::string>(options["locale"]);
+    }
+    return std::make_shared<Options>(Options(format, locale));
+  }
+

Review comment:
       Removed and added a comment to ARROW-13448.




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



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

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r671474903



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -216,5 +216,40 @@ TEST_F(ScalarTemporalTest, DayOfWeek) {
                 DayOfWeek(timestamps, DayOfWeekOptions(/*one_based_numbering=*/false,
                                                        /*week_start=*/8)));
 }
+
+#ifndef _WIN32
+TEST_F(ScalarTemporalTest, Strftime) {
+  auto options_seconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "UTC");
+  auto options_milliseconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "GMT");
+  auto options_microseconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "Asia/Kolkata");
+  auto options_nanoseconds = StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "US/Hawaii");
+
+  const char* times_seconds = R"(["1970-01-01T00:00:59", null])";
+  const char* times_milliseconds = R"(["1970-01-01T00:00:59.123", null])";
+  const char* times_microseconds = R"(["1970-01-01T00:00:59.123456", null])";
+  const char* times_nanoseconds = R"(["1970-01-01T00:00:59.123456789", null])";
+
+  const char* zoned_seconds = R"(["1970-01-01T00:00:59+0000", null])";
+  const char* zoned_milliseconds = R"(["1970-01-01T00:00:59.123+0000", null])";

Review comment:
       I don't have strong feelings on this.  `%f` isn't in the standard anyways (POSIX does not believe in subseconds) so it's a deviation either way.




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-902697581


   Thanks for the review @jorisvandenbossche !
   I think we need to address these points - I'll open a follow-up JIRA and do some work in a week or so.
   
   1. Indeed, current situation (Z after non UTC string) is wrong. I think your suggestion (check input timezone and append `Z` to default format if UTC) would be ok.
   2. I forget what the thought was at the time. But looking at [our discussion](https://github.com/apache/arrow/pull/10647#discussion_r670874881) it might have been a `date.h` limitation. I'll look into it.
   3. Agreed! We could also look into upstreaming a behavior flag to `date.h` but I think we'd need stronger motivation.


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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r668589540



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -321,6 +322,53 @@ struct Nanosecond {
   }
 };
 
+// ----------------------------------------------------------------------
+// Convert timestamps to a string representation with an arbitrary format
+
+template <typename Duration>
+inline std::string get_timestamp(int64_t arg, const StrftimeOptions* options) {
+  auto zt = arrow_vendored::date::zoned_time<Duration>{options->tz,
+                                                       sys_time<Duration>(Duration{arg})};
+  return arrow_vendored::date::format(options->format, zt.get_local_time());
+}
+
+template <typename Duration>
+struct Strftime {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    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));
+    } else {
+      out->is_valid = false;
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    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());
+    RETURN_NOT_OK(string_builder->Reserve(in.length * 30));

Review comment:
       How about a random string size times a fudge factor? :)
   ```
   int expected_string_size = round(get_timestamp<Duration>(0, &options).size() * 1.1);
   RETURN_NOT_OK(string_builder->Reserve(in.length * expected_string_size));
   ```




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r670834181



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -321,6 +322,82 @@ struct Nanosecond {
   }
 };
 
+// ----------------------------------------------------------------------
+// Convert timestamps to a string representation with an arbitrary format
+
+template <typename Duration>
+inline std::string get_timestamp(int64_t arg, const StrftimeOptions* options) {
+  auto zt = arrow_vendored::date::zoned_time<Duration>{options->tz,
+                                                       sys_time<Duration>(Duration{arg})};
+  return arrow_vendored::date::format(options->format, zt);
+}
+
+// If source timestamp is timezone naive we do not print timezone info
+inline StrftimeOptions get_options(KernelContext* ctx, const bool timezone_known) {
+  const StrftimeOptions options = StrftimeState::Get(ctx);
+  if (timezone_known) {
+    return options;
+  }
+
+  std::string new_format = std::move(options.format);
+  for (std::string str : {"%z", "%Z"}) {
+    size_t pos = std::string::npos;
+    while ((pos = new_format.find(str)) != std::string::npos) {
+      new_format.erase(pos, str.length());
+    }
+  }
+  return StrftimeOptions(new_format, options.timezone);
+}
+
+template <typename Duration>
+struct Strftime {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const auto& timezone = GetInputTimezone(in);
+    const StrftimeOptions options = get_options(ctx, !timezone.empty());
+    if (timezone.empty() && options.timezone != "UTC") {
+      return Status::Invalid("Timezone naive timestamp can only be printed in UTC. Got: ",

Review comment:
       My thinking was to have an option to print timezone naive timestamps at least as UTC. I'll disallow printing of timezone naive timestamps.




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



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

Posted by GitBox <gi...@apache.org>.
thisisnic commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-880650445


   > @thisisnic this should unblock [ARROW-13133](https://issues.apache.org/jira/browse/ARROW-13133). Does the [R wrapper look ok](https://github.com/apache/arrow/pull/10647/files#diff-a138b87f0da58d824c72293eab62d9f352718dcb7f41a47ea7e1c3c84bbe27dd)? Note that `StrftimeOptions()` returns by default `StrftimeOptions("%Y-%m-%dT%H:%M:%S%z", "UTC")`.
   
   LGTM.
   
   Naive question: how does strftime return the correct local-specific time when using the format that outputs days of week? I see that in the base R implementation of strftime, it's not a parameter that's passed in; is this something that here is detected by the vendored date libraries that we're using?


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



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

Posted by GitBox <gi...@apache.org>.
rok commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-902702047


   https://issues.apache.org/jira/browse/ARROW-13684


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



[GitHub] [arrow] pitrou closed pull request #10647: ARROW-13174: [C++][Compute] Add strftime kernel

Posted by GitBox <gi...@apache.org>.
pitrou closed pull request #10647:
URL: https://github.com/apache/arrow/pull/10647


   


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



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

Posted by GitBox <gi...@apache.org>.
rok commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r678345309



##########
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:
       Changed to "timestamp" singular. Or did you mean something else?




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



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

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#discussion_r678352985



##########
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(in.length)` would still be useful for the null bitmap and offsets.




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



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

Posted by GitBox <gi...@apache.org>.
rok commented on pull request #10647:
URL: https://github.com/apache/arrow/pull/10647#issuecomment-900263119


   @pitrou I rebased on ARROW-12980 and I think it's ready for review.
   
   I've also included a [comment @nealrichardson requested](https://github.com/apache/arrow/pull/10457#pullrequestreview-731074043).


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