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/06/04 22:12:50 UTC

[GitHub] [arrow] rok opened a new pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


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


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

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,107 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestSimpleTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO);
+  auto timestamps = ArrayFromJSON(unit, times);
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("weekday", int64())});
+
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar = ArrayFromJSON(iso_calendar_type,
+                                    R"([{"iso_year": 1970, "iso_week": 1, "weekday": 4},
+                        {"iso_year": 2000, "iso_week": 9, "weekday": 2},
+                        {"iso_year": 1899, "iso_week": 52, "weekday": 7},
+                        {"iso_year": 2033, "iso_week": 20, "weekday": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59.123456789, 23.999999999, 20.001001001, 20.0, null]";
+  auto millisecond = "[123, 999, 1, 0, null]";
+  auto microsecond = "[456, 999, 1, 0, null]";
+  auto nanosecond = "[789, 999, 1, 0, null]";
+  auto subsecond = "[123456789, 999999999, 1001001, 0, null]";
+
+  CheckScalarUnary("year", unit, times, int64(), year);
+  CheckScalarUnary("month", unit, times, int64(), month);
+  CheckScalarUnary("day", unit, times, int64(), day);
+  CheckScalarUnary("day_of_week", unit, times, int64(), day_of_week);
+  CheckScalarUnary("day_of_year", unit, times, int64(), day_of_year);
+  CheckScalarUnary("iso_year", unit, times, int64(), iso_year);
+  CheckScalarUnary("iso_week", unit, times, int64(), iso_week);
+  CheckScalarUnary("iso_calendar", timestamps, iso_calendar);
+  CheckScalarUnary("quarter", unit, times, int64(), quarter);
+  CheckScalarUnary("hour", unit, times, int64(), hour);
+  CheckScalarUnary("minute", unit, times, int64(), minute);
+  CheckScalarUnary("second", unit, times, float64(), second);
+  CheckScalarUnary("millisecond", unit, times, int64(), millisecond);
+  CheckScalarUnary("microsecond", unit, times, int64(), microsecond);
+  CheckScalarUnary("nanosecond", unit, times, int64(), nanosecond);
+  CheckScalarUnary("subsecond", unit, times, int64(), subsecond);
+}
+
+TEST(ScalarTemporalTest, TestZonedTemporalComponentExtraction) {
+  std::string timezone = "Etc/UTC-2";
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO, timezone);
+  auto timestamps = ArrayFromJSON(unit, times);
+
+  ASSERT_RAISES(Invalid, Year(timestamps));
+  ASSERT_RAISES(Invalid, Month(timestamps));
+  ASSERT_RAISES(Invalid, Day(timestamps));
+  ASSERT_RAISES(Invalid, DayOfWeek(timestamps));
+  ASSERT_RAISES(Invalid, DayOfYear(timestamps));
+  ASSERT_RAISES(Invalid, ISOYear(timestamps));
+  ASSERT_RAISES(Invalid, ISOWeek(timestamps));
+  ASSERT_RAISES(Invalid, ISOCalendar(timestamps));
+  ASSERT_RAISES(Invalid, Quarter(timestamps));
+  ASSERT_RAISES(Invalid, Hour(timestamps));
+  ASSERT_RAISES(Invalid, Minute(timestamps));
+  ASSERT_RAISES(Invalid, Second(timestamps));
+  ASSERT_RAISES(Invalid, Millisecond(timestamps));
+  ASSERT_RAISES(Invalid, Microsecond(timestamps));
+  ASSERT_RAISES(Invalid, Nanosecond(timestamps));
+  ASSERT_RAISES(Invalid, Subsecond(timestamps));

Review comment:
       Indeed, the storage model of timezone-aware data is different compared to stdlib python, we are always storing the UTC timestamp (and not the localized timestamp).
   
   > In any case, this is mostly a documentation issue. If the timestamp value is always in UTC, then "year" and friends will return the year expressed in the UTC timezone. I don't think there is any reason to forbid the operation, just document it properly.
   
   I personally disagree with this. If you have a timezone-aware timestamp, as a user I would expect to get the result in *local* time. If I want to have the UTC version, I can easily first convert to UTC (which is a no-op). 
   For "year" it most of the time doesn't change much (only the hours around New Year), but if I have a timestamp "2021-06-01 09:00:00+02:00" (CEST), and I ask for "hour", I expect to get 9 and not 7.




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

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



[GitHub] [arrow] rok commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   Thanks for the review @pitrou! I've addressed some of the comments and I'll try to finish the rest today.


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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -481,5 +481,184 @@ ARROW_EXPORT
 Result<Datum> IfElse(const Datum& cond, const Datum& left, const Datum& right,
                      ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month for each element of `values`.
+/// Month is encoded as January=1, December=12
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day number for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns number of the day of the week value for each element of
+/// `values`. Week starts on Monday denoted by 0 and ends on Sunday denoted by 6.
+///
+/// \param[in] values input to extract number of the day of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns number of day of the year for each element of `values`.
+/// January 1st maps to day number 1, February 1st to 32, etc.
+///
+/// \param[in] values input to extract number of day of the year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOYear returns ISO year number for each element of `values`.
+/// First week of an ISO year has the majority (4 or more) of it's days in January.
+///
+/// \param[in] values input to extract ISO year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> ISOYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOWeek returns ISO week of year number for each element of `values`.
+/// First ISO week has the majority (4 or more) of it's days in January.
+/// Week of the year starts with 1 and can run up to 53.
+///
+/// \param[in] values input to extract ISO week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> ISOWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOCalendar returns a (ISO year, ISO week, weekday) struct for each element of

Review comment:
       `Day of week`?




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

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



[GitHub] [arrow] rok commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   > General comment: the PR is quite big, and the timezone-related logic further added complexity. So from a workflow perspective (to ensure reviewing / getting this merged is manageable), could it make sense to keep the timezone-related logic for a separate PR? (didn't check the code to see whether that's actually feasible)
   
   Keeping tz-aware logic would be fairly easy to move to a separate PR. I'll move it out.


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

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



[GitHub] [arrow] pitrou commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,632 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::local_days;
+using arrow_vendored::date::locate_zone;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+template <typename Duration>
+inline year_month_day ymd_caster_template(const int64_t data) {
+  return year_month_day(floor<days>(sys_time<Duration>(Duration{data})));
+}
+
+template <typename Duration>
+inline std::function<year_month_day(const int64_t)> ymd_caster_zoned_template(
+    const std::string timezone) {
+  static const arrow_vendored::date::time_zone* tz = locate_zone(timezone);
+  return [](const int64_t data) {
+    return year_month_day(floor<days>(tz->to_local(sys_time<Duration>(Duration{data}))));
+  };
+}
+
+inline std::function<year_month_day(const int64_t)> make_ymd_caster(
+    const std::shared_ptr<DataType> type) {
+  const auto ts_type = std::static_pointer_cast<const TimestampType>(type);
+  const TimeUnit::type unit = ts_type->unit();
+  const std::string timezone = ts_type->timezone();
+
+  if (timezone.empty()) {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return ymd_caster_template<std::chrono::seconds>;
+      case TimeUnit::MILLI:
+        return ymd_caster_template<std::chrono::milliseconds>;
+      case TimeUnit::MICRO:
+        return ymd_caster_template<std::chrono::microseconds>;
+      case TimeUnit::NANO:
+        return ymd_caster_template<std::chrono::nanoseconds>;
+    }
+  } else {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return ymd_caster_zoned_template<std::chrono::seconds>(timezone);
+      case TimeUnit::MILLI:
+        return ymd_caster_zoned_template<std::chrono::milliseconds>(timezone);
+      case TimeUnit::MICRO:
+        return ymd_caster_zoned_template<std::chrono::microseconds>(timezone);
+      case TimeUnit::NANO:
+        return ymd_caster_zoned_template<std::chrono::nanoseconds>(timezone);
+    }
+  }
+  return ymd_caster_template<std::chrono::seconds>;
+}
+
+template <typename DurationIn, typename DurationOut>
+inline hh_mm_ss<DurationOut> hhmmss_caster_template(const int64_t data) {
+  DurationIn t = DurationIn{data};
+  return hh_mm_ss<DurationOut>(
+      std::chrono::duration_cast<DurationOut>(t - floor<days>(t)));
+}
+
+template <typename DurationIn, typename DurationOut>
+inline std::function<hh_mm_ss<DurationOut>(const int64_t)> hhmmss_caster_zoned_template(
+    const std::string timezone) {
+  static const arrow_vendored::date::time_zone* tz = locate_zone(timezone);
+  return [](const int64_t data) {
+    const auto z = sys_time<DurationIn>(DurationIn{data});
+    const auto l = make_zoned(tz, z).get_local_time();
+    return hh_mm_ss<DurationOut>(
+        std::chrono::duration_cast<DurationOut>(l - floor<days>(l)));
+  };
+}
+
+template <typename Duration>
+inline std::function<hh_mm_ss<Duration>(const int64_t)> make_hhmmss_caster(
+    const std::shared_ptr<DataType> type) {
+  const auto ts_type = std::static_pointer_cast<const TimestampType>(type);
+  const TimeUnit::type unit = ts_type->unit();
+  const std::string timezone = ts_type->timezone();
+
+  if (timezone.empty()) {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return hhmmss_caster_template<std::chrono::seconds, Duration>;
+      case TimeUnit::MILLI:
+        return hhmmss_caster_template<std::chrono::milliseconds, Duration>;
+      case TimeUnit::MICRO:
+        return hhmmss_caster_template<std::chrono::microseconds, Duration>;
+      case TimeUnit::NANO:
+        return hhmmss_caster_template<std::chrono::nanoseconds, Duration>;
+    }
+  } else {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return hhmmss_caster_zoned_template<std::chrono::seconds, Duration>(timezone);
+      case TimeUnit::MILLI:
+        return hhmmss_caster_zoned_template<std::chrono::milliseconds, Duration>(
+            timezone);
+      case TimeUnit::MICRO:
+        return hhmmss_caster_zoned_template<std::chrono::microseconds, Duration>(
+            timezone);
+      case TimeUnit::NANO:
+        return hhmmss_caster_zoned_template<std::chrono::nanoseconds, Duration>(timezone);
+    }
+  }
+  return hhmmss_caster_template<std::chrono::seconds, Duration>;
+}
+
+template <typename Duration>
+inline unsigned day_of_year_caster_template(const int64_t data) {
+  const auto sd = sys_days{floor<days>(Duration{data})};
+  const auto y = year_month_day(sd).year();
+  return static_cast<unsigned>((sd - sys_days(y / jan / 0)).count());
+}
+
+template <typename Duration>
+inline std::function<unsigned(const int64_t)> day_of_year_zoned_caster_template(
+    const std::string timezone) {
+  static const arrow_vendored::date::time_zone* tz = locate_zone(timezone);
+  return [](const int64_t data) {
+    auto ld =
+        year_month_day(floor<days>(tz->to_local(sys_time<Duration>(Duration{data}))));
+    return static_cast<unsigned>(
+        (local_days(ld) - local_days(ld.year() / jan / 1) + days{1}).count());
+  };
+}
+
+inline std::function<unsigned(const int64_t)> get_day_of_year_caster(
+    const std::shared_ptr<DataType> type) {
+  const auto ts_type = std::static_pointer_cast<const TimestampType>(type);
+  const TimeUnit::type unit = ts_type->unit();
+  const std::string timezone = ts_type->timezone();
+
+  if (timezone.empty()) {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return day_of_year_caster_template<std::chrono::seconds>;
+      case TimeUnit::MILLI:
+        return day_of_year_caster_template<std::chrono::milliseconds>;
+      case TimeUnit::MICRO:
+        return day_of_year_caster_template<std::chrono::microseconds>;
+      case TimeUnit::NANO:
+        return day_of_year_caster_template<std::chrono::nanoseconds>;
+    }
+  } else {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return day_of_year_zoned_caster_template<std::chrono::seconds>(timezone);
+      case TimeUnit::MILLI:
+        return day_of_year_zoned_caster_template<std::chrono::milliseconds>(timezone);
+      case TimeUnit::MICRO:
+        return day_of_year_zoned_caster_template<std::chrono::microseconds>(timezone);
+      case TimeUnit::NANO:
+        return day_of_year_zoned_caster_template<std::chrono::nanoseconds>(timezone);
+    }
+  }
+  return day_of_year_caster_template<std::chrono::seconds>;
+}
+
+template <typename Duration>
+inline unsigned week_caster_template(const int64_t data) {
+  // Based on
+  // https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+  const auto dp = sys_days{floor<days>(Duration{data})};
+  auto y = year_month_day{dp + days{3}}.year();
+  auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+  if (dp < start) {
+    --y;
+    start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+  }
+  return static_cast<unsigned>(trunc<weeks>(dp - start).count() + 1);
+}
+
+template <typename Duration>
+inline std::function<unsigned(const int64_t)> week_zoned_caster_template(
+    const std::string timezone) {
+  static const arrow_vendored::date::time_zone* tz = locate_zone(timezone);
+  return [](const int64_t data) {
+    const auto ld = floor<days>(tz->to_local(sys_time<Duration>(Duration{data})));
+    auto y = year_month_day{ld + days{3}}.year();
+    auto start = local_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (ld < start) {
+      --y;
+      start = local_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<unsigned>(trunc<weeks>(local_days(ld) - start).count() + 1);
+  };
+}
+
+inline std::function<unsigned(const int64_t)> make_week_caster(
+    const std::shared_ptr<DataType> type) {
+  const auto ts_type = std::static_pointer_cast<const TimestampType>(type);
+  const TimeUnit::type unit = ts_type->unit();
+  const std::string timezone = ts_type->timezone();
+
+  if (timezone.empty()) {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return week_caster_template<std::chrono::seconds>;
+      case TimeUnit::MILLI:
+        return week_caster_template<std::chrono::milliseconds>;
+      case TimeUnit::MICRO:
+        return week_caster_template<std::chrono::microseconds>;
+      case TimeUnit::NANO:
+        return week_caster_template<std::chrono::nanoseconds>;
+    }
+  } else {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return week_zoned_caster_template<std::chrono::seconds>(timezone);
+      case TimeUnit::MILLI:
+        return week_zoned_caster_template<std::chrono::milliseconds>(timezone);
+      case TimeUnit::MICRO:
+        return week_zoned_caster_template<std::chrono::microseconds>(timezone);
+      case TimeUnit::NANO:
+        return week_zoned_caster_template<std::chrono::nanoseconds>(timezone);
+    }
+  }
+  return day_of_year_caster_template<std::chrono::seconds>;
+}
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename out_type>
+struct Year {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const auto& in_data = internal::UnboxScalar<const TimestampType>::Unbox(in);
+    auto ymd_caster = make_ymd_caster(in.type);
+    checked_cast<Int64Scalar*>(out)->value = static_cast<int>(ymd_caster(in_data).year());
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    auto in_data = in.GetValues<uint64_t>(1);
+    auto out_data = out->GetMutableValues<out_type>(1);
+    auto ymd_caster = make_ymd_caster(in.type);
+    for (int64_t i = 0; i < in.length; i++) {
+      out_data[i] = static_cast<int>(ymd_caster(in_data[i]).year());
+    }
+    return Status::OK();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename out_type>
+struct Month {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const auto& in_data = internal::UnboxScalar<const TimestampType>::Unbox(in);
+    auto ymd_caster = make_ymd_caster(in.type);
+    checked_cast<Int64Scalar*>(out)->value =
+        static_cast<unsigned>(ymd_caster(in_data).month());
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    auto in_data = in.GetValues<uint64_t>(1);
+    auto out_data = out->GetMutableValues<out_type>(1);
+    auto ymd_caster = make_ymd_caster(in.type);
+    for (int64_t i = 0; i < in.length; i++) {
+      out_data[i] = static_cast<unsigned>(ymd_caster(in_data[i]).month());
+    }
+    return Status::OK();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename out_type>
+struct Day {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const int64_t& in_data = internal::UnboxScalar<const TimestampType>::Unbox(in);
+    auto ymd_caster = make_ymd_caster(in.type);
+    checked_cast<Int64Scalar*>(out)->value =
+        static_cast<unsigned>(ymd_caster(in_data).day());
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    auto in_data = in.GetValues<uint64_t>(1);
+    auto out_data = out->GetMutableValues<out_type>(1);
+    auto ymd_caster = make_ymd_caster(in.type);
+    for (int64_t i = 0; i < in.length; i++) {
+      out_data[i] = static_cast<unsigned>(ymd_caster(in_data[i]).day());
+    }
+    return Status::OK();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract week from timestamp
+
+template <typename out_type>
+struct Week {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const auto& in_data = internal::UnboxScalar<const TimestampType>::Unbox(in);
+    auto week_caster = make_week_caster(in.type);
+    checked_cast<Int64Scalar*>(out)->value = week_caster(in_data);
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    auto in_data = in.GetValues<uint64_t>(1);
+    auto out_data = out->GetMutableValues<out_type>(1);
+    auto week_caster = make_week_caster(in.type);
+    for (int64_t i = 0; i < in.length; i++) {
+      out_data[i] = week_caster(in_data[i]);
+    }
+    return Status::OK();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract quarter from timestamp
+
+template <typename out_type>
+struct Quarter {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const auto& in_data = internal::UnboxScalar<const TimestampType>::Unbox(in);
+    auto ymd_caster = make_ymd_caster(in.type);
+    checked_cast<Int64Scalar*>(out)->value =
+        (static_cast<unsigned>(ymd_caster(in_data).month()) - 1) / 3 + 1;
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    auto in_data = in.GetValues<uint64_t>(1);
+    auto out_data = out->GetMutableValues<out_type>(1);
+    auto ymd_caster = make_ymd_caster(in.type);
+    for (int64_t i = 0; i < in.length; i++) {
+      out_data[i] = (static_cast<unsigned>(ymd_caster(in_data[i]).month()) - 1) / 3 + 1;
+    }
+    return Status::OK();
+  }
+};

Review comment:
       It seems ok to me. It's probably not very performant, since we're calling a `std::function` every time, but that should be ok for 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.

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,632 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::local_days;
+using arrow_vendored::date::locate_zone;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+template <typename Duration>
+inline year_month_day ymd_caster_template(const int64_t data) {
+  return year_month_day(floor<days>(sys_time<Duration>(Duration{data})));
+}
+
+template <typename Duration>
+inline std::function<year_month_day(const int64_t)> ymd_caster_zoned_template(
+    const std::string timezone) {
+  static const arrow_vendored::date::time_zone* tz = locate_zone(timezone);
+  return [](const int64_t data) {
+    return year_month_day(floor<days>(tz->to_local(sys_time<Duration>(Duration{data}))));
+  };
+}
+
+inline std::function<year_month_day(const int64_t)> make_ymd_caster(
+    const std::shared_ptr<DataType> type) {
+  const auto ts_type = std::static_pointer_cast<const TimestampType>(type);
+  const TimeUnit::type unit = ts_type->unit();
+  const std::string timezone = ts_type->timezone();
+
+  if (timezone.empty()) {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return ymd_caster_template<std::chrono::seconds>;
+      case TimeUnit::MILLI:
+        return ymd_caster_template<std::chrono::milliseconds>;
+      case TimeUnit::MICRO:
+        return ymd_caster_template<std::chrono::microseconds>;
+      case TimeUnit::NANO:
+        return ymd_caster_template<std::chrono::nanoseconds>;
+    }
+  } else {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return ymd_caster_zoned_template<std::chrono::seconds>(timezone);
+      case TimeUnit::MILLI:
+        return ymd_caster_zoned_template<std::chrono::milliseconds>(timezone);
+      case TimeUnit::MICRO:
+        return ymd_caster_zoned_template<std::chrono::microseconds>(timezone);
+      case TimeUnit::NANO:
+        return ymd_caster_zoned_template<std::chrono::nanoseconds>(timezone);
+    }
+  }
+  return ymd_caster_template<std::chrono::seconds>;
+}
+
+template <typename DurationIn, typename DurationOut>
+inline hh_mm_ss<DurationOut> hhmmss_caster_template(const int64_t data) {
+  DurationIn t = DurationIn{data};
+  return hh_mm_ss<DurationOut>(
+      std::chrono::duration_cast<DurationOut>(t - floor<days>(t)));
+}
+
+template <typename DurationIn, typename DurationOut>
+inline std::function<hh_mm_ss<DurationOut>(const int64_t)> hhmmss_caster_zoned_template(
+    const std::string timezone) {
+  static const arrow_vendored::date::time_zone* tz = locate_zone(timezone);
+  return [](const int64_t data) {
+    const auto z = sys_time<DurationIn>(DurationIn{data});
+    const auto l = make_zoned(tz, z).get_local_time();
+    return hh_mm_ss<DurationOut>(
+        std::chrono::duration_cast<DurationOut>(l - floor<days>(l)));
+  };
+}
+
+template <typename Duration>
+inline std::function<hh_mm_ss<Duration>(const int64_t)> make_hhmmss_caster(
+    const std::shared_ptr<DataType> type) {
+  const auto ts_type = std::static_pointer_cast<const TimestampType>(type);
+  const TimeUnit::type unit = ts_type->unit();
+  const std::string timezone = ts_type->timezone();
+
+  if (timezone.empty()) {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return hhmmss_caster_template<std::chrono::seconds, Duration>;
+      case TimeUnit::MILLI:
+        return hhmmss_caster_template<std::chrono::milliseconds, Duration>;
+      case TimeUnit::MICRO:
+        return hhmmss_caster_template<std::chrono::microseconds, Duration>;
+      case TimeUnit::NANO:
+        return hhmmss_caster_template<std::chrono::nanoseconds, Duration>;
+    }
+  } else {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return hhmmss_caster_zoned_template<std::chrono::seconds, Duration>(timezone);
+      case TimeUnit::MILLI:
+        return hhmmss_caster_zoned_template<std::chrono::milliseconds, Duration>(
+            timezone);
+      case TimeUnit::MICRO:
+        return hhmmss_caster_zoned_template<std::chrono::microseconds, Duration>(
+            timezone);
+      case TimeUnit::NANO:
+        return hhmmss_caster_zoned_template<std::chrono::nanoseconds, Duration>(timezone);
+    }
+  }
+  return hhmmss_caster_template<std::chrono::seconds, Duration>;
+}
+
+template <typename Duration>
+inline unsigned day_of_year_caster_template(const int64_t data) {
+  const auto sd = sys_days{floor<days>(Duration{data})};
+  const auto y = year_month_day(sd).year();
+  return static_cast<unsigned>((sd - sys_days(y / jan / 0)).count());
+}
+
+template <typename Duration>
+inline std::function<unsigned(const int64_t)> day_of_year_zoned_caster_template(
+    const std::string timezone) {
+  static const arrow_vendored::date::time_zone* tz = locate_zone(timezone);
+  return [](const int64_t data) {
+    auto ld =
+        year_month_day(floor<days>(tz->to_local(sys_time<Duration>(Duration{data}))));
+    return static_cast<unsigned>(
+        (local_days(ld) - local_days(ld.year() / jan / 1) + days{1}).count());
+  };
+}
+
+inline std::function<unsigned(const int64_t)> get_day_of_year_caster(
+    const std::shared_ptr<DataType> type) {
+  const auto ts_type = std::static_pointer_cast<const TimestampType>(type);
+  const TimeUnit::type unit = ts_type->unit();
+  const std::string timezone = ts_type->timezone();
+
+  if (timezone.empty()) {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return day_of_year_caster_template<std::chrono::seconds>;
+      case TimeUnit::MILLI:
+        return day_of_year_caster_template<std::chrono::milliseconds>;
+      case TimeUnit::MICRO:
+        return day_of_year_caster_template<std::chrono::microseconds>;
+      case TimeUnit::NANO:
+        return day_of_year_caster_template<std::chrono::nanoseconds>;
+    }
+  } else {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return day_of_year_zoned_caster_template<std::chrono::seconds>(timezone);
+      case TimeUnit::MILLI:
+        return day_of_year_zoned_caster_template<std::chrono::milliseconds>(timezone);
+      case TimeUnit::MICRO:
+        return day_of_year_zoned_caster_template<std::chrono::microseconds>(timezone);
+      case TimeUnit::NANO:
+        return day_of_year_zoned_caster_template<std::chrono::nanoseconds>(timezone);
+    }
+  }
+  return day_of_year_caster_template<std::chrono::seconds>;
+}
+
+template <typename Duration>
+inline unsigned week_caster_template(const int64_t data) {
+  // Based on
+  // https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+  const auto dp = sys_days{floor<days>(Duration{data})};
+  auto y = year_month_day{dp + days{3}}.year();
+  auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+  if (dp < start) {
+    --y;
+    start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+  }
+  return static_cast<unsigned>(trunc<weeks>(dp - start).count() + 1);
+}
+
+template <typename Duration>
+inline std::function<unsigned(const int64_t)> week_zoned_caster_template(
+    const std::string timezone) {
+  static const arrow_vendored::date::time_zone* tz = locate_zone(timezone);
+  return [](const int64_t data) {
+    const auto ld = floor<days>(tz->to_local(sys_time<Duration>(Duration{data})));
+    auto y = year_month_day{ld + days{3}}.year();
+    auto start = local_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (ld < start) {
+      --y;
+      start = local_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<unsigned>(trunc<weeks>(local_days(ld) - start).count() + 1);
+  };
+}
+
+inline std::function<unsigned(const int64_t)> make_week_caster(
+    const std::shared_ptr<DataType> type) {
+  const auto ts_type = std::static_pointer_cast<const TimestampType>(type);
+  const TimeUnit::type unit = ts_type->unit();
+  const std::string timezone = ts_type->timezone();
+
+  if (timezone.empty()) {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return week_caster_template<std::chrono::seconds>;
+      case TimeUnit::MILLI:
+        return week_caster_template<std::chrono::milliseconds>;
+      case TimeUnit::MICRO:
+        return week_caster_template<std::chrono::microseconds>;
+      case TimeUnit::NANO:
+        return week_caster_template<std::chrono::nanoseconds>;
+    }
+  } else {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return week_zoned_caster_template<std::chrono::seconds>(timezone);
+      case TimeUnit::MILLI:
+        return week_zoned_caster_template<std::chrono::milliseconds>(timezone);
+      case TimeUnit::MICRO:
+        return week_zoned_caster_template<std::chrono::microseconds>(timezone);
+      case TimeUnit::NANO:
+        return week_zoned_caster_template<std::chrono::nanoseconds>(timezone);
+    }
+  }
+  return day_of_year_caster_template<std::chrono::seconds>;
+}
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename out_type>

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.

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -481,5 +481,184 @@ ARROW_EXPORT
 Result<Datum> IfElse(const Datum& cond, const Datum& left, const Datum& right,
                      ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month for each element of `values`.
+/// Month is encoded as January=1, December=12
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day number for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns number of the day of the week value for each element of
+/// `values`. Week starts on Monday denoted by 0 and ends on Sunday denoted by 6.
+///
+/// \param[in] values input to extract number of the day of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns number of day of the year for each element of `values`.
+/// January 1st maps to day number 1, February 1st to 32, etc.
+///
+/// \param[in] values input to extract number of day of the year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOYear returns ISO year number for each element of `values`.
+/// First week of an ISO year has the majority (4 or more) of it's days in January.
+///
+/// \param[in] values input to extract ISO year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> ISOYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOWeek returns ISO week of year number for each element of `values`.
+/// First ISO week has the majority (4 or more) of it's days in January.
+/// Week of the year starts with 1 and can run up to 53.
+///
+/// \param[in] values input to extract ISO week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> ISOWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOCalendar returns a (ISO year, ISO week, weekday) struct for each element of
+/// `values`
+///
+/// \param[in] values input to ISO calendar struct from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> ISOCalendar(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns the quarter of year number for each element of `values`
+/// First quarter maps to 1 and forth quarter maps to 4.
+///
+/// \param[in] values input to extract quarter of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Quarter(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Hour returns hour value for each element of `values`
+///
+/// \param[in] values input to extract hour from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Hour(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Minute returns minutes value for each element of `values`
+///
+/// \param[in] values input to extract minutes from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Minute(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Second returns seconds value for each element of `values`
+///
+/// \param[in] values input to extract seconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Second(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Millisecond returns milliseconds value for each element of `values`
+///
+/// \param[in] values input to extract milliseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Millisecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Microsecond returns microseconds value for each element of `values`

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.

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,348 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+        .iso_encoding();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return (sd - sys_days(year_month_day(sd).year() / jan / 0)).count();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct Week {

Review comment:
       We should probably be explicit here about it being the "ISO week" number (and not the "logical" week number), as it gives some surprising results around New Year (https://en.wikipedia.org/wiki/ISO_week_date#First_week)
   
   (and if adding a ISO week, we should maybe also add the corresponding ISO year)

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -450,5 +450,145 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Week returns week of year value for each element of `values`
+///
+/// \param[in] values input to extract week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Week(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns quarter of year value for each element of `values`
+///
+/// \param[in] values input to extract quarter of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Quarter(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Hour returns hour value for each element of `values`
+///
+/// \param[in] values input to extract hour from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Hour(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Minute returns minutes value for each element of `values`
+///
+/// \param[in] values input to extract minutes from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Minute(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Second returns seconds value for each element of `values`
+///
+/// \param[in] values input to extract seconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Second(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Millisecond returns milliseconds value for each element of `values`
+///
+/// \param[in] values input to extract milliseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Millisecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Microsecond returns microseconds value for each element of `values`
+///
+/// \param[in] values input to extract microseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Microsecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Nanosecond returns nanoseconds value for each element of `values`
+///
+/// \param[in] values input to extract nanoseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Nanosecond(const Datum& values, ExecContext* ctx = NULLPTR);

Review comment:
       Are this the total nanoseconds, or only the nanoseconds part after the microseconds? (i.e. are microseconds and nanoseconds additive components, or rather both representing the full fractional part of the second using a different resolution?)

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,348 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+        .iso_encoding();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return (sd - sys_days(year_month_day(sd).year() / jan / 0)).count();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct Week {

Review comment:
       A struct? (with year, week, day fields)

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -450,5 +450,145 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Week returns week of year value for each element of `values`
+///
+/// \param[in] values input to extract week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Week(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns quarter of year value for each element of `values`
+///
+/// \param[in] values input to extract quarter of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Quarter(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Hour returns hour value for each element of `values`
+///
+/// \param[in] values input to extract hour from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Hour(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Minute returns minutes value for each element of `values`
+///
+/// \param[in] values input to extract minutes from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Minute(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Second returns seconds value for each element of `values`
+///
+/// \param[in] values input to extract seconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Second(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Millisecond returns milliseconds value for each element of `values`
+///
+/// \param[in] values input to extract milliseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Millisecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Microsecond returns microseconds value for each element of `values`
+///
+/// \param[in] values input to extract microseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Microsecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Nanosecond returns nanoseconds value for each element of `values`
+///
+/// \param[in] values input to extract nanoseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Nanosecond(const Datum& values, ExecContext* ctx = NULLPTR);

Review comment:
       Seeing the tests, it's clearly the first (additive components), which I think is good. I am only wondering if we can make that clearer in the terminology of the docstring

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -450,5 +450,145 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Week returns week of year value for each element of `values`
+///
+/// \param[in] values input to extract week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Week(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns quarter of year value for each element of `values`
+///
+/// \param[in] values input to extract quarter of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Quarter(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Hour returns hour value for each element of `values`
+///
+/// \param[in] values input to extract hour from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Hour(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Minute returns minutes value for each element of `values`
+///
+/// \param[in] values input to extract minutes from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Minute(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Second returns seconds value for each element of `values`
+///
+/// \param[in] values input to extract seconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Second(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Millisecond returns milliseconds value for each element of `values`
+///
+/// \param[in] values input to extract milliseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Millisecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Microsecond returns microseconds value for each element of `values`
+///
+/// \param[in] values input to extract microseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Microsecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Nanosecond returns nanoseconds value for each element of `values`
+///
+/// \param[in] values input to extract nanoseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Nanosecond(const Datum& values, ExecContext* ctx = NULLPTR);

Review comment:
       Seeing the tests, it's clearly the first (additive components), which I think is good (and the most useful). I am only wondering if we can make that clearer in the terminology of the docstring

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -450,5 +450,145 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Week returns week of year value for each element of `values`
+///
+/// \param[in] values input to extract week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Week(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns quarter of year value for each element of `values`
+///
+/// \param[in] values input to extract quarter of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Quarter(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Hour returns hour value for each element of `values`
+///
+/// \param[in] values input to extract hour from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Hour(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Minute returns minutes value for each element of `values`
+///
+/// \param[in] values input to extract minutes from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Minute(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Second returns seconds value for each element of `values`
+///
+/// \param[in] values input to extract seconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Second(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Millisecond returns milliseconds value for each element of `values`
+///
+/// \param[in] values input to extract milliseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Millisecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Microsecond returns microseconds value for each element of `values`
+///
+/// \param[in] values input to extract microseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Microsecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Nanosecond returns nanoseconds value for each element of `values`
+///
+/// \param[in] values input to extract nanoseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Nanosecond(const Datum& values, ExecContext* ctx = NULLPTR);

Review comment:
       To be clear, judging the tests, the current implementation is not using the "total" nanoseconds right now, but only the part after the microseconds. Personally I think this is the useful behaviour

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -450,5 +450,145 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Week returns week of year value for each element of `values`
+///
+/// \param[in] values input to extract week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Week(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns quarter of year value for each element of `values`
+///
+/// \param[in] values input to extract quarter of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Quarter(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Hour returns hour value for each element of `values`
+///
+/// \param[in] values input to extract hour from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Hour(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Minute returns minutes value for each element of `values`
+///
+/// \param[in] values input to extract minutes from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Minute(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Second returns seconds value for each element of `values`
+///
+/// \param[in] values input to extract seconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Second(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Millisecond returns milliseconds value for each element of `values`
+///
+/// \param[in] values input to extract milliseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Millisecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Microsecond returns microseconds value for each element of `values`
+///
+/// \param[in] values input to extract microseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Microsecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Nanosecond returns nanoseconds value for each element of `values`
+///
+/// \param[in] values input to extract nanoseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Nanosecond(const Datum& values, ExecContext* ctx = NULLPTR);

Review comment:
       Yeah, I was also looking at pandas to compare ;) And that was the reason I am asking.
   
   So the situation is a bit complex there:
   
   ```
   >>> ts = pd.Timestamp("2012-01-02 03:04:05.123456789")
   >>> ts.second
   5
   ```
   
   There is a `microsecond` attribute which gives the total microseconds, but this is inherited from `datetime.datetime` (as this actually stores the sub-second part as microseconds):
   
   ```
   >>> ts.microsecond
   123456
   ```
   
   and then there is also a `nanosecond` attribute which is only the part after the microseconds (so the "additive componenent", not total nanoseconds):
   ```
   >>> ts.nanosecond
   789
   ```
   
   
   (the example in the pandas docs is indeed not good! As you can't tell which of the two cases it is because of the used value)

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -450,5 +450,145 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Week returns week of year value for each element of `values`
+///
+/// \param[in] values input to extract week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Week(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns quarter of year value for each element of `values`
+///
+/// \param[in] values input to extract quarter of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Quarter(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Hour returns hour value for each element of `values`
+///
+/// \param[in] values input to extract hour from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Hour(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Minute returns minutes value for each element of `values`
+///
+/// \param[in] values input to extract minutes from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Minute(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Second returns seconds value for each element of `values`
+///
+/// \param[in] values input to extract seconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Second(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Millisecond returns milliseconds value for each element of `values`
+///
+/// \param[in] values input to extract milliseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Millisecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Microsecond returns microseconds value for each element of `values`
+///
+/// \param[in] values input to extract microseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Microsecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Nanosecond returns nanoseconds value for each element of `values`
+///
+/// \param[in] values input to extract nanoseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Nanosecond(const Datum& values, ExecContext* ctx = NULLPTR);

Review comment:
       We don't need to match exactly what pandas does to be clear. pyarrow.compute should just follow Arrow's behaviour.
   
   So for Arrow the question is, what is most useful: the total units vs the component? (or do we need both?)
   
   As comparison, the R lubridate package only has component getter functions up to `second()`, and not for `milli/micro/..second`, but the `second` function returns the fractional part of the second as well:
   
   ```R
   > library(lubridate)
   > x <- ymd_hms("2012-03-26 01:02:03.123456")
   > second(x)
   [1] 3.123456
   ```
   
   
   




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -462,5 +462,177 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`

Review comment:
       Done.

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -462,5 +462,177 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`

Review comment:
       Done.

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -462,5 +462,177 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`

Review comment:
       Done.

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -462,5 +462,177 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`

Review comment:
       Done.

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -462,5 +462,177 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOYear returns ISO year value for each element of `values`

Review comment:
       Done.

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -462,5 +462,177 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOYear returns ISO year value for each element of `values`
+///
+/// \param[in] values input to extract ISO year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> ISOYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOWeek returns ISO week of year value for each element of `values`

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.

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



[GitHub] [arrow] pitrou commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,107 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestSimpleTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO);
+  auto timestamps = ArrayFromJSON(unit, times);
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("weekday", int64())});
+
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar = ArrayFromJSON(iso_calendar_type,
+                                    R"([{"iso_year": 1970, "iso_week": 1, "weekday": 4},
+                        {"iso_year": 2000, "iso_week": 9, "weekday": 2},
+                        {"iso_year": 1899, "iso_week": 52, "weekday": 7},
+                        {"iso_year": 2033, "iso_week": 20, "weekday": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59.123456789, 23.999999999, 20.001001001, 20.0, null]";
+  auto millisecond = "[123, 999, 1, 0, null]";
+  auto microsecond = "[456, 999, 1, 0, null]";
+  auto nanosecond = "[789, 999, 1, 0, null]";
+  auto subsecond = "[123456789, 999999999, 1001001, 0, null]";
+
+  CheckScalarUnary("year", unit, times, int64(), year);
+  CheckScalarUnary("month", unit, times, int64(), month);
+  CheckScalarUnary("day", unit, times, int64(), day);
+  CheckScalarUnary("day_of_week", unit, times, int64(), day_of_week);
+  CheckScalarUnary("day_of_year", unit, times, int64(), day_of_year);
+  CheckScalarUnary("iso_year", unit, times, int64(), iso_year);
+  CheckScalarUnary("iso_week", unit, times, int64(), iso_week);
+  CheckScalarUnary("iso_calendar", timestamps, iso_calendar);
+  CheckScalarUnary("quarter", unit, times, int64(), quarter);
+  CheckScalarUnary("hour", unit, times, int64(), hour);
+  CheckScalarUnary("minute", unit, times, int64(), minute);
+  CheckScalarUnary("second", unit, times, float64(), second);
+  CheckScalarUnary("millisecond", unit, times, int64(), millisecond);
+  CheckScalarUnary("microsecond", unit, times, int64(), microsecond);
+  CheckScalarUnary("nanosecond", unit, times, int64(), nanosecond);
+  CheckScalarUnary("subsecond", unit, times, int64(), subsecond);
+}
+
+TEST(ScalarTemporalTest, TestZonedTemporalComponentExtraction) {
+  std::string timezone = "Etc/UTC-2";
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO, timezone);
+  auto timestamps = ArrayFromJSON(unit, times);
+
+  ASSERT_RAISES(Invalid, Year(timestamps));
+  ASSERT_RAISES(Invalid, Month(timestamps));
+  ASSERT_RAISES(Invalid, Day(timestamps));
+  ASSERT_RAISES(Invalid, DayOfWeek(timestamps));
+  ASSERT_RAISES(Invalid, DayOfYear(timestamps));
+  ASSERT_RAISES(Invalid, ISOYear(timestamps));
+  ASSERT_RAISES(Invalid, ISOWeek(timestamps));
+  ASSERT_RAISES(Invalid, ISOCalendar(timestamps));
+  ASSERT_RAISES(Invalid, Quarter(timestamps));
+  ASSERT_RAISES(Invalid, Hour(timestamps));
+  ASSERT_RAISES(Invalid, Minute(timestamps));
+  ASSERT_RAISES(Invalid, Second(timestamps));
+  ASSERT_RAISES(Invalid, Millisecond(timestamps));
+  ASSERT_RAISES(Invalid, Microsecond(timestamps));
+  ASSERT_RAISES(Invalid, Nanosecond(timestamps));
+  ASSERT_RAISES(Invalid, Subsecond(timestamps));

Review comment:
       Hmm, that makes sense.




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -450,5 +450,145 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Week returns week of year value for each element of `values`
+///
+/// \param[in] values input to extract week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Week(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns quarter of year value for each element of `values`
+///
+/// \param[in] values input to extract quarter of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Quarter(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Hour returns hour value for each element of `values`
+///
+/// \param[in] values input to extract hour from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Hour(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Minute returns minutes value for each element of `values`
+///
+/// \param[in] values input to extract minutes from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Minute(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Second returns seconds value for each element of `values`
+///
+/// \param[in] values input to extract seconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Second(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Millisecond returns milliseconds value for each element of `values`
+///
+/// \param[in] values input to extract milliseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Millisecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Microsecond returns microseconds value for each element of `values`
+///
+/// \param[in] values input to extract microseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Microsecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Nanosecond returns nanoseconds value for each element of `values`
+///
+/// \param[in] values input to extract nanoseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Nanosecond(const Datum& values, ExecContext* ctx = NULLPTR);

Review comment:
       Do you know by hear what does pandas do? [Documentation](https://pandas.pydata.org/docs/reference/api/pandas.Series.dt.microsecond.html) is not clear.
   Perhaps it would be good to then have a `subseconds` function that would return the total nanoseconds since midnight?

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -450,5 +450,145 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Week returns week of year value for each element of `values`
+///
+/// \param[in] values input to extract week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Week(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns quarter of year value for each element of `values`
+///
+/// \param[in] values input to extract quarter of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Quarter(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Hour returns hour value for each element of `values`
+///
+/// \param[in] values input to extract hour from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Hour(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Minute returns minutes value for each element of `values`
+///
+/// \param[in] values input to extract minutes from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Minute(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Second returns seconds value for each element of `values`
+///
+/// \param[in] values input to extract seconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Second(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Millisecond returns milliseconds value for each element of `values`
+///
+/// \param[in] values input to extract milliseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Millisecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Microsecond returns microseconds value for each element of `values`
+///
+/// \param[in] values input to extract microseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Microsecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Nanosecond returns nanoseconds value for each element of `values`
+///
+/// \param[in] values input to extract nanoseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Nanosecond(const Datum& values, ExecContext* ctx = NULLPTR);

Review comment:
       Do you know by heart what does pandas do? [Documentation](https://pandas.pydata.org/docs/reference/api/pandas.Series.dt.microsecond.html) is not clear.
   Perhaps it would be good to then have a `subseconds` function that would return the total nanoseconds since midnight?




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

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



[GitHub] [arrow] rok commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   @pitrou It seems kernels do not match on units of timestamps?
   [This test fails](https://github.com/apache/arrow/pull/10176/commits/efd6b9e746dadaaaa6d1cc69614e6145b30b8037#diff-19248dfd2de25c17622290aae9c78024ba474de2913e44bc8cf0dce0bfe7dcdaR107) for every unit except for nanosecond even though we [register kernels for all units](https://github.com/apache/arrow/pull/10176/commits/efd6b9e746dadaaaa6d1cc69614e6145b30b8037#diff-53850537a2f3f7977473b30241fdd3ddb35aa97269284c06471d9b0c599e2c59R415).
   Is this expected? We can get timestamp units at runtime but matching would be preferable I think.
   
   Other than that I still need to [add a couple of test times](https://github.com/apache/arrow/pull/10176#discussion_r643061679).


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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -462,5 +462,177 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOYear returns ISO year value for each element of `values`
+///
+/// \param[in] values input to extract ISO year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> ISOYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOWeek returns ISO week of year value for each element of `values`
+///
+/// \param[in] values input to extract ISO week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> ISOWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOCalendar returns a (year, ISO week, weekday) struct for each element of
+/// `values`
+///
+/// \param[in] values input to ISO calendar struct from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> ISOCalendar(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns quarter of year value for each element of `values`

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.

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: docs/source/cpp/compute.rst
##########
@@ -637,6 +637,55 @@ String extraction
   e.g. 'letter' and 'digit' for the regular expression
   ``(?P<letter>[ab])(?P<digit>\\d)``.
 
+Temporal component extraction
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+These functions extract datetime components (year, month, day, etc) from timestamp type.
+Note: timezone information is currently ignored if present.

Review comment:
       Updated.




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,151 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO);
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("day_of_week", int64())});
+
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar =
+      ArrayFromJSON(iso_calendar_type,
+                    R"([{"iso_year": 1970, "iso_week": 1, "day_of_week": 4},
+                        {"iso_year": 2000, "iso_week": 9, "day_of_week": 2},
+                        {"iso_year": 1899, "iso_week": 52, "day_of_week": 7},
+                        {"iso_year": 2033, "iso_week": 20, "day_of_week": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59.123456789, 23.999999999, 20.001001001, 20.0, null]";
+  auto millisecond = "[123, 999, 1, 0, null]";
+  auto microsecond = "[456, 999, 1, 0, null]";
+  auto nanosecond = "[789, 999, 1, 0, null]";
+  auto subsecond = "[123456789, 999999999, 1001001, 0, null]";

Review comment:
       I've changed this to float now as it is more intuitive indeed. Definition is now seconds since last full second.
   
   If it was an integer as it was before the definition would be: total nanoseconds since last full second.




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return static_cast<T>((sd - sys_days(year_month_day(sd).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day{sys_days{floor<days>(Duration{arg})} + days{3}}.year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto dp = sys_days{floor<days>(Duration{arg})};
+    auto y = year_month_day{dp + days{3}}.year();
+    auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (dp < start) {
+      --y;
+      start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(dp - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        std::chrono::duration<double>(t - floor<std::chrono::minutes>(t)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::seconds>(t)) /
+                          std::chrono::nanoseconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+template <typename Duration, typename OutType>
+struct ISOCalendar {
+  using T = typename OutType::c_type;
+
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    using ScalarType = typename TypeTraits<OutType>::ScalarType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();

Review comment:
       Done.

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return static_cast<T>((sd - sys_days(year_month_day(sd).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day{sys_days{floor<days>(Duration{arg})} + days{3}}.year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto dp = sys_days{floor<days>(Duration{arg})};
+    auto y = year_month_day{dp + days{3}}.year();
+    auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (dp < start) {
+      --y;
+      start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(dp - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        std::chrono::duration<double>(t - floor<std::chrono::minutes>(t)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::seconds>(t)) /
+                          std::chrono::nanoseconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+template <typename Duration, typename OutType>
+struct ISOCalendar {
+  using T = typename OutType::c_type;
+
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    using ScalarType = typename TypeTraits<OutType>::ScalarType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    if (!in.is_valid) {
+      out->is_valid = false;
+    } else {
+      const std::shared_ptr<DataType> iso_calendar_type =
+          struct_({field("iso_year", out_type), field("iso_week", out_type),
+                   field("weekday", out_type)});
+
+      const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+      const auto dp = sys_days{floor<days>(Duration{in_val})};
+      const auto ymd = year_month_day(dp);
+      auto y = year_month_day{dp + days{3}}.year();
+      auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      if (dp < start) {
+        --y;
+        start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      }

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.

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



[GitHub] [arrow] github-actions[bot] commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


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


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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,239 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestSimpleTemporalComponentExtraction) {
+  const char* json =
+      R"(["1970-01-01T00:00:59","2000-02-29T23:23:23",
+          "3989-07-14T18:04:01","1900-01-01T01:59:20","2033-05-18T03:33:20"])";
+  auto time_points = ArrayFromJSON(timestamp(TimeUnit::SECOND), json);
+
+  auto year = ArrayFromJSON(int64(), "[1970, 2000, 3989, 1900, 2033]");
+  auto month = ArrayFromJSON(int64(), "[1, 2, 7, 1, 5]");
+  auto day = ArrayFromJSON(int64(), "[1, 29, 14, 1, 18]");
+  auto day_of_year = ArrayFromJSON(int64(), "[1, 60, 195, 1, 138]");
+  auto week = ArrayFromJSON(int64(), "[1, 9, 28, 1, 20]");
+  auto quarter = ArrayFromJSON(int64(), "[1, 1, 3, 1, 2]");
+  auto day_of_week = ArrayFromJSON(int64(), "[4, 2, 5, 1, 3]");
+  auto hour = ArrayFromJSON(int64(), "[0, 23, 18, 1, 3]");
+  auto minute = ArrayFromJSON(int64(), "[0, 23, 4, 59, 33]");
+  auto second = ArrayFromJSON(int64(), "[59, 23, 1, 20, 20]");
+  auto millisecond = ArrayFromJSON(int64(), "[0, 0, 0, 0, 0]");
+  auto microsecond = ArrayFromJSON(int64(), "[0, 0, 0, 0, 0]");
+  auto nanosecond = ArrayFromJSON(int64(), "[0, 0, 0, 0, 0]");
+
+  ASSERT_OK_AND_ASSIGN(Datum actual_year, Year(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_month, Month(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_day, Day(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_day_of_year, DayOfYear(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_week, Week(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_quarter, Quarter(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_day_of_week, DayOfWeek(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_hour, Hour(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_minute, Minute(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_second, Second(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_millisecond, Millisecond(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_microsecond, Microsecond(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_nanosecond, Nanosecond(time_points));
+
+  ASSERT_EQ(actual_year, year);
+  ASSERT_EQ(actual_month, month);
+  ASSERT_EQ(actual_day, day);
+  ASSERT_EQ(actual_day_of_year, day_of_year);
+  ASSERT_EQ(actual_week, week);
+  ASSERT_EQ(actual_quarter, quarter);
+  ASSERT_EQ(actual_day_of_week, day_of_week);
+  ASSERT_EQ(actual_hour, hour);
+  ASSERT_EQ(actual_minute, minute);
+  ASSERT_EQ(actual_second, second);
+  ASSERT_EQ(actual_millisecond, millisecond);
+  ASSERT_EQ(actual_microsecond, microsecond);
+  ASSERT_EQ(actual_nanosecond, nanosecond);
+
+  CheckScalarUnary("year", time_points, year);
+  CheckScalarUnary("month", time_points, month);
+  CheckScalarUnary("day", time_points, day);
+  CheckScalarUnary("day_of_year", time_points, day_of_year);
+  CheckScalarUnary("week", time_points, week);
+  CheckScalarUnary("quarter", time_points, quarter);
+  CheckScalarUnary("day_of_week", time_points, day_of_week);
+  CheckScalarUnary("hour", time_points, hour);
+  CheckScalarUnary("minute", time_points, minute);
+  CheckScalarUnary("second", time_points, second);
+  CheckScalarUnary("millisecond", time_points, millisecond);
+  CheckScalarUnary("microsecond", time_points, microsecond);
+  CheckScalarUnary("nanosecond", time_points, nanosecond);
+}
+
+TEST(ScalarTemporalTest, TestTemporalComponentExtraction) {
+  const char* json_second = "[59, 951866603, -2208981640, 2000000000]";
+  const char* json_milli = "[59000, 951866603000, -2208981640000, 2000000000000]";
+  const char* json_micro =
+      "[59000000, 951866603000000, -2208981640000000, 2000000000000000]";
+  const char* json_nano =
+      "[59000000000, 951866603000000000, -2208981640000000000, 2000000000000000000]";
+
+  auto time_points_second = ArrayFromJSON(timestamp(TimeUnit::SECOND), json_second);
+  auto time_points_milli = ArrayFromJSON(timestamp(TimeUnit::MILLI), json_milli);
+  auto time_points_micro = ArrayFromJSON(timestamp(TimeUnit::MICRO), json_micro);
+  auto time_points_nano = ArrayFromJSON(timestamp(TimeUnit::NANO), json_nano);
+
+  auto year = ArrayFromJSON(int64(), "[1970, 2000, 1900, 2033]");
+  auto month = ArrayFromJSON(int64(), "[1, 2, 1, 5]");
+  auto day = ArrayFromJSON(int64(), "[1, 29, 1, 18]");
+  auto day_of_year = ArrayFromJSON(int64(), "[1, 60, 1, 138]");
+  auto week = ArrayFromJSON(int64(), "[1, 9, 1, 20]");
+  auto quarter = ArrayFromJSON(int64(), "[1, 1, 1, 2]");
+  auto day_of_week = ArrayFromJSON(int64(), "[4, 2, 1, 3]");
+  auto hour = ArrayFromJSON(int64(), "[0, 23, 1, 3]");
+  auto minute = ArrayFromJSON(int64(), "[0, 23, 59, 33]");
+  auto second = ArrayFromJSON(int64(), "[59, 23, 20, 20]");
+  auto millisecond = ArrayFromJSON(int64(), "[0, 0, 0, 0]");
+  auto microsecond = ArrayFromJSON(int64(), "[0, 0, 0, 0]");
+  auto nanosecond = ArrayFromJSON(int64(), "[0, 0, 0, 0]");
+
+  for (auto time_points :
+       {time_points_second, time_points_milli, time_points_micro, time_points_nano}) {
+    CheckScalarUnary("year", time_points, year);
+    CheckScalarUnary("month", time_points, month);
+    CheckScalarUnary("day", time_points, day);
+    CheckScalarUnary("day_of_year", time_points, day_of_year);
+    CheckScalarUnary("week", time_points, week);
+    CheckScalarUnary("quarter", time_points, quarter);
+    CheckScalarUnary("day_of_week", time_points, day_of_week);
+    CheckScalarUnary("hour", time_points, hour);
+    CheckScalarUnary("minute", time_points, minute);
+    CheckScalarUnary("second", time_points, second);
+    CheckScalarUnary("millisecond", time_points, millisecond);
+    CheckScalarUnary("microsecond", time_points, microsecond);
+    CheckScalarUnary("nanosecond", time_points, nanosecond);
+  }
+
+  std::string in = "[123, 999, 1, 31231000]";
+  auto out = ArrayFromJSON(int64(), "[123, 999, 1, 0]");
+
+  auto tp_milli = ArrayFromJSON(timestamp(TimeUnit::MILLI), in);
+  auto tp_milli_zoned = ArrayFromJSON(timestamp(TimeUnit::MILLI, "Etc/GMT+2"), in);
+  CheckScalarUnary("millisecond", tp_milli, out);
+  CheckScalarUnary("millisecond", tp_milli, out);
+
+  auto tp_micro = ArrayFromJSON(timestamp(TimeUnit::MICRO), in);
+  auto tp_micro_zoned = ArrayFromJSON(timestamp(TimeUnit::MICRO, "Etc/GMT+2"), in);
+  CheckScalarUnary("microsecond", tp_micro, out);
+  CheckScalarUnary("microsecond", tp_micro_zoned, out);
+
+  auto tp_nano = ArrayFromJSON(timestamp(TimeUnit::NANO), in);
+  auto tp_nano_zoned = ArrayFromJSON(timestamp(TimeUnit::NANO, "Etc/GMT+2"), in);
+  CheckScalarUnary("nanosecond", tp_nano, out);
+  CheckScalarUnary("nanosecond", tp_nano_zoned, out);
+}
+
+TEST(ScalarTemporalTest, TestSimpleZonedTemporalComponentExtraction) {
+  const char* json =
+      R"(["1970-01-01T00:00:59","2000-02-29T23:23:23",
+          "3989-07-14T18:04:01","1900-01-01T01:59:20","2033-05-18T03:33:20"])";
+  auto time_points = ArrayFromJSON(timestamp(TimeUnit::SECOND, "Etc/GMT+2"), json);
+
+  auto year = ArrayFromJSON(int64(), "[1969, 2000, 3989, 1899, 2033]");

Review comment:
       It turns out `Etc/GMT+X == UTC-X` so this was actually ok. Weird mapping.




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

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



[GitHub] [arrow] rok commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   @jorisvandenbossche @pitrou - other then the iso_calendar I feel this is ready for another review round.


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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(

Review comment:
       Similar discussion here for [weekaday](https://github.com/HowardHinnant/date/issues/391). Didn't find a nice way to fix it.
   This happens in many locations as date.h is quite strict with types. Will add comments.




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,107 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestSimpleTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO);
+  auto timestamps = ArrayFromJSON(unit, times);
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("weekday", int64())});
+
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar = ArrayFromJSON(iso_calendar_type,
+                                    R"([{"iso_year": 1970, "iso_week": 1, "weekday": 4},
+                        {"iso_year": 2000, "iso_week": 9, "weekday": 2},
+                        {"iso_year": 1899, "iso_week": 52, "weekday": 7},
+                        {"iso_year": 2033, "iso_week": 20, "weekday": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59.123456789, 23.999999999, 20.001001001, 20.0, null]";
+  auto millisecond = "[123, 999, 1, 0, null]";
+  auto microsecond = "[456, 999, 1, 0, null]";
+  auto nanosecond = "[789, 999, 1, 0, null]";
+  auto subsecond = "[123456789, 999999999, 1001001, 0, null]";
+
+  CheckScalarUnary("year", unit, times, int64(), year);
+  CheckScalarUnary("month", unit, times, int64(), month);
+  CheckScalarUnary("day", unit, times, int64(), day);
+  CheckScalarUnary("day_of_week", unit, times, int64(), day_of_week);
+  CheckScalarUnary("day_of_year", unit, times, int64(), day_of_year);
+  CheckScalarUnary("iso_year", unit, times, int64(), iso_year);
+  CheckScalarUnary("iso_week", unit, times, int64(), iso_week);
+  CheckScalarUnary("iso_calendar", timestamps, iso_calendar);
+  CheckScalarUnary("quarter", unit, times, int64(), quarter);
+  CheckScalarUnary("hour", unit, times, int64(), hour);
+  CheckScalarUnary("minute", unit, times, int64(), minute);
+  CheckScalarUnary("second", unit, times, float64(), second);
+  CheckScalarUnary("millisecond", unit, times, int64(), millisecond);
+  CheckScalarUnary("microsecond", unit, times, int64(), microsecond);
+  CheckScalarUnary("nanosecond", unit, times, int64(), nanosecond);
+  CheckScalarUnary("subsecond", unit, times, int64(), subsecond);

Review comment:
       We'll probably need to calculate these to mimic pandas and R. We can calculate them here or later. [See discussion.](https://github.com/apache/arrow/pull/10176#discussion_r633564079)




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

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,151 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO);
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("day_of_week", int64())});
+
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar =
+      ArrayFromJSON(iso_calendar_type,
+                    R"([{"iso_year": 1970, "iso_week": 1, "day_of_week": 4},
+                        {"iso_year": 2000, "iso_week": 9, "day_of_week": 2},
+                        {"iso_year": 1899, "iso_week": 52, "day_of_week": 7},
+                        {"iso_year": 2033, "iso_week": 20, "day_of_week": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59.123456789, 23.999999999, 20.001001001, 20.0, null]";
+  auto millisecond = "[123, 999, 1, 0, null]";
+  auto microsecond = "[456, 999, 1, 0, null]";
+  auto nanosecond = "[789, 999, 1, 0, null]";
+  auto subsecond = "[123456789, 999999999, 1001001, 0, null]";
+
+  CheckScalarUnary("year", unit, times, int64(), year);
+  CheckScalarUnary("month", unit, times, int64(), month);
+  CheckScalarUnary("day", unit, times, int64(), day);
+  CheckScalarUnary("day_of_week", unit, times, int64(), day_of_week);
+  CheckScalarUnary("day_of_year", unit, times, int64(), day_of_year);
+  CheckScalarUnary("iso_year", unit, times, int64(), iso_year);
+  CheckScalarUnary("iso_week", unit, times, int64(), iso_week);
+  CheckScalarUnary("iso_calendar", ArrayFromJSON(unit, times), iso_calendar);
+  CheckScalarUnary("quarter", unit, times, int64(), quarter);
+  CheckScalarUnary("hour", unit, times, int64(), hour);
+  CheckScalarUnary("minute", unit, times, int64(), minute);
+  CheckScalarUnary("second", unit, times, float64(), second);
+  CheckScalarUnary("millisecond", unit, times, int64(), millisecond);
+  CheckScalarUnary("microsecond", unit, times, int64(), microsecond);
+  CheckScalarUnary("nanosecond", unit, times, int64(), nanosecond);
+  CheckScalarUnary("subsecond", unit, times, int64(), subsecond);
+}
+
+TEST(ScalarTemporalTest, TestTemporalComponentExtractionWithDifferentUnits) {
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("day_of_week", int64())});
+  const char* times =
+      R"(["1970-01-01T00:00:59","2000-02-29T23:23:23",
+            "1899-01-01T00:59:20","2033-05-18T03:33:20", null])";
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar =
+      ArrayFromJSON(iso_calendar_type,
+                    R"([{"iso_year": 1970, "iso_week": 1, "day_of_week": 4},
+                          {"iso_year": 2000, "iso_week": 9, "day_of_week": 2},
+                          {"iso_year": 1899, "iso_week": 52, "day_of_week": 7},
+                          {"iso_year": 2033, "iso_week": 20, "day_of_week": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59, 23, 20, 20, null]";
+
+  //  for (auto u : internal::AllTimeUnits()) {
+  for (auto u : {TimeUnit::NANO}) {

Review comment:
       Ah, sorry, missed that comment.




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

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



[GitHub] [arrow] rok commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   @pitrou It seems kernels do not match on units of timestamps?
   [This test fails](https://github.com/apache/arrow/pull/10176/commits/efd6b9e746dadaaaa6d1cc69614e6145b30b8037#diff-19248dfd2de25c17622290aae9c78024ba474de2913e44bc8cf0dce0bfe7dcdaR107) for every unit except for nanosecond even though we [register kernels for all units](https://github.com/apache/arrow/pull/10176/commits/efd6b9e746dadaaaa6d1cc69614e6145b30b8037#diff-53850537a2f3f7977473b30241fdd3ddb35aa97269284c06471d9b0c599e2c59R415).
   Is this expected? We can get timestamp units at runtime but matching would be preferable I think.
   
   Other than that I still need to [add a couple of test times](https://github.com/apache/arrow/pull/10176#discussion_r643061679).


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

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



[GitHub] [arrow] rok removed a comment on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   > CI issues don't seem related.
   > I've also created a [followup Jira](https://issues.apache.org/jira/browse/ARROW-12980) for the timezone aware component extraction.
   
   


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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,107 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestSimpleTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO);
+  auto timestamps = ArrayFromJSON(unit, times);
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("weekday", int64())});
+
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar = ArrayFromJSON(iso_calendar_type,
+                                    R"([{"iso_year": 1970, "iso_week": 1, "weekday": 4},
+                        {"iso_year": 2000, "iso_week": 9, "weekday": 2},
+                        {"iso_year": 1899, "iso_week": 52, "weekday": 7},
+                        {"iso_year": 2033, "iso_week": 20, "weekday": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59.123456789, 23.999999999, 20.001001001, 20.0, null]";
+  auto millisecond = "[123, 999, 1, 0, null]";
+  auto microsecond = "[456, 999, 1, 0, null]";
+  auto nanosecond = "[789, 999, 1, 0, null]";
+  auto subsecond = "[123456789, 999999999, 1001001, 0, null]";
+
+  CheckScalarUnary("year", unit, times, int64(), year);
+  CheckScalarUnary("month", unit, times, int64(), month);
+  CheckScalarUnary("day", unit, times, int64(), day);
+  CheckScalarUnary("day_of_week", unit, times, int64(), day_of_week);
+  CheckScalarUnary("day_of_year", unit, times, int64(), day_of_year);
+  CheckScalarUnary("iso_year", unit, times, int64(), iso_year);
+  CheckScalarUnary("iso_week", unit, times, int64(), iso_week);
+  CheckScalarUnary("iso_calendar", timestamps, iso_calendar);
+  CheckScalarUnary("quarter", unit, times, int64(), quarter);
+  CheckScalarUnary("hour", unit, times, int64(), hour);
+  CheckScalarUnary("minute", unit, times, int64(), minute);
+  CheckScalarUnary("second", unit, times, float64(), second);
+  CheckScalarUnary("millisecond", unit, times, int64(), millisecond);
+  CheckScalarUnary("microsecond", unit, times, int64(), microsecond);
+  CheckScalarUnary("nanosecond", unit, times, int64(), nanosecond);
+  CheckScalarUnary("subsecond", unit, times, int64(), subsecond);
+}
+
+TEST(ScalarTemporalTest, TestZonedTemporalComponentExtraction) {
+  std::string timezone = "Etc/UTC-2";
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO, timezone);
+  auto timestamps = ArrayFromJSON(unit, times);
+
+  ASSERT_RAISES(Invalid, Year(timestamps));
+  ASSERT_RAISES(Invalid, Month(timestamps));
+  ASSERT_RAISES(Invalid, Day(timestamps));
+  ASSERT_RAISES(Invalid, DayOfWeek(timestamps));
+  ASSERT_RAISES(Invalid, DayOfYear(timestamps));
+  ASSERT_RAISES(Invalid, ISOYear(timestamps));
+  ASSERT_RAISES(Invalid, ISOWeek(timestamps));
+  ASSERT_RAISES(Invalid, ISOCalendar(timestamps));
+  ASSERT_RAISES(Invalid, Quarter(timestamps));
+  ASSERT_RAISES(Invalid, Hour(timestamps));
+  ASSERT_RAISES(Invalid, Minute(timestamps));
+  ASSERT_RAISES(Invalid, Second(timestamps));
+  ASSERT_RAISES(Invalid, Millisecond(timestamps));
+  ASSERT_RAISES(Invalid, Microsecond(timestamps));
+  ASSERT_RAISES(Invalid, Nanosecond(timestamps));
+  ASSERT_RAISES(Invalid, Subsecond(timestamps));

Review comment:
       Wouldn't ignoring be a bit unexpected? It would be relatively easy to add timezone processing now - [see here](https://github.com/apache/arrow/commit/36abbcb512b2b45c8b5615fe29074be2afbd01d8).




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,151 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO);
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("day_of_week", int64())});
+
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar =
+      ArrayFromJSON(iso_calendar_type,
+                    R"([{"iso_year": 1970, "iso_week": 1, "day_of_week": 4},
+                        {"iso_year": 2000, "iso_week": 9, "day_of_week": 2},
+                        {"iso_year": 1899, "iso_week": 52, "day_of_week": 7},
+                        {"iso_year": 2033, "iso_week": 20, "day_of_week": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59.123456789, 23.999999999, 20.001001001, 20.0, null]";
+  auto millisecond = "[123, 999, 1, 0, null]";
+  auto microsecond = "[456, 999, 1, 0, null]";
+  auto nanosecond = "[789, 999, 1, 0, null]";
+  auto subsecond = "[123456789, 999999999, 1001001, 0, null]";
+
+  CheckScalarUnary("year", unit, times, int64(), year);
+  CheckScalarUnary("month", unit, times, int64(), month);
+  CheckScalarUnary("day", unit, times, int64(), day);
+  CheckScalarUnary("day_of_week", unit, times, int64(), day_of_week);
+  CheckScalarUnary("day_of_year", unit, times, int64(), day_of_year);
+  CheckScalarUnary("iso_year", unit, times, int64(), iso_year);
+  CheckScalarUnary("iso_week", unit, times, int64(), iso_week);
+  CheckScalarUnary("iso_calendar", ArrayFromJSON(unit, times), iso_calendar);
+  CheckScalarUnary("quarter", unit, times, int64(), quarter);
+  CheckScalarUnary("hour", unit, times, int64(), hour);
+  CheckScalarUnary("minute", unit, times, int64(), minute);
+  CheckScalarUnary("second", unit, times, float64(), second);
+  CheckScalarUnary("millisecond", unit, times, int64(), millisecond);
+  CheckScalarUnary("microsecond", unit, times, int64(), microsecond);
+  CheckScalarUnary("nanosecond", unit, times, int64(), nanosecond);
+  CheckScalarUnary("subsecond", unit, times, int64(), subsecond);
+}
+
+TEST(ScalarTemporalTest, TestTemporalComponentExtractionWithDifferentUnits) {
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("day_of_week", int64())});
+  const char* times =
+      R"(["1970-01-01T00:00:59","2000-02-29T23:23:23",
+            "1899-01-01T00:59:20","2033-05-18T03:33:20", null])";
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar =
+      ArrayFromJSON(iso_calendar_type,
+                    R"([{"iso_year": 1970, "iso_week": 1, "day_of_week": 4},
+                          {"iso_year": 2000, "iso_week": 9, "day_of_week": 2},
+                          {"iso_year": 1899, "iso_week": 52, "day_of_week": 7},
+                          {"iso_year": 2033, "iso_week": 20, "day_of_week": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59, 23, 20, 20, null]";

Review comment:
       I've currently not added these because ArrayFromJSON fails when timestamps are more precise than their units. E.g.:
   ```
   CheckScalarUnary("millisecond", ArrayFromJSON(timestamp(TimeUnit::SECOND), R"(["1970-01-01T00:00:59.123"])"), ArrayFromJSON(float64(), "[0]"));
   ```
   Fails with:
   ```
   -- Arrow Fatal Error --
   Invalid: couldn't parse timestamp from 1970-01-01T00:00:59.123
   ```
   
   I'll think about how to do this nicely.




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

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,626 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+
+namespace internal {
+
+namespace {
+
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+using internal::applicator::ScalarUnaryNotNull;
+using internal::applicator::SimpleUnary;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename Op, typename OutType>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<Op, OutType>;
+  using OutValue = typename internal::GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          checked_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        internal::VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        checked_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = internal::UnboxScalar<Int64Type>::Unbox(arg0);
+        internal::BoxScalar<OutType>::Box(
+            this->op.template Call<OutValue>(ctx, arg0_val, &st), out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename Op, typename OutType>
+struct ScalarUnaryTemporal {
+  using OutValue = typename internal::GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<Op, OutType> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding() -
+        1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    return static_cast<T>(
+        (t - sys_time<days>(year_month_day(t).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+//
+// First week of an ISO year has the majority (4 or more) of it's days in January.
+// Last week of an ISO year has the year's last Thursday in it.
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    auto y = year_month_day{t + days{3}}.year();
+    auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (t < start) {
+      --y;
+    }
+    return static_cast<T>(static_cast<int32_t>(y));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+//
+// First week of an ISO year has the majority (4 or more) of it's days in January.
+// Last week of an ISO year has the year's last Thursday in it.
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    auto y = year_month_day{t + days{3}}.year();
+    auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (t < start) {
+      --y;
+      start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(t - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::minutes>(t)) / std::chrono::seconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        (std::chrono::duration<double>(t - floor<std::chrono::seconds>(t)).count()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+template <typename Duration>
+inline std::vector<int64_t> get_iso_calendar(int64_t arg) {
+  const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+  const auto ymd = year_month_day(t);
+  auto y = year_month_day{t + days{3}}.year();
+  auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+  if (t < start) {
+    --y;
+    start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+  }
+  return {static_cast<int64_t>(static_cast<int32_t>(y)),
+          static_cast<int64_t>(trunc<weeks>(t - start).count() + 1),
+          static_cast<int64_t>(weekday(ymd).iso_encoding())};
+}
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+template <typename Duration>
+struct ISOCalendar {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const std::string timezone =
+        checked_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    if (in.is_valid) {
+      const std::shared_ptr<DataType> iso_calendar_type =
+          struct_({field("iso_year", int64()), field("iso_week", int64()),
+                   field("day_of_week", int64())});
+      const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+      const auto iso_calendar = get_iso_calendar<Duration>(in_val);
+
+      std::vector<std::shared_ptr<Scalar>> values = {
+          std::make_shared<Int64Scalar>(iso_calendar[0]),
+          std::make_shared<Int64Scalar>(iso_calendar[1]),
+          std::make_shared<Int64Scalar>(iso_calendar[2])};
+      *checked_cast<StructScalar*>(out) = StructScalar(values, iso_calendar_type);
+    } else {
+      out->is_valid = false;
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    using BuilderType = typename TypeTraits<Int64Type>::BuilderType;
+    const std::string timezone =
+        checked_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+    const std::shared_ptr<DataType> iso_calendar_type =
+        struct_({field("iso_year", int64()), field("iso_week", int64()),
+                 field("day_of_week", int64())});
+
+    std::unique_ptr<ArrayBuilder> array_builder;
+    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), iso_calendar_type, &array_builder));
+    StructBuilder* struct_builder = checked_cast<StructBuilder*>(array_builder.get());
+    RETURN_NOT_OK(struct_builder->Reserve(in.length));
+
+    std::vector<BuilderType*> field_builders;
+    field_builders.reserve(3);
+    for (int i = 0; i < 3; i++) {
+      field_builders.push_back(
+          checked_cast<BuilderType*>(struct_builder->field_builder(i)));
+      RETURN_NOT_OK(field_builders[i]->Reserve(1));
+    }
+    auto visit_null = [&]() { return struct_builder->AppendNull(); };
+    auto visit_value = [&](int64_t arg) {
+      const auto iso_calendar = get_iso_calendar<Duration>(arg);
+      field_builders[0]->UnsafeAppend(iso_calendar[0]);
+      field_builders[1]->UnsafeAppend(iso_calendar[1]);
+      field_builders[2]->UnsafeAppend(iso_calendar[2]);
+      return struct_builder->Append();
+    };
+    RETURN_NOT_OK(VisitArrayDataInline<Int64Type>(in, visit_value, visit_null));
+
+    std::shared_ptr<Array> out_array;
+    RETURN_NOT_OK(struct_builder->Finish(&out_array));
+    *out = *std::move(out_array->data());
+
+    return Status::OK();
+  }
+};
+
+template <template <typename...> class Op, typename OutType>
+std::shared_ptr<ScalarFunction> MakeTemporal(std::string name, const FunctionDoc* doc) {
+  const auto& out_type = TypeTraits<OutType>::type_singleton();
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+
+  for (auto unit : internal::AllTimeUnits()) {
+    InputType in_type{match::TimestampTypeUnit(unit)};
+    switch (unit) {
+      case TimeUnit::SECOND: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::seconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MILLI: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::milliseconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MICRO: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::microseconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::NANO: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::nanoseconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+    }
+  }
+  return func;
+}
+
+template <template <typename...> class Op>
+std::shared_ptr<ScalarFunction> MakeSimpleTemporal(std::string name,

Review comment:
       ```suggestion
   std::shared_ptr<ScalarFunction> MakeStructTemporal(std::string name,
   ```
   
   ? (the kernel returning a struct type doesn't seem "simpler")

##########
File path: docs/source/cpp/compute.rst
##########
@@ -665,6 +665,57 @@ String component extraction
   e.g. 'letter' and 'digit' for the regular expression
   ``(?P<letter>[ab])(?P<digit>\\d)``.
 
+Temporal component extraction
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+These functions extract datetime components (year, month, day, etc) from timestamp type.
+Note: this is currently not supported for timestamps with timezone information.
+
++--------------------+------------+-------------------+---------------+--------+
+| Function name      | Arity      | Input types       | Output type   | Notes  |
++====================+============+===================+===============+========+
+| year               | Unary      | Temporal          | Int64         |        |
++--------------------+------------+-------------------+---------------+--------+
+| month              | Unary      | Temporal          | Int64         |        |
++--------------------+------------+-------------------+---------------+--------+
+| day                | Unary      | Temporal          | Int64         |        |
++--------------------+------------+-------------------+---------------+--------+
+| day_of_week        | Unary      | Temporal          | Int64         | \(1)   |
++--------------------+------------+-------------------+---------------+--------+
+| day_of_year        | Unary      | Temporal          | Int64         |        |
++--------------------+------------+-------------------+---------------+--------+
+| iso_year           | Unary      | Temporal          | Int64         | \(2)   |
++--------------------+------------+-------------------+---------------+--------+
+| iso_week           | Unary      | Temporal          | Int64         | \(2)   |
++--------------------+------------+-------------------+---------------+--------+
+| iso_calendar       | Unary      | Temporal          | Scalar Struct | \(3)   |

Review comment:
       ```suggestion
   | iso_calendar       | Unary      | Temporal          | Struct        | \(3)   |
   ```
   
   (not sure what you mean with the "scalar" struct here?)

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,626 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+
+namespace internal {
+
+namespace {
+
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+using internal::applicator::ScalarUnaryNotNull;
+using internal::applicator::SimpleUnary;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename Op, typename OutType>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<Op, OutType>;
+  using OutValue = typename internal::GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          checked_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        internal::VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        checked_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = internal::UnboxScalar<Int64Type>::Unbox(arg0);
+        internal::BoxScalar<OutType>::Box(
+            this->op.template Call<OutValue>(ctx, arg0_val, &st), out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename Op, typename OutType>
+struct ScalarUnaryTemporal {
+  using OutValue = typename internal::GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<Op, OutType> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding() -
+        1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    return static_cast<T>(
+        (t - sys_time<days>(year_month_day(t).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+//
+// First week of an ISO year has the majority (4 or more) of it's days in January.
+// Last week of an ISO year has the year's last Thursday in it.
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    auto y = year_month_day{t + days{3}}.year();
+    auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (t < start) {
+      --y;
+    }
+    return static_cast<T>(static_cast<int32_t>(y));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+//
+// First week of an ISO year has the majority (4 or more) of it's days in January.
+// Last week of an ISO year has the year's last Thursday in it.
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    auto y = year_month_day{t + days{3}}.year();
+    auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (t < start) {
+      --y;
+      start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(t - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::minutes>(t)) / std::chrono::seconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        (std::chrono::duration<double>(t - floor<std::chrono::seconds>(t)).count()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+template <typename Duration>
+inline std::vector<int64_t> get_iso_calendar(int64_t arg) {
+  const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+  const auto ymd = year_month_day(t);
+  auto y = year_month_day{t + days{3}}.year();
+  auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+  if (t < start) {
+    --y;
+    start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+  }
+  return {static_cast<int64_t>(static_cast<int32_t>(y)),
+          static_cast<int64_t>(trunc<weeks>(t - start).count() + 1),
+          static_cast<int64_t>(weekday(ymd).iso_encoding())};

Review comment:
       In the DayOfWeek implementation you subtract 1 (to have 0-6), while here not. I assume that's the ISO definition? If so, we should clearly document that difference in the ISOCalendar documentation (since now it seems to imply that the "day_of_week" field is the same as the "day_of_week" kernel. Maybe also prepend it with "iso_" ?

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,626 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+
+namespace internal {
+
+namespace {
+
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+using internal::applicator::ScalarUnaryNotNull;
+using internal::applicator::SimpleUnary;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename Op, typename OutType>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<Op, OutType>;
+  using OutValue = typename internal::GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          checked_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        internal::VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        checked_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = internal::UnboxScalar<Int64Type>::Unbox(arg0);
+        internal::BoxScalar<OutType>::Box(
+            this->op.template Call<OutValue>(ctx, arg0_val, &st), out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename Op, typename OutType>
+struct ScalarUnaryTemporal {
+  using OutValue = typename internal::GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<Op, OutType> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding() -
+        1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    return static_cast<T>(
+        (t - sys_time<days>(year_month_day(t).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+//
+// First week of an ISO year has the majority (4 or more) of it's days in January.
+// Last week of an ISO year has the year's last Thursday in it.
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    auto y = year_month_day{t + days{3}}.year();
+    auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (t < start) {
+      --y;
+    }
+    return static_cast<T>(static_cast<int32_t>(y));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+//
+// First week of an ISO year has the majority (4 or more) of it's days in January.
+// Last week of an ISO year has the year's last Thursday in it.
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    auto y = year_month_day{t + days{3}}.year();
+    auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (t < start) {
+      --y;
+      start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(t - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::minutes>(t)) / std::chrono::seconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        (std::chrono::duration<double>(t - floor<std::chrono::seconds>(t)).count()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+template <typename Duration>
+inline std::vector<int64_t> get_iso_calendar(int64_t arg) {
+  const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+  const auto ymd = year_month_day(t);
+  auto y = year_month_day{t + days{3}}.year();
+  auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+  if (t < start) {
+    --y;
+    start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+  }
+  return {static_cast<int64_t>(static_cast<int32_t>(y)),
+          static_cast<int64_t>(trunc<weeks>(t - start).count() + 1),
+          static_cast<int64_t>(weekday(ymd).iso_encoding())};
+}
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+template <typename Duration>
+struct ISOCalendar {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const std::string timezone =
+        checked_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    if (in.is_valid) {
+      const std::shared_ptr<DataType> iso_calendar_type =
+          struct_({field("iso_year", int64()), field("iso_week", int64()),
+                   field("day_of_week", int64())});
+      const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+      const auto iso_calendar = get_iso_calendar<Duration>(in_val);
+
+      std::vector<std::shared_ptr<Scalar>> values = {
+          std::make_shared<Int64Scalar>(iso_calendar[0]),
+          std::make_shared<Int64Scalar>(iso_calendar[1]),
+          std::make_shared<Int64Scalar>(iso_calendar[2])};
+      *checked_cast<StructScalar*>(out) = StructScalar(values, iso_calendar_type);
+    } else {
+      out->is_valid = false;
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    using BuilderType = typename TypeTraits<Int64Type>::BuilderType;
+    const std::string timezone =
+        checked_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+    const std::shared_ptr<DataType> iso_calendar_type =
+        struct_({field("iso_year", int64()), field("iso_week", int64()),
+                 field("day_of_week", int64())});
+
+    std::unique_ptr<ArrayBuilder> array_builder;
+    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), iso_calendar_type, &array_builder));
+    StructBuilder* struct_builder = checked_cast<StructBuilder*>(array_builder.get());
+    RETURN_NOT_OK(struct_builder->Reserve(in.length));
+
+    std::vector<BuilderType*> field_builders;
+    field_builders.reserve(3);
+    for (int i = 0; i < 3; i++) {
+      field_builders.push_back(
+          checked_cast<BuilderType*>(struct_builder->field_builder(i)));
+      RETURN_NOT_OK(field_builders[i]->Reserve(1));
+    }
+    auto visit_null = [&]() { return struct_builder->AppendNull(); };
+    auto visit_value = [&](int64_t arg) {
+      const auto iso_calendar = get_iso_calendar<Duration>(arg);
+      field_builders[0]->UnsafeAppend(iso_calendar[0]);
+      field_builders[1]->UnsafeAppend(iso_calendar[1]);
+      field_builders[2]->UnsafeAppend(iso_calendar[2]);
+      return struct_builder->Append();
+    };
+    RETURN_NOT_OK(VisitArrayDataInline<Int64Type>(in, visit_value, visit_null));
+
+    std::shared_ptr<Array> out_array;
+    RETURN_NOT_OK(struct_builder->Finish(&out_array));
+    *out = *std::move(out_array->data());
+
+    return Status::OK();
+  }
+};
+
+template <template <typename...> class Op, typename OutType>
+std::shared_ptr<ScalarFunction> MakeTemporal(std::string name, const FunctionDoc* doc) {
+  const auto& out_type = TypeTraits<OutType>::type_singleton();
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+
+  for (auto unit : internal::AllTimeUnits()) {
+    InputType in_type{match::TimestampTypeUnit(unit)};
+    switch (unit) {
+      case TimeUnit::SECOND: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::seconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MILLI: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::milliseconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MICRO: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::microseconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::NANO: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::nanoseconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+    }
+  }
+  return func;
+}
+
+template <template <typename...> class Op>
+std::shared_ptr<ScalarFunction> MakeSimpleTemporal(std::string name,
+                                                   const FunctionDoc* doc) {
+  const auto& out_type = struct_({field("iso_year", int64()), field("iso_week", int64()),
+                                  field("day_of_week", int64())});
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+
+  for (auto unit : internal::AllTimeUnits()) {
+    InputType in_type{match::TimestampTypeUnit(unit)};
+    switch (unit) {
+      case TimeUnit::SECOND: {
+        auto exec = SimpleUnary<Op<std::chrono::seconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MILLI: {
+        auto exec = SimpleUnary<Op<std::chrono::milliseconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MICRO: {
+        auto exec = SimpleUnary<Op<std::chrono::microseconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::NANO: {
+        auto exec = SimpleUnary<Op<std::chrono::nanoseconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+    }
+  }
+  return func;
+}
+
+const FunctionDoc year_doc{
+    "Extract year from timestamp",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc month_doc{
+    "Extract month number",
+    ("Month is encoded as January=1, December=12.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc day_doc{
+    "Extract day number",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc day_of_week_doc{
+    "Extract day of the week number",
+    ("Week starts on Monday denoted by 0 and ends on Sunday denoted by 6.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc day_of_year_doc{
+    "Extract number of day of year",
+    ("January 1st maps to day number 1, February 1st to 32, etc.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc iso_year_doc{
+    "Extract ISO year number",
+    ("First week of an ISO year has the majority (4 or more) of it's days in January."
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc iso_week_doc{
+    "Extract ISO week of year number",
+    ("First ISO week has the majority (4 or more) of it's days in January.\n"

Review comment:
       ```suggestion
       ("First ISO week has the majority (4 or more) of its days in January.\n"
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,626 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+
+namespace internal {
+
+namespace {
+
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+using internal::applicator::ScalarUnaryNotNull;
+using internal::applicator::SimpleUnary;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename Op, typename OutType>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<Op, OutType>;
+  using OutValue = typename internal::GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          checked_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        internal::VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        checked_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = internal::UnboxScalar<Int64Type>::Unbox(arg0);
+        internal::BoxScalar<OutType>::Box(
+            this->op.template Call<OutValue>(ctx, arg0_val, &st), out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename Op, typename OutType>
+struct ScalarUnaryTemporal {
+  using OutValue = typename internal::GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<Op, OutType> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding() -
+        1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    return static_cast<T>(
+        (t - sys_time<days>(year_month_day(t).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+//
+// First week of an ISO year has the majority (4 or more) of it's days in January.
+// Last week of an ISO year has the year's last Thursday in it.
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    auto y = year_month_day{t + days{3}}.year();
+    auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (t < start) {
+      --y;
+    }
+    return static_cast<T>(static_cast<int32_t>(y));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+//
+// First week of an ISO year has the majority (4 or more) of it's days in January.
+// Last week of an ISO year has the year's last Thursday in it.
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    auto y = year_month_day{t + days{3}}.year();
+    auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (t < start) {
+      --y;
+      start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(t - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::minutes>(t)) / std::chrono::seconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        (std::chrono::duration<double>(t - floor<std::chrono::seconds>(t)).count()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+template <typename Duration>
+inline std::vector<int64_t> get_iso_calendar(int64_t arg) {
+  const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+  const auto ymd = year_month_day(t);
+  auto y = year_month_day{t + days{3}}.year();
+  auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+  if (t < start) {
+    --y;
+    start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+  }
+  return {static_cast<int64_t>(static_cast<int32_t>(y)),
+          static_cast<int64_t>(trunc<weeks>(t - start).count() + 1),
+          static_cast<int64_t>(weekday(ymd).iso_encoding())};
+}
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+template <typename Duration>
+struct ISOCalendar {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const std::string timezone =
+        checked_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    if (in.is_valid) {
+      const std::shared_ptr<DataType> iso_calendar_type =
+          struct_({field("iso_year", int64()), field("iso_week", int64()),
+                   field("day_of_week", int64())});
+      const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+      const auto iso_calendar = get_iso_calendar<Duration>(in_val);
+
+      std::vector<std::shared_ptr<Scalar>> values = {
+          std::make_shared<Int64Scalar>(iso_calendar[0]),
+          std::make_shared<Int64Scalar>(iso_calendar[1]),
+          std::make_shared<Int64Scalar>(iso_calendar[2])};
+      *checked_cast<StructScalar*>(out) = StructScalar(values, iso_calendar_type);
+    } else {
+      out->is_valid = false;
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    using BuilderType = typename TypeTraits<Int64Type>::BuilderType;
+    const std::string timezone =
+        checked_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+    const std::shared_ptr<DataType> iso_calendar_type =
+        struct_({field("iso_year", int64()), field("iso_week", int64()),
+                 field("day_of_week", int64())});
+
+    std::unique_ptr<ArrayBuilder> array_builder;
+    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), iso_calendar_type, &array_builder));
+    StructBuilder* struct_builder = checked_cast<StructBuilder*>(array_builder.get());
+    RETURN_NOT_OK(struct_builder->Reserve(in.length));
+
+    std::vector<BuilderType*> field_builders;
+    field_builders.reserve(3);
+    for (int i = 0; i < 3; i++) {
+      field_builders.push_back(
+          checked_cast<BuilderType*>(struct_builder->field_builder(i)));
+      RETURN_NOT_OK(field_builders[i]->Reserve(1));
+    }
+    auto visit_null = [&]() { return struct_builder->AppendNull(); };
+    auto visit_value = [&](int64_t arg) {
+      const auto iso_calendar = get_iso_calendar<Duration>(arg);
+      field_builders[0]->UnsafeAppend(iso_calendar[0]);
+      field_builders[1]->UnsafeAppend(iso_calendar[1]);
+      field_builders[2]->UnsafeAppend(iso_calendar[2]);
+      return struct_builder->Append();
+    };
+    RETURN_NOT_OK(VisitArrayDataInline<Int64Type>(in, visit_value, visit_null));
+
+    std::shared_ptr<Array> out_array;
+    RETURN_NOT_OK(struct_builder->Finish(&out_array));
+    *out = *std::move(out_array->data());
+
+    return Status::OK();
+  }
+};
+
+template <template <typename...> class Op, typename OutType>
+std::shared_ptr<ScalarFunction> MakeTemporal(std::string name, const FunctionDoc* doc) {
+  const auto& out_type = TypeTraits<OutType>::type_singleton();
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+
+  for (auto unit : internal::AllTimeUnits()) {
+    InputType in_type{match::TimestampTypeUnit(unit)};
+    switch (unit) {
+      case TimeUnit::SECOND: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::seconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MILLI: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::milliseconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MICRO: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::microseconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::NANO: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::nanoseconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+    }
+  }
+  return func;
+}
+
+template <template <typename...> class Op>
+std::shared_ptr<ScalarFunction> MakeSimpleTemporal(std::string name,
+                                                   const FunctionDoc* doc) {
+  const auto& out_type = struct_({field("iso_year", int64()), field("iso_week", int64()),
+                                  field("day_of_week", int64())});
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+
+  for (auto unit : internal::AllTimeUnits()) {
+    InputType in_type{match::TimestampTypeUnit(unit)};
+    switch (unit) {
+      case TimeUnit::SECOND: {
+        auto exec = SimpleUnary<Op<std::chrono::seconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MILLI: {
+        auto exec = SimpleUnary<Op<std::chrono::milliseconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MICRO: {
+        auto exec = SimpleUnary<Op<std::chrono::microseconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::NANO: {
+        auto exec = SimpleUnary<Op<std::chrono::nanoseconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+    }
+  }
+  return func;
+}
+
+const FunctionDoc year_doc{
+    "Extract year from timestamp",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc month_doc{
+    "Extract month number",
+    ("Month is encoded as January=1, December=12.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc day_doc{
+    "Extract day number",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc day_of_week_doc{
+    "Extract day of the week number",
+    ("Week starts on Monday denoted by 0 and ends on Sunday denoted by 6.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc day_of_year_doc{
+    "Extract number of day of year",
+    ("January 1st maps to day number 1, February 1st to 32, etc.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc iso_year_doc{
+    "Extract ISO year number",
+    ("First week of an ISO year has the majority (4 or more) of it's days in January."

Review comment:
       ```suggestion
       ("First week of an ISO year has the majority (4 or more) of its days in January."
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,626 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+
+namespace internal {
+
+namespace {
+
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+using internal::applicator::ScalarUnaryNotNull;
+using internal::applicator::SimpleUnary;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename Op, typename OutType>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<Op, OutType>;
+  using OutValue = typename internal::GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          checked_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        internal::VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        checked_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = internal::UnboxScalar<Int64Type>::Unbox(arg0);
+        internal::BoxScalar<OutType>::Box(
+            this->op.template Call<OutValue>(ctx, arg0_val, &st), out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename Op, typename OutType>
+struct ScalarUnaryTemporal {
+  using OutValue = typename internal::GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<Op, OutType> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding() -
+        1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    return static_cast<T>(
+        (t - sys_time<days>(year_month_day(t).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+//
+// First week of an ISO year has the majority (4 or more) of it's days in January.
+// Last week of an ISO year has the year's last Thursday in it.
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    auto y = year_month_day{t + days{3}}.year();
+    auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (t < start) {
+      --y;
+    }
+    return static_cast<T>(static_cast<int32_t>(y));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+//
+// First week of an ISO year has the majority (4 or more) of it's days in January.
+// Last week of an ISO year has the year's last Thursday in it.
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    auto y = year_month_day{t + days{3}}.year();
+    auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (t < start) {
+      --y;
+      start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(t - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::minutes>(t)) / std::chrono::seconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        (std::chrono::duration<double>(t - floor<std::chrono::seconds>(t)).count()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+template <typename Duration>
+inline std::vector<int64_t> get_iso_calendar(int64_t arg) {
+  const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+  const auto ymd = year_month_day(t);
+  auto y = year_month_day{t + days{3}}.year();
+  auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+  if (t < start) {
+    --y;
+    start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+  }
+  return {static_cast<int64_t>(static_cast<int32_t>(y)),
+          static_cast<int64_t>(trunc<weeks>(t - start).count() + 1),
+          static_cast<int64_t>(weekday(ymd).iso_encoding())};
+}
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+template <typename Duration>
+struct ISOCalendar {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const std::string timezone =
+        checked_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    if (in.is_valid) {
+      const std::shared_ptr<DataType> iso_calendar_type =
+          struct_({field("iso_year", int64()), field("iso_week", int64()),
+                   field("day_of_week", int64())});
+      const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+      const auto iso_calendar = get_iso_calendar<Duration>(in_val);
+
+      std::vector<std::shared_ptr<Scalar>> values = {
+          std::make_shared<Int64Scalar>(iso_calendar[0]),
+          std::make_shared<Int64Scalar>(iso_calendar[1]),
+          std::make_shared<Int64Scalar>(iso_calendar[2])};
+      *checked_cast<StructScalar*>(out) = StructScalar(values, iso_calendar_type);
+    } else {
+      out->is_valid = false;
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    using BuilderType = typename TypeTraits<Int64Type>::BuilderType;
+    const std::string timezone =
+        checked_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+    const std::shared_ptr<DataType> iso_calendar_type =
+        struct_({field("iso_year", int64()), field("iso_week", int64()),
+                 field("day_of_week", int64())});
+
+    std::unique_ptr<ArrayBuilder> array_builder;
+    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), iso_calendar_type, &array_builder));
+    StructBuilder* struct_builder = checked_cast<StructBuilder*>(array_builder.get());
+    RETURN_NOT_OK(struct_builder->Reserve(in.length));
+
+    std::vector<BuilderType*> field_builders;
+    field_builders.reserve(3);
+    for (int i = 0; i < 3; i++) {
+      field_builders.push_back(
+          checked_cast<BuilderType*>(struct_builder->field_builder(i)));
+      RETURN_NOT_OK(field_builders[i]->Reserve(1));
+    }
+    auto visit_null = [&]() { return struct_builder->AppendNull(); };
+    auto visit_value = [&](int64_t arg) {
+      const auto iso_calendar = get_iso_calendar<Duration>(arg);
+      field_builders[0]->UnsafeAppend(iso_calendar[0]);
+      field_builders[1]->UnsafeAppend(iso_calendar[1]);
+      field_builders[2]->UnsafeAppend(iso_calendar[2]);
+      return struct_builder->Append();
+    };
+    RETURN_NOT_OK(VisitArrayDataInline<Int64Type>(in, visit_value, visit_null));
+
+    std::shared_ptr<Array> out_array;
+    RETURN_NOT_OK(struct_builder->Finish(&out_array));
+    *out = *std::move(out_array->data());
+
+    return Status::OK();
+  }
+};
+
+template <template <typename...> class Op, typename OutType>
+std::shared_ptr<ScalarFunction> MakeTemporal(std::string name, const FunctionDoc* doc) {
+  const auto& out_type = TypeTraits<OutType>::type_singleton();
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+
+  for (auto unit : internal::AllTimeUnits()) {
+    InputType in_type{match::TimestampTypeUnit(unit)};
+    switch (unit) {
+      case TimeUnit::SECOND: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::seconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MILLI: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::milliseconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MICRO: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::microseconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::NANO: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::nanoseconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+    }
+  }
+  return func;
+}
+
+template <template <typename...> class Op>
+std::shared_ptr<ScalarFunction> MakeSimpleTemporal(std::string name,
+                                                   const FunctionDoc* doc) {
+  const auto& out_type = struct_({field("iso_year", int64()), field("iso_week", int64()),
+                                  field("day_of_week", int64())});
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+
+  for (auto unit : internal::AllTimeUnits()) {
+    InputType in_type{match::TimestampTypeUnit(unit)};
+    switch (unit) {
+      case TimeUnit::SECOND: {
+        auto exec = SimpleUnary<Op<std::chrono::seconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MILLI: {
+        auto exec = SimpleUnary<Op<std::chrono::milliseconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MICRO: {
+        auto exec = SimpleUnary<Op<std::chrono::microseconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::NANO: {
+        auto exec = SimpleUnary<Op<std::chrono::nanoseconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+    }
+  }
+  return func;
+}
+
+const FunctionDoc year_doc{
+    "Extract year from timestamp",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc month_doc{
+    "Extract month number",
+    ("Month is encoded as January=1, December=12.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc day_doc{
+    "Extract day number",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc day_of_week_doc{
+    "Extract day of the week number",
+    ("Week starts on Monday denoted by 0 and ends on Sunday denoted by 6.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc day_of_year_doc{
+    "Extract number of day of year",
+    ("January 1st maps to day number 1, February 1st to 32, etc.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc iso_year_doc{
+    "Extract ISO year number",
+    ("First week of an ISO year has the majority (4 or more) of it's days in January."
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc iso_week_doc{
+    "Extract ISO week of year number",
+    ("First ISO week has the majority (4 or more) of it's days in January.\n"
+     "Week of the year starts with 1 and can run up to 53.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc iso_calendar_doc{
+    "Extract (ISO year, ISO week, day of week) struct",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc quarter_doc{
+    "Extract quarter of year number",
+    ("First quarter maps to 1 and forth quarter maps to 4.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc hour_doc{
+    "Extract hour values",

Review comment:
       ```suggestion
       "Extract hour value",
   ```
   
   (above you also use singular form for eg "day number")
   
   (and the same for the ones below)

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,626 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+
+namespace internal {
+
+namespace {
+
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+using internal::applicator::ScalarUnaryNotNull;
+using internal::applicator::SimpleUnary;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename Op, typename OutType>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<Op, OutType>;
+  using OutValue = typename internal::GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          checked_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        internal::VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        checked_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = internal::UnboxScalar<Int64Type>::Unbox(arg0);
+        internal::BoxScalar<OutType>::Box(
+            this->op.template Call<OutValue>(ctx, arg0_val, &st), out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename Op, typename OutType>
+struct ScalarUnaryTemporal {
+  using OutValue = typename internal::GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<Op, OutType> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding() -
+        1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    return static_cast<T>(
+        (t - sys_time<days>(year_month_day(t).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+//
+// First week of an ISO year has the majority (4 or more) of it's days in January.
+// Last week of an ISO year has the year's last Thursday in it.
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    auto y = year_month_day{t + days{3}}.year();
+    auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (t < start) {
+      --y;
+    }
+    return static_cast<T>(static_cast<int32_t>(y));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+//
+// First week of an ISO year has the majority (4 or more) of it's days in January.
+// Last week of an ISO year has the year's last Thursday in it.
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    auto y = year_month_day{t + days{3}}.year();
+    auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (t < start) {
+      --y;
+      start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(t - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::minutes>(t)) / std::chrono::seconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        (std::chrono::duration<double>(t - floor<std::chrono::seconds>(t)).count()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+template <typename Duration>
+inline std::vector<int64_t> get_iso_calendar(int64_t arg) {
+  const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+  const auto ymd = year_month_day(t);
+  auto y = year_month_day{t + days{3}}.year();
+  auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+  if (t < start) {
+    --y;
+    start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+  }
+  return {static_cast<int64_t>(static_cast<int32_t>(y)),
+          static_cast<int64_t>(trunc<weeks>(t - start).count() + 1),
+          static_cast<int64_t>(weekday(ymd).iso_encoding())};
+}
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+template <typename Duration>
+struct ISOCalendar {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const std::string timezone =
+        checked_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    if (in.is_valid) {
+      const std::shared_ptr<DataType> iso_calendar_type =
+          struct_({field("iso_year", int64()), field("iso_week", int64()),
+                   field("day_of_week", int64())});
+      const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+      const auto iso_calendar = get_iso_calendar<Duration>(in_val);
+
+      std::vector<std::shared_ptr<Scalar>> values = {
+          std::make_shared<Int64Scalar>(iso_calendar[0]),
+          std::make_shared<Int64Scalar>(iso_calendar[1]),
+          std::make_shared<Int64Scalar>(iso_calendar[2])};
+      *checked_cast<StructScalar*>(out) = StructScalar(values, iso_calendar_type);
+    } else {
+      out->is_valid = false;
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    using BuilderType = typename TypeTraits<Int64Type>::BuilderType;
+    const std::string timezone =
+        checked_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+    const std::shared_ptr<DataType> iso_calendar_type =
+        struct_({field("iso_year", int64()), field("iso_week", int64()),
+                 field("day_of_week", int64())});
+
+    std::unique_ptr<ArrayBuilder> array_builder;
+    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), iso_calendar_type, &array_builder));
+    StructBuilder* struct_builder = checked_cast<StructBuilder*>(array_builder.get());
+    RETURN_NOT_OK(struct_builder->Reserve(in.length));
+
+    std::vector<BuilderType*> field_builders;
+    field_builders.reserve(3);
+    for (int i = 0; i < 3; i++) {
+      field_builders.push_back(
+          checked_cast<BuilderType*>(struct_builder->field_builder(i)));
+      RETURN_NOT_OK(field_builders[i]->Reserve(1));
+    }
+    auto visit_null = [&]() { return struct_builder->AppendNull(); };
+    auto visit_value = [&](int64_t arg) {
+      const auto iso_calendar = get_iso_calendar<Duration>(arg);
+      field_builders[0]->UnsafeAppend(iso_calendar[0]);
+      field_builders[1]->UnsafeAppend(iso_calendar[1]);
+      field_builders[2]->UnsafeAppend(iso_calendar[2]);
+      return struct_builder->Append();
+    };
+    RETURN_NOT_OK(VisitArrayDataInline<Int64Type>(in, visit_value, visit_null));
+
+    std::shared_ptr<Array> out_array;
+    RETURN_NOT_OK(struct_builder->Finish(&out_array));
+    *out = *std::move(out_array->data());
+
+    return Status::OK();
+  }
+};
+
+template <template <typename...> class Op, typename OutType>
+std::shared_ptr<ScalarFunction> MakeTemporal(std::string name, const FunctionDoc* doc) {
+  const auto& out_type = TypeTraits<OutType>::type_singleton();
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+
+  for (auto unit : internal::AllTimeUnits()) {
+    InputType in_type{match::TimestampTypeUnit(unit)};
+    switch (unit) {
+      case TimeUnit::SECOND: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::seconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MILLI: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::milliseconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MICRO: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::microseconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::NANO: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::nanoseconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+    }
+  }
+  return func;
+}
+
+template <template <typename...> class Op>
+std::shared_ptr<ScalarFunction> MakeSimpleTemporal(std::string name,
+                                                   const FunctionDoc* doc) {
+  const auto& out_type = struct_({field("iso_year", int64()), field("iso_week", int64()),
+                                  field("day_of_week", int64())});
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+
+  for (auto unit : internal::AllTimeUnits()) {
+    InputType in_type{match::TimestampTypeUnit(unit)};
+    switch (unit) {
+      case TimeUnit::SECOND: {
+        auto exec = SimpleUnary<Op<std::chrono::seconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MILLI: {
+        auto exec = SimpleUnary<Op<std::chrono::milliseconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MICRO: {
+        auto exec = SimpleUnary<Op<std::chrono::microseconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::NANO: {
+        auto exec = SimpleUnary<Op<std::chrono::nanoseconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+    }
+  }
+  return func;
+}
+
+const FunctionDoc year_doc{
+    "Extract year from timestamp",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc month_doc{
+    "Extract month number",
+    ("Month is encoded as January=1, December=12.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc day_doc{
+    "Extract day number",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc day_of_week_doc{
+    "Extract day of the week number",
+    ("Week starts on Monday denoted by 0 and ends on Sunday denoted by 6.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc day_of_year_doc{
+    "Extract number of day of year",
+    ("January 1st maps to day number 1, February 1st to 32, etc.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc iso_year_doc{
+    "Extract ISO year number",
+    ("First week of an ISO year has the majority (4 or more) of it's days in January."
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc iso_week_doc{
+    "Extract ISO week of year number",
+    ("First ISO week has the majority (4 or more) of it's days in January.\n"
+     "Week of the year starts with 1 and can run up to 53.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc iso_calendar_doc{
+    "Extract (ISO year, ISO week, day of week) struct",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc quarter_doc{
+    "Extract quarter of year number",
+    ("First quarter maps to 1 and forth quarter maps to 4.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc hour_doc{
+    "Extract hour values",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc minute_doc{
+    "Extract minute values",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc second_doc{
+    "Extract second values",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc millisecond_doc{
+    "Extract millisecond values",

Review comment:
       Can you add here the clarification about it being the number of milliseconds since the last full second, as you did for the C++ functions? (and same for microsecond/nanosecond below)

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,626 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+
+namespace internal {
+
+namespace {
+
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+using internal::applicator::ScalarUnaryNotNull;
+using internal::applicator::SimpleUnary;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename Op, typename OutType>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<Op, OutType>;
+  using OutValue = typename internal::GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          checked_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        internal::VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        checked_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = internal::UnboxScalar<Int64Type>::Unbox(arg0);
+        internal::BoxScalar<OutType>::Box(
+            this->op.template Call<OutValue>(ctx, arg0_val, &st), out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename Op, typename OutType>
+struct ScalarUnaryTemporal {
+  using OutValue = typename internal::GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<Op, OutType> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding() -
+        1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    return static_cast<T>(
+        (t - sys_time<days>(year_month_day(t).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+//
+// First week of an ISO year has the majority (4 or more) of it's days in January.
+// Last week of an ISO year has the year's last Thursday in it.
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    auto y = year_month_day{t + days{3}}.year();
+    auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (t < start) {
+      --y;
+    }
+    return static_cast<T>(static_cast<int32_t>(y));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+//
+// First week of an ISO year has the majority (4 or more) of it's days in January.
+// Last week of an ISO year has the year's last Thursday in it.
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    auto y = year_month_day{t + days{3}}.year();
+    auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (t < start) {
+      --y;
+      start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(t - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::minutes>(t)) / std::chrono::seconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        (std::chrono::duration<double>(t - floor<std::chrono::seconds>(t)).count()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+template <typename Duration>
+inline std::vector<int64_t> get_iso_calendar(int64_t arg) {
+  const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+  const auto ymd = year_month_day(t);
+  auto y = year_month_day{t + days{3}}.year();
+  auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+  if (t < start) {
+    --y;
+    start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+  }
+  return {static_cast<int64_t>(static_cast<int32_t>(y)),
+          static_cast<int64_t>(trunc<weeks>(t - start).count() + 1),
+          static_cast<int64_t>(weekday(ymd).iso_encoding())};
+}
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+template <typename Duration>
+struct ISOCalendar {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const std::string timezone =
+        checked_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    if (in.is_valid) {
+      const std::shared_ptr<DataType> iso_calendar_type =
+          struct_({field("iso_year", int64()), field("iso_week", int64()),
+                   field("day_of_week", int64())});
+      const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+      const auto iso_calendar = get_iso_calendar<Duration>(in_val);
+
+      std::vector<std::shared_ptr<Scalar>> values = {
+          std::make_shared<Int64Scalar>(iso_calendar[0]),
+          std::make_shared<Int64Scalar>(iso_calendar[1]),
+          std::make_shared<Int64Scalar>(iso_calendar[2])};
+      *checked_cast<StructScalar*>(out) = StructScalar(values, iso_calendar_type);
+    } else {
+      out->is_valid = false;
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    using BuilderType = typename TypeTraits<Int64Type>::BuilderType;
+    const std::string timezone =
+        checked_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+    const std::shared_ptr<DataType> iso_calendar_type =
+        struct_({field("iso_year", int64()), field("iso_week", int64()),
+                 field("day_of_week", int64())});
+
+    std::unique_ptr<ArrayBuilder> array_builder;
+    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), iso_calendar_type, &array_builder));
+    StructBuilder* struct_builder = checked_cast<StructBuilder*>(array_builder.get());
+    RETURN_NOT_OK(struct_builder->Reserve(in.length));
+
+    std::vector<BuilderType*> field_builders;
+    field_builders.reserve(3);
+    for (int i = 0; i < 3; i++) {
+      field_builders.push_back(
+          checked_cast<BuilderType*>(struct_builder->field_builder(i)));
+      RETURN_NOT_OK(field_builders[i]->Reserve(1));
+    }
+    auto visit_null = [&]() { return struct_builder->AppendNull(); };
+    auto visit_value = [&](int64_t arg) {
+      const auto iso_calendar = get_iso_calendar<Duration>(arg);
+      field_builders[0]->UnsafeAppend(iso_calendar[0]);
+      field_builders[1]->UnsafeAppend(iso_calendar[1]);
+      field_builders[2]->UnsafeAppend(iso_calendar[2]);
+      return struct_builder->Append();
+    };
+    RETURN_NOT_OK(VisitArrayDataInline<Int64Type>(in, visit_value, visit_null));
+
+    std::shared_ptr<Array> out_array;
+    RETURN_NOT_OK(struct_builder->Finish(&out_array));
+    *out = *std::move(out_array->data());
+
+    return Status::OK();
+  }
+};
+
+template <template <typename...> class Op, typename OutType>
+std::shared_ptr<ScalarFunction> MakeTemporal(std::string name, const FunctionDoc* doc) {
+  const auto& out_type = TypeTraits<OutType>::type_singleton();
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+
+  for (auto unit : internal::AllTimeUnits()) {
+    InputType in_type{match::TimestampTypeUnit(unit)};
+    switch (unit) {
+      case TimeUnit::SECOND: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::seconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MILLI: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::milliseconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MICRO: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::microseconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::NANO: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::nanoseconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+    }
+  }
+  return func;
+}
+
+template <template <typename...> class Op>
+std::shared_ptr<ScalarFunction> MakeSimpleTemporal(std::string name,
+                                                   const FunctionDoc* doc) {
+  const auto& out_type = struct_({field("iso_year", int64()), field("iso_week", int64()),
+                                  field("day_of_week", int64())});
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+
+  for (auto unit : internal::AllTimeUnits()) {
+    InputType in_type{match::TimestampTypeUnit(unit)};
+    switch (unit) {
+      case TimeUnit::SECOND: {
+        auto exec = SimpleUnary<Op<std::chrono::seconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MILLI: {
+        auto exec = SimpleUnary<Op<std::chrono::milliseconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MICRO: {
+        auto exec = SimpleUnary<Op<std::chrono::microseconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::NANO: {
+        auto exec = SimpleUnary<Op<std::chrono::nanoseconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+    }
+  }
+  return func;
+}
+
+const FunctionDoc year_doc{
+    "Extract year from timestamp",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc month_doc{
+    "Extract month number",
+    ("Month is encoded as January=1, December=12.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc day_doc{
+    "Extract day number",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc day_of_week_doc{
+    "Extract day of the week number",
+    ("Week starts on Monday denoted by 0 and ends on Sunday denoted by 6.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc day_of_year_doc{
+    "Extract number of day of year",
+    ("January 1st maps to day number 1, February 1st to 32, etc.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc iso_year_doc{
+    "Extract ISO year number",
+    ("First week of an ISO year has the majority (4 or more) of it's days in January."
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc iso_week_doc{
+    "Extract ISO week of year number",
+    ("First ISO week has the majority (4 or more) of it's days in January.\n"
+     "Week of the year starts with 1 and can run up to 53.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc iso_calendar_doc{
+    "Extract (ISO year, ISO week, day of week) struct",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc quarter_doc{
+    "Extract quarter of year number",
+    ("First quarter maps to 1 and forth quarter maps to 4.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc hour_doc{
+    "Extract hour values",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc minute_doc{
+    "Extract minute values",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc second_doc{
+    "Extract second values",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc millisecond_doc{
+    "Extract millisecond values",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc microsecond_doc{
+    "Extract microsecond values",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc nanosecond_doc{
+    "Extract nanosecond values",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc subsecond_doc{
+    "Extract subsecond values",

Review comment:
       Here also best to add the clarification of what subsecond is from the `Subsecond` docstring




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

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



[GitHub] [arrow] rok commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   I've adopted [some code](https://github.com/apache/arrow/blob/2765cefc737ac8b5c364a20f8ff94fc158b91b37/cpp/src/arrow/compute/kernels/scalar_temporal.cc#L197) from [iso_week.h](https://github.com/HowardHinnant/date/blob/master/include/date/iso_week.h#L1503). Would it make more sense to add iso_week.h as a vendored library instead? Should we add a note to our license? (iso_week.h is MIT licensed).


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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.

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.

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};

Review comment:
       `sys_days` is shorthand for `sys_time<days>`. I've made everything `sys_time<days>` 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.

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



[GitHub] [arrow] rok commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   Thanks for the review @jorisvandenbossche! I think I've addressed the second/subsecond issue and docs. (Sorry for amend commits).
   
   I'll now look into:
   * iso_year/iso_week/day_of_week tests
   * all timestamps being interpreted as having nanosecond unit


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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,107 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestSimpleTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO);
+  auto timestamps = ArrayFromJSON(unit, times);
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("weekday", int64())});
+
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar = ArrayFromJSON(iso_calendar_type,
+                                    R"([{"iso_year": 1970, "iso_week": 1, "weekday": 4},

Review comment:
       `day_of_week`




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,107 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestSimpleTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO);
+  auto timestamps = ArrayFromJSON(unit, times);
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("weekday", int64())});
+
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar = ArrayFromJSON(iso_calendar_type,
+                                    R"([{"iso_year": 1970, "iso_week": 1, "weekday": 4},
+                        {"iso_year": 2000, "iso_week": 9, "weekday": 2},
+                        {"iso_year": 1899, "iso_week": 52, "weekday": 7},
+                        {"iso_year": 2033, "iso_week": 20, "weekday": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59.123456789, 23.999999999, 20.001001001, 20.0, null]";
+  auto millisecond = "[123, 999, 1, 0, null]";
+  auto microsecond = "[456, 999, 1, 0, null]";
+  auto nanosecond = "[789, 999, 1, 0, null]";
+  auto subsecond = "[123456789, 999999999, 1001001, 0, null]";
+
+  CheckScalarUnary("year", unit, times, int64(), year);
+  CheckScalarUnary("month", unit, times, int64(), month);
+  CheckScalarUnary("day", unit, times, int64(), day);
+  CheckScalarUnary("day_of_week", unit, times, int64(), day_of_week);
+  CheckScalarUnary("day_of_year", unit, times, int64(), day_of_year);
+  CheckScalarUnary("iso_year", unit, times, int64(), iso_year);
+  CheckScalarUnary("iso_week", unit, times, int64(), iso_week);
+  CheckScalarUnary("iso_calendar", timestamps, iso_calendar);
+  CheckScalarUnary("quarter", unit, times, int64(), quarter);
+  CheckScalarUnary("hour", unit, times, int64(), hour);
+  CheckScalarUnary("minute", unit, times, int64(), minute);
+  CheckScalarUnary("second", unit, times, float64(), second);
+  CheckScalarUnary("millisecond", unit, times, int64(), millisecond);
+  CheckScalarUnary("microsecond", unit, times, int64(), microsecond);
+  CheckScalarUnary("nanosecond", unit, times, int64(), nanosecond);
+  CheckScalarUnary("subsecond", unit, times, int64(), subsecond);
+}
+
+TEST(ScalarTemporalTest, TestZonedTemporalComponentExtraction) {
+  std::string timezone = "Etc/UTC-2";
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO, timezone);
+  auto timestamps = ArrayFromJSON(unit, times);
+
+  ASSERT_RAISES(Invalid, Year(timestamps));
+  ASSERT_RAISES(Invalid, Month(timestamps));
+  ASSERT_RAISES(Invalid, Day(timestamps));
+  ASSERT_RAISES(Invalid, DayOfWeek(timestamps));
+  ASSERT_RAISES(Invalid, DayOfYear(timestamps));
+  ASSERT_RAISES(Invalid, ISOYear(timestamps));
+  ASSERT_RAISES(Invalid, ISOWeek(timestamps));
+  ASSERT_RAISES(Invalid, ISOCalendar(timestamps));
+  ASSERT_RAISES(Invalid, Quarter(timestamps));
+  ASSERT_RAISES(Invalid, Hour(timestamps));
+  ASSERT_RAISES(Invalid, Minute(timestamps));
+  ASSERT_RAISES(Invalid, Second(timestamps));
+  ASSERT_RAISES(Invalid, Millisecond(timestamps));
+  ASSERT_RAISES(Invalid, Microsecond(timestamps));
+  ASSERT_RAISES(Invalid, Nanosecond(timestamps));
+  ASSERT_RAISES(Invalid, Subsecond(timestamps));

Review comment:
       Arrow timestamp is UTC unix time with optional timezone information (if I understand correctly).
   So if I want to know e.g. hour in my timezone I need to convert to my timezone at some point.
   Timezone conversion kernel is an option, but then timestamp becomes local time instead of unix time?




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return static_cast<T>((sd - sys_days(year_month_day(sd).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day{sys_days{floor<days>(Duration{arg})} + days{3}}.year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto dp = sys_days{floor<days>(Duration{arg})};
+    auto y = year_month_day{dp + days{3}}.year();
+    auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (dp < start) {
+      --y;
+      start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(dp - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        std::chrono::duration<double>(t - floor<std::chrono::minutes>(t)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::seconds>(t)) /
+                          std::chrono::nanoseconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+template <typename Duration, typename OutType>
+struct ISOCalendar {
+  using T = typename OutType::c_type;
+
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    using ScalarType = typename TypeTraits<OutType>::ScalarType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    if (!in.is_valid) {
+      out->is_valid = false;
+    } else {
+      const std::shared_ptr<DataType> iso_calendar_type =
+          struct_({field("iso_year", out_type), field("iso_week", out_type),
+                   field("weekday", out_type)});
+
+      const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+      const auto dp = sys_days{floor<days>(Duration{in_val})};
+      const auto ymd = year_month_day(dp);
+      auto y = year_month_day{dp + days{3}}.year();
+      auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      if (dp < start) {
+        --y;
+        start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      }
+
+      std::vector<std::shared_ptr<Scalar>> values = {
+          std::make_shared<ScalarType>(static_cast<T>(static_cast<int32_t>(ymd.year()))),
+          std::make_shared<ScalarType>(
+              static_cast<T>(trunc<weeks>(dp - start).count() + 1)),
+          std::make_shared<ScalarType>(static_cast<T>(weekday(ymd).iso_encoding()))};
+      *checked_cast<StructScalar*>(out) = StructScalar(values, iso_calendar_type);
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    using BuilderType = typename TypeTraits<OutType>::BuilderType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    const std::shared_ptr<DataType> iso_calendar_type =
+        struct_({field("iso_year", out_type), field("iso_week", out_type),
+                 field("weekday", out_type)});
+
+    std::unique_ptr<ArrayBuilder> array_builder;
+    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), iso_calendar_type, &array_builder));
+    StructBuilder* struct_builder = checked_cast<StructBuilder*>(array_builder.get());
+
+    std::vector<BuilderType*> field_builders;
+    field_builders.reserve(3);
+    for (int i = 0; i < 3; i++) {
+      field_builders.push_back(
+          checked_cast<BuilderType*>(struct_builder->field_builder(i)));
+    }
+    auto visit_null = [&]() { return struct_builder->AppendNull(); };
+    auto visit_value = [&](int64_t arg) {
+      const auto dp = sys_days{floor<days>(Duration{arg})};
+      const auto ymd = year_month_day(dp);
+      auto y = year_month_day{dp + days{3}}.year();
+      auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      if (dp < start) {
+        --y;
+        start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      }
+
+      RETURN_NOT_OK(
+          field_builders[0]->Append(static_cast<T>(static_cast<int32_t>(ymd.year()))));
+      RETURN_NOT_OK(field_builders[1]->Append(
+          static_cast<T>(trunc<weeks>(dp - start).count() + 1)));
+      RETURN_NOT_OK(
+          field_builders[2]->Append(static_cast<T>(weekday(ymd).iso_encoding())));
+
+      return struct_builder->Append();
+    };
+    RETURN_NOT_OK(VisitArrayDataInline<OutType>(in, visit_value, visit_null));
+
+    std::shared_ptr<Array> out_array;
+    RETURN_NOT_OK(struct_builder->Finish(&out_array));
+    *out = *std::move(out_array->data());
+
+    return Status::OK();
+  }
+};
+
+// Generate a kernel given an arithmetic functor
+template <template <typename... Args> class KernelGenerator,
+          template <typename... Args> class Op, typename Duration>
+ArrayKernelExec ExecFromOp(detail::GetTypeId get_id) {
+  switch (get_id.id) {

Review comment:
       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.

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: docs/source/cpp/compute.rst
##########
@@ -637,6 +637,54 @@ String extraction
   e.g. 'letter' and 'digit' for the regular expression
   ``(?P<letter>[ab])(?P<digit>\\d)``.
 
+Temporal component extraction
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+These functions extract datetime components (year, month, day, etc) from timestamp type.
+Note: timezone information is currently ignored if present.
+
++--------------------+------------+-------------------+-----------------+--------+
+| Function name      | Arity      | Input types       | Output type     | Notes  |
++====================+============+===================+=================+========+
+| year               | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| month              | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| day                | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| day_of_week        | Unary      | Temporal          | Numeric         |        |

Review comment:
       Done.

##########
File path: docs/source/cpp/compute.rst
##########
@@ -637,6 +637,54 @@ String extraction
   e.g. 'letter' and 'digit' for the regular expression
   ``(?P<letter>[ab])(?P<digit>\\d)``.
 
+Temporal component extraction
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+These functions extract datetime components (year, month, day, etc) from timestamp type.
+Note: timezone information is currently ignored if present.
+
++--------------------+------------+-------------------+-----------------+--------+
+| Function name      | Arity      | Input types       | Output type     | Notes  |
++====================+============+===================+=================+========+
+| year               | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| month              | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| day                | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| day_of_week        | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| day_of_year        | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| iso_year           | Unary      | Temporal          | Numeric         | \(1)   |
++--------------------+------------+-------------------+-----------------+--------+
+| iso_week           | Unary      | Temporal          | Numeric         | \(1)   |
++--------------------+------------+-------------------+-----------------+--------+
+| iso_calendar       | Unary      | Temporal          | Scalar Struct   | \(2)   |
++--------------------+------------+-------------------+-----------------+--------+
+| quarter            | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| hour               | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| minute             | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| second             | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| millisecond        | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| microsecond        | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| nanosecond         | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| subsecond          | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+
+* \(1) The ISO 8601 definition for week 01 is the week with the first Thursday
+  of the Gregorian year (i.e. of January) in it.
+  .. _Wikipedia ISO Week date: https://en.wikipedia.org/wiki/ISO_week_date#First_week
+
+* \(2) Struct with fields ISO year, ISO week number, and weekday.

Review comment:
       Done.

##########
File path: docs/source/cpp/compute.rst
##########
@@ -637,6 +637,54 @@ String extraction
   e.g. 'letter' and 'digit' for the regular expression
   ``(?P<letter>[ab])(?P<digit>\\d)``.
 
+Temporal component extraction
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+These functions extract datetime components (year, month, day, etc) from timestamp type.
+Note: timezone information is currently ignored if present.
+
++--------------------+------------+-------------------+-----------------+--------+
+| Function name      | Arity      | Input types       | Output type     | Notes  |
++====================+============+===================+=================+========+
+| year               | Unary      | Temporal          | Numeric         |        |

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.

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -450,5 +450,145 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Week returns week of year value for each element of `values`
+///
+/// \param[in] values input to extract week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Week(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns quarter of year value for each element of `values`
+///
+/// \param[in] values input to extract quarter of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Quarter(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Hour returns hour value for each element of `values`
+///
+/// \param[in] values input to extract hour from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Hour(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Minute returns minutes value for each element of `values`
+///
+/// \param[in] values input to extract minutes from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Minute(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Second returns seconds value for each element of `values`
+///
+/// \param[in] values input to extract seconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Second(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Millisecond returns milliseconds value for each element of `values`
+///
+/// \param[in] values input to extract milliseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Millisecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Microsecond returns microseconds value for each element of `values`
+///
+/// \param[in] values input to extract microseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Microsecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Nanosecond returns nanoseconds value for each element of `values`
+///
+/// \param[in] values input to extract nanoseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Nanosecond(const Datum& values, ExecContext* ctx = NULLPTR);

Review comment:
       To be clear, judging the tests, the current implementation is not using the "total" nanoseconds right now, but only the part after the microseconds. Personally I think this is the useful behaviour




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

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



[GitHub] [arrow] rok commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   I fixed the iso_calendar issue so I'd ask for another round of review please :)


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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return static_cast<T>((sd - sys_days(year_month_day(sd).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day{sys_days{floor<days>(Duration{arg})} + days{3}}.year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto dp = sys_days{floor<days>(Duration{arg})};
+    auto y = year_month_day{dp + days{3}}.year();
+    auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (dp < start) {
+      --y;
+      start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(dp - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        std::chrono::duration<double>(t - floor<std::chrono::minutes>(t)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::seconds>(t)) /
+                          std::chrono::nanoseconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+template <typename Duration, typename OutType>
+struct ISOCalendar {
+  using T = typename OutType::c_type;
+
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    using ScalarType = typename TypeTraits<OutType>::ScalarType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    if (!in.is_valid) {
+      out->is_valid = false;
+    } else {
+      const std::shared_ptr<DataType> iso_calendar_type =
+          struct_({field("iso_year", out_type), field("iso_week", out_type),
+                   field("weekday", out_type)});
+
+      const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+      const auto dp = sys_days{floor<days>(Duration{in_val})};
+      const auto ymd = year_month_day(dp);
+      auto y = year_month_day{dp + days{3}}.year();
+      auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      if (dp < start) {
+        --y;
+        start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      }
+
+      std::vector<std::shared_ptr<Scalar>> values = {
+          std::make_shared<ScalarType>(static_cast<T>(static_cast<int32_t>(ymd.year()))),
+          std::make_shared<ScalarType>(
+              static_cast<T>(trunc<weeks>(dp - start).count() + 1)),
+          std::make_shared<ScalarType>(static_cast<T>(weekday(ymd).iso_encoding()))};
+      *checked_cast<StructScalar*>(out) = StructScalar(values, iso_calendar_type);
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    using BuilderType = typename TypeTraits<OutType>::BuilderType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    const std::shared_ptr<DataType> iso_calendar_type =
+        struct_({field("iso_year", out_type), field("iso_week", out_type),
+                 field("weekday", out_type)});
+
+    std::unique_ptr<ArrayBuilder> array_builder;
+    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), iso_calendar_type, &array_builder));
+    StructBuilder* struct_builder = checked_cast<StructBuilder*>(array_builder.get());
+
+    std::vector<BuilderType*> field_builders;
+    field_builders.reserve(3);
+    for (int i = 0; i < 3; i++) {
+      field_builders.push_back(
+          checked_cast<BuilderType*>(struct_builder->field_builder(i)));
+    }
+    auto visit_null = [&]() { return struct_builder->AppendNull(); };
+    auto visit_value = [&](int64_t arg) {
+      const auto dp = sys_days{floor<days>(Duration{arg})};
+      const auto ymd = year_month_day(dp);
+      auto y = year_month_day{dp + days{3}}.year();
+      auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      if (dp < start) {
+        --y;
+        start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      }
+
+      RETURN_NOT_OK(
+          field_builders[0]->Append(static_cast<T>(static_cast<int32_t>(ymd.year()))));
+      RETURN_NOT_OK(field_builders[1]->Append(
+          static_cast<T>(trunc<weeks>(dp - start).count() + 1)));
+      RETURN_NOT_OK(
+          field_builders[2]->Append(static_cast<T>(weekday(ymd).iso_encoding())));
+
+      return struct_builder->Append();
+    };
+    RETURN_NOT_OK(VisitArrayDataInline<OutType>(in, visit_value, visit_null));
+
+    std::shared_ptr<Array> out_array;
+    RETURN_NOT_OK(struct_builder->Finish(&out_array));
+    *out = *std::move(out_array->data());
+
+    return Status::OK();
+  }
+};
+
+// Generate a kernel given an arithmetic functor
+template <template <typename... Args> class KernelGenerator,
+          template <typename... Args> class Op, typename Duration>
+ArrayKernelExec ExecFromOp(detail::GetTypeId get_id) {
+  switch (get_id.id) {

Review comment:
       Wouldn't control of output type be useful sometimes?
   Will remove.




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -450,5 +450,145 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Week returns week of year value for each element of `values`
+///
+/// \param[in] values input to extract week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Week(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns quarter of year value for each element of `values`
+///
+/// \param[in] values input to extract quarter of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Quarter(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Hour returns hour value for each element of `values`
+///
+/// \param[in] values input to extract hour from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Hour(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Minute returns minutes value for each element of `values`
+///
+/// \param[in] values input to extract minutes from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Minute(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Second returns seconds value for each element of `values`
+///
+/// \param[in] values input to extract seconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Second(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Millisecond returns milliseconds value for each element of `values`
+///
+/// \param[in] values input to extract milliseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Millisecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Microsecond returns microseconds value for each element of `values`
+///
+/// \param[in] values input to extract microseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Microsecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Nanosecond returns nanoseconds value for each element of `values`
+///
+/// \param[in] values input to extract nanoseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Nanosecond(const Datum& values, ExecContext* ctx = NULLPTR);

Review comment:
       Well this is not ideal :).
   I propose to implement this as `total units since last second` and then add an exception in python for nanoseconds.




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,626 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+
+namespace internal {
+
+namespace {
+
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+using internal::applicator::ScalarUnaryNotNull;
+using internal::applicator::SimpleUnary;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename Op, typename OutType>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<Op, OutType>;
+  using OutValue = typename internal::GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          checked_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        internal::VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        checked_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = internal::UnboxScalar<Int64Type>::Unbox(arg0);
+        internal::BoxScalar<OutType>::Box(
+            this->op.template Call<OutValue>(ctx, arg0_val, &st), out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename Op, typename OutType>
+struct ScalarUnaryTemporal {
+  using OutValue = typename internal::GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<Op, OutType> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding() -
+        1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    return static_cast<T>(
+        (t - sys_time<days>(year_month_day(t).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+//
+// First week of an ISO year has the majority (4 or more) of it's days in January.
+// Last week of an ISO year has the year's last Thursday in it.
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    auto y = year_month_day{t + days{3}}.year();
+    auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (t < start) {
+      --y;
+    }
+    return static_cast<T>(static_cast<int32_t>(y));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+//
+// First week of an ISO year has the majority (4 or more) of it's days in January.
+// Last week of an ISO year has the year's last Thursday in it.
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    auto y = year_month_day{t + days{3}}.year();
+    auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (t < start) {
+      --y;
+      start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(t - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::minutes>(t)) / std::chrono::seconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        (std::chrono::duration<double>(t - floor<std::chrono::seconds>(t)).count()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+template <typename Duration>
+inline std::vector<int64_t> get_iso_calendar(int64_t arg) {
+  const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+  const auto ymd = year_month_day(t);
+  auto y = year_month_day{t + days{3}}.year();
+  auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+  if (t < start) {
+    --y;
+    start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+  }
+  return {static_cast<int64_t>(static_cast<int32_t>(y)),
+          static_cast<int64_t>(trunc<weeks>(t - start).count() + 1),
+          static_cast<int64_t>(weekday(ymd).iso_encoding())};
+}
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+template <typename Duration>
+struct ISOCalendar {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const std::string timezone =
+        checked_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    if (in.is_valid) {
+      const std::shared_ptr<DataType> iso_calendar_type =
+          struct_({field("iso_year", int64()), field("iso_week", int64()),
+                   field("day_of_week", int64())});
+      const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+      const auto iso_calendar = get_iso_calendar<Duration>(in_val);
+
+      std::vector<std::shared_ptr<Scalar>> values = {
+          std::make_shared<Int64Scalar>(iso_calendar[0]),
+          std::make_shared<Int64Scalar>(iso_calendar[1]),
+          std::make_shared<Int64Scalar>(iso_calendar[2])};
+      *checked_cast<StructScalar*>(out) = StructScalar(values, iso_calendar_type);
+    } else {
+      out->is_valid = false;
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    using BuilderType = typename TypeTraits<Int64Type>::BuilderType;
+    const std::string timezone =
+        checked_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+    const std::shared_ptr<DataType> iso_calendar_type =
+        struct_({field("iso_year", int64()), field("iso_week", int64()),
+                 field("day_of_week", int64())});
+
+    std::unique_ptr<ArrayBuilder> array_builder;
+    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), iso_calendar_type, &array_builder));
+    StructBuilder* struct_builder = checked_cast<StructBuilder*>(array_builder.get());
+    RETURN_NOT_OK(struct_builder->Reserve(in.length));
+
+    std::vector<BuilderType*> field_builders;
+    field_builders.reserve(3);
+    for (int i = 0; i < 3; i++) {
+      field_builders.push_back(
+          checked_cast<BuilderType*>(struct_builder->field_builder(i)));
+      RETURN_NOT_OK(field_builders[i]->Reserve(1));
+    }
+    auto visit_null = [&]() { return struct_builder->AppendNull(); };
+    auto visit_value = [&](int64_t arg) {
+      const auto iso_calendar = get_iso_calendar<Duration>(arg);
+      field_builders[0]->UnsafeAppend(iso_calendar[0]);
+      field_builders[1]->UnsafeAppend(iso_calendar[1]);
+      field_builders[2]->UnsafeAppend(iso_calendar[2]);
+      return struct_builder->Append();
+    };
+    RETURN_NOT_OK(VisitArrayDataInline<Int64Type>(in, visit_value, visit_null));
+
+    std::shared_ptr<Array> out_array;
+    RETURN_NOT_OK(struct_builder->Finish(&out_array));
+    *out = *std::move(out_array->data());
+
+    return Status::OK();
+  }
+};
+
+template <template <typename...> class Op, typename OutType>
+std::shared_ptr<ScalarFunction> MakeTemporal(std::string name, const FunctionDoc* doc) {
+  const auto& out_type = TypeTraits<OutType>::type_singleton();
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+
+  for (auto unit : internal::AllTimeUnits()) {
+    InputType in_type{match::TimestampTypeUnit(unit)};
+    switch (unit) {
+      case TimeUnit::SECOND: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::seconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MILLI: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::milliseconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MICRO: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::microseconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::NANO: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::nanoseconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+    }
+  }
+  return func;
+}
+
+template <template <typename...> class Op>
+std::shared_ptr<ScalarFunction> MakeSimpleTemporal(std::string name,
+                                                   const FunctionDoc* doc) {
+  const auto& out_type = struct_({field("iso_year", int64()), field("iso_week", int64()),
+                                  field("day_of_week", int64())});
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+
+  for (auto unit : internal::AllTimeUnits()) {
+    InputType in_type{match::TimestampTypeUnit(unit)};
+    switch (unit) {
+      case TimeUnit::SECOND: {
+        auto exec = SimpleUnary<Op<std::chrono::seconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MILLI: {
+        auto exec = SimpleUnary<Op<std::chrono::milliseconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MICRO: {
+        auto exec = SimpleUnary<Op<std::chrono::microseconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::NANO: {
+        auto exec = SimpleUnary<Op<std::chrono::nanoseconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+    }
+  }
+  return func;
+}
+
+const FunctionDoc year_doc{
+    "Extract year from timestamp",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc month_doc{
+    "Extract month number",
+    ("Month is encoded as January=1, December=12.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc day_doc{
+    "Extract day number",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc day_of_week_doc{
+    "Extract day of the week number",
+    ("Week starts on Monday denoted by 0 and ends on Sunday denoted by 6.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc day_of_year_doc{
+    "Extract number of day of year",
+    ("January 1st maps to day number 1, February 1st to 32, etc.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc iso_year_doc{
+    "Extract ISO year number",
+    ("First week of an ISO year has the majority (4 or more) of it's days in January."
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc iso_week_doc{
+    "Extract ISO week of year number",
+    ("First ISO week has the majority (4 or more) of it's days in January.\n"
+     "Week of the year starts with 1 and can run up to 53.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc iso_calendar_doc{
+    "Extract (ISO year, ISO week, day of week) struct",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc quarter_doc{
+    "Extract quarter of year number",
+    ("First quarter maps to 1 and forth quarter maps to 4.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc hour_doc{
+    "Extract hour values",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc minute_doc{
+    "Extract minute values",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc second_doc{
+    "Extract second values",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc millisecond_doc{
+    "Extract millisecond values",

Review comment:
       Done!

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,626 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+
+namespace internal {
+
+namespace {
+
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+using internal::applicator::ScalarUnaryNotNull;
+using internal::applicator::SimpleUnary;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename Op, typename OutType>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<Op, OutType>;
+  using OutValue = typename internal::GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          checked_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        internal::VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        checked_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = internal::UnboxScalar<Int64Type>::Unbox(arg0);
+        internal::BoxScalar<OutType>::Box(
+            this->op.template Call<OutValue>(ctx, arg0_val, &st), out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename Op, typename OutType>
+struct ScalarUnaryTemporal {
+  using OutValue = typename internal::GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<Op, OutType> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding() -
+        1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    return static_cast<T>(
+        (t - sys_time<days>(year_month_day(t).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+//
+// First week of an ISO year has the majority (4 or more) of it's days in January.
+// Last week of an ISO year has the year's last Thursday in it.
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    auto y = year_month_day{t + days{3}}.year();
+    auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (t < start) {
+      --y;
+    }
+    return static_cast<T>(static_cast<int32_t>(y));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+//
+// First week of an ISO year has the majority (4 or more) of it's days in January.
+// Last week of an ISO year has the year's last Thursday in it.
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    auto y = year_month_day{t + days{3}}.year();
+    auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (t < start) {
+      --y;
+      start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(t - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::minutes>(t)) / std::chrono::seconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        (std::chrono::duration<double>(t - floor<std::chrono::seconds>(t)).count()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+template <typename Duration>
+inline std::vector<int64_t> get_iso_calendar(int64_t arg) {
+  const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+  const auto ymd = year_month_day(t);
+  auto y = year_month_day{t + days{3}}.year();
+  auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+  if (t < start) {
+    --y;
+    start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+  }
+  return {static_cast<int64_t>(static_cast<int32_t>(y)),
+          static_cast<int64_t>(trunc<weeks>(t - start).count() + 1),
+          static_cast<int64_t>(weekday(ymd).iso_encoding())};
+}
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+template <typename Duration>
+struct ISOCalendar {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const std::string timezone =
+        checked_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    if (in.is_valid) {
+      const std::shared_ptr<DataType> iso_calendar_type =
+          struct_({field("iso_year", int64()), field("iso_week", int64()),
+                   field("day_of_week", int64())});
+      const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+      const auto iso_calendar = get_iso_calendar<Duration>(in_val);
+
+      std::vector<std::shared_ptr<Scalar>> values = {
+          std::make_shared<Int64Scalar>(iso_calendar[0]),
+          std::make_shared<Int64Scalar>(iso_calendar[1]),
+          std::make_shared<Int64Scalar>(iso_calendar[2])};
+      *checked_cast<StructScalar*>(out) = StructScalar(values, iso_calendar_type);
+    } else {
+      out->is_valid = false;
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    using BuilderType = typename TypeTraits<Int64Type>::BuilderType;
+    const std::string timezone =
+        checked_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+    const std::shared_ptr<DataType> iso_calendar_type =
+        struct_({field("iso_year", int64()), field("iso_week", int64()),
+                 field("day_of_week", int64())});
+
+    std::unique_ptr<ArrayBuilder> array_builder;
+    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), iso_calendar_type, &array_builder));
+    StructBuilder* struct_builder = checked_cast<StructBuilder*>(array_builder.get());
+    RETURN_NOT_OK(struct_builder->Reserve(in.length));
+
+    std::vector<BuilderType*> field_builders;
+    field_builders.reserve(3);
+    for (int i = 0; i < 3; i++) {
+      field_builders.push_back(
+          checked_cast<BuilderType*>(struct_builder->field_builder(i)));
+      RETURN_NOT_OK(field_builders[i]->Reserve(1));
+    }
+    auto visit_null = [&]() { return struct_builder->AppendNull(); };
+    auto visit_value = [&](int64_t arg) {
+      const auto iso_calendar = get_iso_calendar<Duration>(arg);
+      field_builders[0]->UnsafeAppend(iso_calendar[0]);
+      field_builders[1]->UnsafeAppend(iso_calendar[1]);
+      field_builders[2]->UnsafeAppend(iso_calendar[2]);
+      return struct_builder->Append();
+    };
+    RETURN_NOT_OK(VisitArrayDataInline<Int64Type>(in, visit_value, visit_null));
+
+    std::shared_ptr<Array> out_array;
+    RETURN_NOT_OK(struct_builder->Finish(&out_array));
+    *out = *std::move(out_array->data());
+
+    return Status::OK();
+  }
+};
+
+template <template <typename...> class Op, typename OutType>
+std::shared_ptr<ScalarFunction> MakeTemporal(std::string name, const FunctionDoc* doc) {
+  const auto& out_type = TypeTraits<OutType>::type_singleton();
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+
+  for (auto unit : internal::AllTimeUnits()) {
+    InputType in_type{match::TimestampTypeUnit(unit)};
+    switch (unit) {
+      case TimeUnit::SECOND: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::seconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MILLI: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::milliseconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MICRO: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::microseconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::NANO: {
+        auto exec = ScalarUnaryTemporal<Op<std::chrono::nanoseconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+    }
+  }
+  return func;
+}
+
+template <template <typename...> class Op>
+std::shared_ptr<ScalarFunction> MakeSimpleTemporal(std::string name,
+                                                   const FunctionDoc* doc) {
+  const auto& out_type = struct_({field("iso_year", int64()), field("iso_week", int64()),
+                                  field("day_of_week", int64())});
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+
+  for (auto unit : internal::AllTimeUnits()) {
+    InputType in_type{match::TimestampTypeUnit(unit)};
+    switch (unit) {
+      case TimeUnit::SECOND: {
+        auto exec = SimpleUnary<Op<std::chrono::seconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MILLI: {
+        auto exec = SimpleUnary<Op<std::chrono::milliseconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::MICRO: {
+        auto exec = SimpleUnary<Op<std::chrono::microseconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+      case TimeUnit::NANO: {
+        auto exec = SimpleUnary<Op<std::chrono::nanoseconds>>;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec)));
+        break;
+      }
+    }
+  }
+  return func;
+}
+
+const FunctionDoc year_doc{
+    "Extract year from timestamp",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc month_doc{
+    "Extract month number",
+    ("Month is encoded as January=1, December=12.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc day_doc{
+    "Extract day number",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc day_of_week_doc{
+    "Extract day of the week number",
+    ("Week starts on Monday denoted by 0 and ends on Sunday denoted by 6.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc day_of_year_doc{
+    "Extract number of day of year",
+    ("January 1st maps to day number 1, February 1st to 32, etc.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc iso_year_doc{
+    "Extract ISO year number",
+    ("First week of an ISO year has the majority (4 or more) of it's days in January."
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc iso_week_doc{
+    "Extract ISO week of year number",
+    ("First ISO week has the majority (4 or more) of it's days in January.\n"
+     "Week of the year starts with 1 and can run up to 53.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc iso_calendar_doc{
+    "Extract (ISO year, ISO week, day of week) struct",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc quarter_doc{
+    "Extract quarter of year number",
+    ("First quarter maps to 1 and forth quarter maps to 4.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null."),
+    {"values"}};
+
+const FunctionDoc hour_doc{
+    "Extract hour values",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc minute_doc{
+    "Extract minute values",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc second_doc{
+    "Extract second values",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc millisecond_doc{
+    "Extract millisecond values",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc microsecond_doc{
+    "Extract microsecond values",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc nanosecond_doc{
+    "Extract nanosecond values",
+    "Returns an error if timestamp has a defined timezone. Null values return null.",
+    {"values"}};
+
+const FunctionDoc subsecond_doc{
+    "Extract subsecond values",

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.

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



[GitHub] [arrow] pitrou commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -462,5 +462,177 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`

Review comment:
       What is "month value"? Do you mean "month number"? Does it start at 0 or 1?

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -462,5 +462,177 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOYear returns ISO year value for each element of `values`
+///
+/// \param[in] values input to extract ISO year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> ISOYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOWeek returns ISO week of year value for each element of `values`

Review comment:
       "week number"? Does it start at 0, 1?

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -462,5 +462,177 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`

Review comment:
       "Number of day in the week"? Does it start at 0, 1? For Monday, Sunday?

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -462,5 +462,177 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`

Review comment:
       "Number of day in the year"? Does it start at 0 or 1?

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,107 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestSimpleTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO);
+  auto timestamps = ArrayFromJSON(unit, times);
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("weekday", int64())});
+
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar = ArrayFromJSON(iso_calendar_type,
+                                    R"([{"iso_year": 1970, "iso_week": 1, "weekday": 4},
+                        {"iso_year": 2000, "iso_week": 9, "weekday": 2},
+                        {"iso_year": 1899, "iso_week": 52, "weekday": 7},
+                        {"iso_year": 2033, "iso_week": 20, "weekday": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59.123456789, 23.999999999, 20.001001001, 20.0, null]";
+  auto millisecond = "[123, 999, 1, 0, null]";
+  auto microsecond = "[456, 999, 1, 0, null]";
+  auto nanosecond = "[789, 999, 1, 0, null]";
+  auto subsecond = "[123456789, 999999999, 1001001, 0, null]";
+
+  CheckScalarUnary("year", unit, times, int64(), year);
+  CheckScalarUnary("month", unit, times, int64(), month);
+  CheckScalarUnary("day", unit, times, int64(), day);
+  CheckScalarUnary("day_of_week", unit, times, int64(), day_of_week);
+  CheckScalarUnary("day_of_year", unit, times, int64(), day_of_year);
+  CheckScalarUnary("iso_year", unit, times, int64(), iso_year);
+  CheckScalarUnary("iso_week", unit, times, int64(), iso_week);
+  CheckScalarUnary("iso_calendar", timestamps, iso_calendar);
+  CheckScalarUnary("quarter", unit, times, int64(), quarter);
+  CheckScalarUnary("hour", unit, times, int64(), hour);
+  CheckScalarUnary("minute", unit, times, int64(), minute);
+  CheckScalarUnary("second", unit, times, float64(), second);
+  CheckScalarUnary("millisecond", unit, times, int64(), millisecond);
+  CheckScalarUnary("microsecond", unit, times, int64(), microsecond);
+  CheckScalarUnary("nanosecond", unit, times, int64(), nanosecond);
+  CheckScalarUnary("subsecond", unit, times, int64(), subsecond);

Review comment:
       Is there any point in `millisecond` and friends? The floating-point `second` should have enough precision to represent them all exactly.

##########
File path: docs/source/cpp/compute.rst
##########
@@ -637,6 +637,54 @@ String extraction
   e.g. 'letter' and 'digit' for the regular expression
   ``(?P<letter>[ab])(?P<digit>\\d)``.
 
+Temporal component extraction
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+These functions extract datetime components (year, month, day, etc) from timestamp type.
+Note: timezone information is currently ignored if present.
+
++--------------------+------------+-------------------+-----------------+--------+
+| Function name      | Arity      | Input types       | Output type     | Notes  |
++====================+============+===================+=================+========+
+| year               | Unary      | Temporal          | Numeric         |        |

Review comment:
       The output types should be given more precisely here.

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.

Review comment:
       Since most of this file consists of private class/function implementations, can you put them under the anonymous namespace? it avoids exposing the symbols externally.

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return static_cast<T>((sd - sys_days(year_month_day(sd).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day{sys_days{floor<days>(Duration{arg})} + days{3}}.year()));

Review comment:
       Can you add a comment explaining why adding 3 days is necessary?

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(

Review comment:
       Can you explain why two successive casts are needed?

##########
File path: docs/source/cpp/compute.rst
##########
@@ -637,6 +637,54 @@ String extraction
   e.g. 'letter' and 'digit' for the regular expression
   ``(?P<letter>[ab])(?P<digit>\\d)``.
 
+Temporal component extraction
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+These functions extract datetime components (year, month, day, etc) from timestamp type.
+Note: timezone information is currently ignored if present.
+
++--------------------+------------+-------------------+-----------------+--------+
+| Function name      | Arity      | Input types       | Output type     | Notes  |
++====================+============+===================+=================+========+
+| year               | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| month              | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| day                | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| day_of_week        | Unary      | Temporal          | Numeric         |        |

Review comment:
       Can you also add a note explaining this?

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return static_cast<T>((sd - sys_days(year_month_day(sd).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day{sys_days{floor<days>(Duration{arg})} + days{3}}.year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto dp = sys_days{floor<days>(Duration{arg})};
+    auto y = year_month_day{dp + days{3}}.year();
+    auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (dp < start) {
+      --y;
+      start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(dp - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        std::chrono::duration<double>(t - floor<std::chrono::minutes>(t)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::seconds>(t)) /
+                          std::chrono::nanoseconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+template <typename Duration, typename OutType>
+struct ISOCalendar {
+  using T = typename OutType::c_type;
+
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    using ScalarType = typename TypeTraits<OutType>::ScalarType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    if (!in.is_valid) {
+      out->is_valid = false;
+    } else {
+      const std::shared_ptr<DataType> iso_calendar_type =
+          struct_({field("iso_year", out_type), field("iso_week", out_type),
+                   field("weekday", out_type)});
+
+      const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+      const auto dp = sys_days{floor<days>(Duration{in_val})};
+      const auto ymd = year_month_day(dp);
+      auto y = year_month_day{dp + days{3}}.year();
+      auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      if (dp < start) {
+        --y;
+        start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      }
+
+      std::vector<std::shared_ptr<Scalar>> values = {
+          std::make_shared<ScalarType>(static_cast<T>(static_cast<int32_t>(ymd.year()))),
+          std::make_shared<ScalarType>(
+              static_cast<T>(trunc<weeks>(dp - start).count() + 1)),
+          std::make_shared<ScalarType>(static_cast<T>(weekday(ymd).iso_encoding()))};
+      *checked_cast<StructScalar*>(out) = StructScalar(values, iso_calendar_type);
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    using BuilderType = typename TypeTraits<OutType>::BuilderType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    const std::shared_ptr<DataType> iso_calendar_type =
+        struct_({field("iso_year", out_type), field("iso_week", out_type),
+                 field("weekday", out_type)});
+
+    std::unique_ptr<ArrayBuilder> array_builder;
+    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), iso_calendar_type, &array_builder));
+    StructBuilder* struct_builder = checked_cast<StructBuilder*>(array_builder.get());
+
+    std::vector<BuilderType*> field_builders;
+    field_builders.reserve(3);
+    for (int i = 0; i < 3; i++) {
+      field_builders.push_back(
+          checked_cast<BuilderType*>(struct_builder->field_builder(i)));

Review comment:
       You can call `Reserve` here to presize the field arrays. Then you call `UnsafeAppend` below.

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,107 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestSimpleTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";

Review comment:
       Can you test a bit more boundary cases, especially for `day_of_week` and `iso_week`? Start of year, end of year, start of week, end of week, etc.

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return static_cast<T>((sd - sys_days(year_month_day(sd).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day{sys_days{floor<days>(Duration{arg})} + days{3}}.year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto dp = sys_days{floor<days>(Duration{arg})};
+    auto y = year_month_day{dp + days{3}}.year();
+    auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (dp < start) {
+      --y;
+      start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(dp - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        std::chrono::duration<double>(t - floor<std::chrono::minutes>(t)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::seconds>(t)) /
+                          std::chrono::nanoseconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+template <typename Duration, typename OutType>
+struct ISOCalendar {
+  using T = typename OutType::c_type;
+
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    using ScalarType = typename TypeTraits<OutType>::ScalarType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();

Review comment:
       Please use `checked_pointer_cast`.

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return static_cast<T>((sd - sys_days(year_month_day(sd).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day{sys_days{floor<days>(Duration{arg})} + days{3}}.year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto dp = sys_days{floor<days>(Duration{arg})};
+    auto y = year_month_day{dp + days{3}}.year();
+    auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (dp < start) {
+      --y;
+      start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(dp - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of quarter from timestamp

Review comment:
       Are you sure this is the "day of quarter"? 

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return static_cast<T>((sd - sys_days(year_month_day(sd).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day{sys_days{floor<days>(Duration{arg})} + days{3}}.year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto dp = sys_days{floor<days>(Duration{arg})};
+    auto y = year_month_day{dp + days{3}}.year();
+    auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (dp < start) {
+      --y;
+      start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(dp - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        std::chrono::duration<double>(t - floor<std::chrono::minutes>(t)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::seconds>(t)) /
+                          std::chrono::nanoseconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+template <typename Duration, typename OutType>
+struct ISOCalendar {
+  using T = typename OutType::c_type;
+
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    using ScalarType = typename TypeTraits<OutType>::ScalarType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    if (!in.is_valid) {
+      out->is_valid = false;
+    } else {
+      const std::shared_ptr<DataType> iso_calendar_type =
+          struct_({field("iso_year", out_type), field("iso_week", out_type),
+                   field("weekday", out_type)});
+
+      const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+      const auto dp = sys_days{floor<days>(Duration{in_val})};
+      const auto ymd = year_month_day(dp);
+      auto y = year_month_day{dp + days{3}}.year();
+      auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      if (dp < start) {
+        --y;
+        start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      }
+
+      std::vector<std::shared_ptr<Scalar>> values = {
+          std::make_shared<ScalarType>(static_cast<T>(static_cast<int32_t>(ymd.year()))),
+          std::make_shared<ScalarType>(
+              static_cast<T>(trunc<weeks>(dp - start).count() + 1)),
+          std::make_shared<ScalarType>(static_cast<T>(weekday(ymd).iso_encoding()))};
+      *checked_cast<StructScalar*>(out) = StructScalar(values, iso_calendar_type);
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    using BuilderType = typename TypeTraits<OutType>::BuilderType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    const std::shared_ptr<DataType> iso_calendar_type =
+        struct_({field("iso_year", out_type), field("iso_week", out_type),
+                 field("weekday", out_type)});
+
+    std::unique_ptr<ArrayBuilder> array_builder;
+    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), iso_calendar_type, &array_builder));
+    StructBuilder* struct_builder = checked_cast<StructBuilder*>(array_builder.get());
+
+    std::vector<BuilderType*> field_builders;
+    field_builders.reserve(3);
+    for (int i = 0; i < 3; i++) {
+      field_builders.push_back(
+          checked_cast<BuilderType*>(struct_builder->field_builder(i)));
+    }
+    auto visit_null = [&]() { return struct_builder->AppendNull(); };
+    auto visit_value = [&](int64_t arg) {
+      const auto dp = sys_days{floor<days>(Duration{arg})};
+      const auto ymd = year_month_day(dp);
+      auto y = year_month_day{dp + days{3}}.year();
+      auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      if (dp < start) {
+        --y;
+        start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      }
+
+      RETURN_NOT_OK(
+          field_builders[0]->Append(static_cast<T>(static_cast<int32_t>(ymd.year()))));
+      RETURN_NOT_OK(field_builders[1]->Append(
+          static_cast<T>(trunc<weeks>(dp - start).count() + 1)));
+      RETURN_NOT_OK(
+          field_builders[2]->Append(static_cast<T>(weekday(ymd).iso_encoding())));
+
+      return struct_builder->Append();
+    };
+    RETURN_NOT_OK(VisitArrayDataInline<OutType>(in, visit_value, visit_null));
+
+    std::shared_ptr<Array> out_array;
+    RETURN_NOT_OK(struct_builder->Finish(&out_array));
+    *out = *std::move(out_array->data());
+
+    return Status::OK();
+  }
+};
+
+// Generate a kernel given an arithmetic functor
+template <template <typename... Args> class KernelGenerator,
+          template <typename... Args> class Op, typename Duration>
+ArrayKernelExec ExecFromOp(detail::GetTypeId get_id) {
+  switch (get_id.id) {
+    case Type::INT8:
+      return KernelGenerator<Int8Type, Op<Duration>>::Exec;
+    case Type::UINT8:
+      return KernelGenerator<UInt8Type, Op<Duration>>::Exec;
+    case Type::INT16:
+      return KernelGenerator<Int16Type, Op<Duration>>::Exec;
+    case Type::UINT16:
+      return KernelGenerator<UInt16Type, Op<Duration>>::Exec;
+    case Type::INT32:
+      return KernelGenerator<Int32Type, Op<Duration>>::Exec;
+    case Type::UINT32:
+      return KernelGenerator<UInt32Type, Op<Duration>>::Exec;
+    case Type::INT64:
+    case Type::TIMESTAMP:
+      return KernelGenerator<Int64Type, Op<Duration>>::Exec;
+    case Type::UINT64:
+      return KernelGenerator<UInt64Type, Op<Duration>>::Exec;
+    case Type::FLOAT:
+      return KernelGenerator<FloatType, Op<Duration>>::Exec;
+    case Type::DOUBLE:
+      return KernelGenerator<DoubleType, Op<Duration>>::Exec;
+    default:
+      DCHECK(false);
+      return ExecFail;
+  }
+}
+
+template <template <typename...> class Op>
+std::shared_ptr<ScalarFunction> MakeTemporalFunction(
+    std::string name, const FunctionDoc* doc,
+    std::vector<std::shared_ptr<DataType>> types) {
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+
+  for (auto ty : types) {
+    for (auto unit : AllTimeUnits()) {
+      ArrayKernelExec exec;
+      switch (unit) {
+        case TimeUnit::SECOND: {
+          exec = ExecFromOp<ScalarUnaryTemporal, Op, std::chrono::seconds>(ty);
+        }
+        case TimeUnit::MILLI: {
+          exec = ExecFromOp<ScalarUnaryTemporal, Op, std::chrono::milliseconds>(ty);
+        }
+        case TimeUnit::MICRO: {
+          exec = ExecFromOp<ScalarUnaryTemporal, Op, std::chrono::microseconds>(ty);
+        }
+        case TimeUnit::NANO: {
+          exec = ExecFromOp<ScalarUnaryTemporal, Op, std::chrono::nanoseconds>(ty);
+        }
+      }
+      ScalarKernel kernel =
+          ScalarKernel({match::TimestampTypeUnit(unit)}, OutputType(ty), exec);
+      DCHECK_OK(func->AddKernel(kernel));
+    }
+  }
+  return func;
+}
+
+// Generate a kernel given an arithmetic functor
+template <template <typename... Args> class Op, typename Duration>
+ArrayKernelExec SimpleUnaryFromOp(detail::GetTypeId get_id) {
+  switch (get_id.id) {
+    case Type::INT32:
+      return SimpleUnary<Op<Duration, Int32Type>>;
+    case Type::INT64:
+      return SimpleUnary<Op<Duration, Int64Type>>;
+    default:
+      DCHECK(false);
+      return ExecFail;
+  }
+}
+
+template <template <typename...> class Op>
+std::shared_ptr<ScalarFunction> MakeSimpleUnaryTemporalFunction(
+    std::string name, const FunctionDoc* doc,
+    std::vector<std::shared_ptr<DataType>> types) {
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+
+  for (auto ty : types) {
+    for (auto unit : AllTimeUnits()) {
+      ArrayKernelExec exec;
+      switch (unit) {
+        case TimeUnit::SECOND: {
+          exec = SimpleUnaryFromOp<Op, std::chrono::seconds>(ty);
+        }
+        case TimeUnit::MILLI: {
+          exec = SimpleUnaryFromOp<Op, std::chrono::milliseconds>(ty);
+        }
+        case TimeUnit::MICRO: {
+          exec = SimpleUnaryFromOp<Op, std::chrono::microseconds>(ty);
+        }
+        case TimeUnit::NANO: {
+          exec = SimpleUnaryFromOp<Op, std::chrono::nanoseconds>(ty);
+        }
+      }
+      auto output_type =
+          struct_({field("iso_year", ty), field("iso_week", ty), field("weekday", ty)});
+      ScalarKernel kernel =
+          ScalarKernel({match::TimestampTypeUnit(unit)}, OutputType(output_type), exec);
+      DCHECK_OK(func->AddKernel(kernel));
+    }
+  }
+  return func;
+}
+
+const FunctionDoc year_doc{"Extract year values", "", {"values"}};
+const FunctionDoc month_doc{"Extract month values", "", {"values"}};
+const FunctionDoc day_doc{"Extract day values", "", {"values"}};
+const FunctionDoc day_of_week_doc{"Extract day of week values", "", {"values"}};
+const FunctionDoc day_of_year_doc{"Extract day of year values", "", {"values"}};
+const FunctionDoc iso_year_doc{"Extract ISO year values", "", {"values"}};
+const FunctionDoc iso_week_doc{"Extract ISO week values", "", {"values"}};
+const FunctionDoc iso_calendar_doc{"Extract ISO calendar values", "", {"values"}};
+const FunctionDoc quarter_doc{"Extract quarter values", "", {"values"}};
+const FunctionDoc hour_doc{"Extract hour values", "", {"values"}};
+const FunctionDoc minute_doc{"Extract minute values", "", {"values"}};
+const FunctionDoc second_doc{"Extract second values", "", {"values"}};
+const FunctionDoc millisecond_doc{"Extract millisecond values", "", {"values"}};
+const FunctionDoc microsecond_doc{"Extract microsecond values", "", {"values"}};
+const FunctionDoc nanosecond_doc{"Extract nanosecond values", "", {"values"}};
+const FunctionDoc subsecond_doc{"Extract subsecond values", "", {"values"}};
+
+void RegisterScalarTemporal(FunctionRegistry* registry) {
+  static std::vector<std::shared_ptr<DataType>> kUnsignedFloatTypes8 = {
+      uint8(), int8(),   uint16(), int16(),   uint32(),
+      int32(), uint64(), int64(),  float32(), float64()};
+  static std::vector<std::shared_ptr<DataType>> kUnsignedIntegerTypes8 = {
+      uint8(), int8(), uint16(), int16(), uint32(), int32(), uint64(), int64()};
+  static std::vector<std::shared_ptr<DataType>> kUnsignedIntegerTypes16 = {
+      uint16(), int16(), uint32(), int32(), uint64(), int64()};
+  static std::vector<std::shared_ptr<DataType>> kUnsignedIntegerTypes32 = {
+      uint32(), int32(), uint64(), int64()};
+  static std::vector<std::shared_ptr<DataType>> kSignedIntegerTypes = {int32(), int64()};

Review comment:
       I don't understand why you're having all these types. The input type is timestamp, right?

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return static_cast<T>((sd - sys_days(year_month_day(sd).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day{sys_days{floor<days>(Duration{arg})} + days{3}}.year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto dp = sys_days{floor<days>(Duration{arg})};
+    auto y = year_month_day{dp + days{3}}.year();
+    auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (dp < start) {
+      --y;
+      start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(dp - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        std::chrono::duration<double>(t - floor<std::chrono::minutes>(t)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::seconds>(t)) /
+                          std::chrono::nanoseconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+template <typename Duration, typename OutType>
+struct ISOCalendar {
+  using T = typename OutType::c_type;
+
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    using ScalarType = typename TypeTraits<OutType>::ScalarType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    if (!in.is_valid) {
+      out->is_valid = false;
+    } else {
+      const std::shared_ptr<DataType> iso_calendar_type =
+          struct_({field("iso_year", out_type), field("iso_week", out_type),
+                   field("weekday", out_type)});
+
+      const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+      const auto dp = sys_days{floor<days>(Duration{in_val})};
+      const auto ymd = year_month_day(dp);
+      auto y = year_month_day{dp + days{3}}.year();
+      auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      if (dp < start) {
+        --y;
+        start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      }
+
+      std::vector<std::shared_ptr<Scalar>> values = {
+          std::make_shared<ScalarType>(static_cast<T>(static_cast<int32_t>(ymd.year()))),
+          std::make_shared<ScalarType>(
+              static_cast<T>(trunc<weeks>(dp - start).count() + 1)),
+          std::make_shared<ScalarType>(static_cast<T>(weekday(ymd).iso_encoding()))};
+      *checked_cast<StructScalar*>(out) = StructScalar(values, iso_calendar_type);
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    using BuilderType = typename TypeTraits<OutType>::BuilderType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    const std::shared_ptr<DataType> iso_calendar_type =
+        struct_({field("iso_year", out_type), field("iso_week", out_type),
+                 field("weekday", out_type)});
+
+    std::unique_ptr<ArrayBuilder> array_builder;
+    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), iso_calendar_type, &array_builder));
+    StructBuilder* struct_builder = checked_cast<StructBuilder*>(array_builder.get());
+
+    std::vector<BuilderType*> field_builders;
+    field_builders.reserve(3);
+    for (int i = 0; i < 3; i++) {
+      field_builders.push_back(
+          checked_cast<BuilderType*>(struct_builder->field_builder(i)));
+    }
+    auto visit_null = [&]() { return struct_builder->AppendNull(); };
+    auto visit_value = [&](int64_t arg) {
+      const auto dp = sys_days{floor<days>(Duration{arg})};
+      const auto ymd = year_month_day(dp);
+      auto y = year_month_day{dp + days{3}}.year();
+      auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      if (dp < start) {
+        --y;
+        start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      }
+
+      RETURN_NOT_OK(
+          field_builders[0]->Append(static_cast<T>(static_cast<int32_t>(ymd.year()))));
+      RETURN_NOT_OK(field_builders[1]->Append(
+          static_cast<T>(trunc<weeks>(dp - start).count() + 1)));
+      RETURN_NOT_OK(
+          field_builders[2]->Append(static_cast<T>(weekday(ymd).iso_encoding())));
+
+      return struct_builder->Append();
+    };
+    RETURN_NOT_OK(VisitArrayDataInline<OutType>(in, visit_value, visit_null));
+
+    std::shared_ptr<Array> out_array;
+    RETURN_NOT_OK(struct_builder->Finish(&out_array));
+    *out = *std::move(out_array->data());
+
+    return Status::OK();
+  }
+};
+
+// Generate a kernel given an arithmetic functor
+template <template <typename... Args> class KernelGenerator,
+          template <typename... Args> class Op, typename Duration>
+ArrayKernelExec ExecFromOp(detail::GetTypeId get_id) {
+  switch (get_id.id) {
+    case Type::INT8:
+      return KernelGenerator<Int8Type, Op<Duration>>::Exec;
+    case Type::UINT8:
+      return KernelGenerator<UInt8Type, Op<Duration>>::Exec;
+    case Type::INT16:
+      return KernelGenerator<Int16Type, Op<Duration>>::Exec;
+    case Type::UINT16:
+      return KernelGenerator<UInt16Type, Op<Duration>>::Exec;
+    case Type::INT32:
+      return KernelGenerator<Int32Type, Op<Duration>>::Exec;
+    case Type::UINT32:
+      return KernelGenerator<UInt32Type, Op<Duration>>::Exec;
+    case Type::INT64:
+    case Type::TIMESTAMP:
+      return KernelGenerator<Int64Type, Op<Duration>>::Exec;
+    case Type::UINT64:
+      return KernelGenerator<UInt64Type, Op<Duration>>::Exec;
+    case Type::FLOAT:
+      return KernelGenerator<FloatType, Op<Duration>>::Exec;
+    case Type::DOUBLE:
+      return KernelGenerator<DoubleType, Op<Duration>>::Exec;
+    default:
+      DCHECK(false);
+      return ExecFail;
+  }
+}
+
+template <template <typename...> class Op>
+std::shared_ptr<ScalarFunction> MakeTemporalFunction(
+    std::string name, const FunctionDoc* doc,
+    std::vector<std::shared_ptr<DataType>> types) {
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+
+  for (auto ty : types) {
+    for (auto unit : AllTimeUnits()) {
+      ArrayKernelExec exec;
+      switch (unit) {
+        case TimeUnit::SECOND: {
+          exec = ExecFromOp<ScalarUnaryTemporal, Op, std::chrono::seconds>(ty);
+        }
+        case TimeUnit::MILLI: {
+          exec = ExecFromOp<ScalarUnaryTemporal, Op, std::chrono::milliseconds>(ty);
+        }
+        case TimeUnit::MICRO: {
+          exec = ExecFromOp<ScalarUnaryTemporal, Op, std::chrono::microseconds>(ty);
+        }
+        case TimeUnit::NANO: {
+          exec = ExecFromOp<ScalarUnaryTemporal, Op, std::chrono::nanoseconds>(ty);
+        }
+      }
+      ScalarKernel kernel =
+          ScalarKernel({match::TimestampTypeUnit(unit)}, OutputType(ty), exec);
+      DCHECK_OK(func->AddKernel(kernel));
+    }
+  }
+  return func;
+}
+
+// Generate a kernel given an arithmetic functor
+template <template <typename... Args> class Op, typename Duration>
+ArrayKernelExec SimpleUnaryFromOp(detail::GetTypeId get_id) {
+  switch (get_id.id) {
+    case Type::INT32:
+      return SimpleUnary<Op<Duration, Int32Type>>;
+    case Type::INT64:
+      return SimpleUnary<Op<Duration, Int64Type>>;
+    default:
+      DCHECK(false);
+      return ExecFail;
+  }
+}
+
+template <template <typename...> class Op>
+std::shared_ptr<ScalarFunction> MakeSimpleUnaryTemporalFunction(
+    std::string name, const FunctionDoc* doc,
+    std::vector<std::shared_ptr<DataType>> types) {
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+
+  for (auto ty : types) {
+    for (auto unit : AllTimeUnits()) {
+      ArrayKernelExec exec;
+      switch (unit) {
+        case TimeUnit::SECOND: {
+          exec = SimpleUnaryFromOp<Op, std::chrono::seconds>(ty);
+        }
+        case TimeUnit::MILLI: {
+          exec = SimpleUnaryFromOp<Op, std::chrono::milliseconds>(ty);
+        }
+        case TimeUnit::MICRO: {
+          exec = SimpleUnaryFromOp<Op, std::chrono::microseconds>(ty);
+        }
+        case TimeUnit::NANO: {
+          exec = SimpleUnaryFromOp<Op, std::chrono::nanoseconds>(ty);
+        }
+      }
+      auto output_type =
+          struct_({field("iso_year", ty), field("iso_week", ty), field("weekday", ty)});
+      ScalarKernel kernel =
+          ScalarKernel({match::TimestampTypeUnit(unit)}, OutputType(output_type), exec);
+      DCHECK_OK(func->AddKernel(kernel));
+    }
+  }
+  return func;
+}
+
+const FunctionDoc year_doc{"Extract year values", "", {"values"}};
+const FunctionDoc month_doc{"Extract month values", "", {"values"}};
+const FunctionDoc day_doc{"Extract day values", "", {"values"}};
+const FunctionDoc day_of_week_doc{"Extract day of week values", "", {"values"}};
+const FunctionDoc day_of_year_doc{"Extract day of year values", "", {"values"}};
+const FunctionDoc iso_year_doc{"Extract ISO year values", "", {"values"}};
+const FunctionDoc iso_week_doc{"Extract ISO week values", "", {"values"}};
+const FunctionDoc iso_calendar_doc{"Extract ISO calendar values", "", {"values"}};
+const FunctionDoc quarter_doc{"Extract quarter values", "", {"values"}};
+const FunctionDoc hour_doc{"Extract hour values", "", {"values"}};
+const FunctionDoc minute_doc{"Extract minute values", "", {"values"}};
+const FunctionDoc second_doc{"Extract second values", "", {"values"}};
+const FunctionDoc millisecond_doc{"Extract millisecond values", "", {"values"}};
+const FunctionDoc microsecond_doc{"Extract microsecond values", "", {"values"}};
+const FunctionDoc nanosecond_doc{"Extract nanosecond values", "", {"values"}};
+const FunctionDoc subsecond_doc{"Extract subsecond values", "", {"values"}};

Review comment:
       Can you make an effort for these documentations, as for the C++ docstrings? You can take a look at how other compute functions are documented.

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -462,5 +462,177 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`

Review comment:
       "Day number" perhaps?

##########
File path: docs/source/cpp/compute.rst
##########
@@ -637,6 +637,54 @@ String extraction
   e.g. 'letter' and 'digit' for the regular expression
   ``(?P<letter>[ab])(?P<digit>\\d)``.
 
+Temporal component extraction
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+These functions extract datetime components (year, month, day, etc) from timestamp type.
+Note: timezone information is currently ignored if present.
+
++--------------------+------------+-------------------+-----------------+--------+
+| Function name      | Arity      | Input types       | Output type     | Notes  |
++====================+============+===================+=================+========+
+| year               | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| month              | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| day                | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| day_of_week        | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| day_of_year        | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| iso_year           | Unary      | Temporal          | Numeric         | \(1)   |
++--------------------+------------+-------------------+-----------------+--------+
+| iso_week           | Unary      | Temporal          | Numeric         | \(1)   |
++--------------------+------------+-------------------+-----------------+--------+
+| iso_calendar       | Unary      | Temporal          | Scalar Struct   | \(2)   |
++--------------------+------------+-------------------+-----------------+--------+
+| quarter            | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| hour               | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| minute             | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| second             | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| millisecond        | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| microsecond        | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| nanosecond         | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| subsecond          | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+
+* \(1) The ISO 8601 definition for week 01 is the week with the first Thursday
+  of the Gregorian year (i.e. of January) in it.
+  .. _Wikipedia ISO Week date: https://en.wikipedia.org/wiki/ISO_week_date#First_week

Review comment:
       I don't think this is a valid reST hyperlink :-)

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,107 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestSimpleTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO);
+  auto timestamps = ArrayFromJSON(unit, times);
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("weekday", int64())});
+
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar = ArrayFromJSON(iso_calendar_type,
+                                    R"([{"iso_year": 1970, "iso_week": 1, "weekday": 4},

Review comment:
       Should it be `week_day`?

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -462,5 +462,177 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOYear returns ISO year value for each element of `values`
+///
+/// \param[in] values input to extract ISO year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> ISOYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOWeek returns ISO week of year value for each element of `values`
+///
+/// \param[in] values input to extract ISO week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> ISOWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOCalendar returns a (year, ISO week, weekday) struct for each element of
+/// `values`
+///
+/// \param[in] values input to ISO calendar struct from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> ISOCalendar(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns quarter of year value for each element of `values`

Review comment:
       I think you will be able to guess my next questions...

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,107 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestSimpleTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO);
+  auto timestamps = ArrayFromJSON(unit, times);
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("weekday", int64())});
+
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar = ArrayFromJSON(iso_calendar_type,
+                                    R"([{"iso_year": 1970, "iso_week": 1, "weekday": 4},
+                        {"iso_year": 2000, "iso_week": 9, "weekday": 2},
+                        {"iso_year": 1899, "iso_week": 52, "weekday": 7},
+                        {"iso_year": 2033, "iso_week": 20, "weekday": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59.123456789, 23.999999999, 20.001001001, 20.0, null]";
+  auto millisecond = "[123, 999, 1, 0, null]";
+  auto microsecond = "[456, 999, 1, 0, null]";
+  auto nanosecond = "[789, 999, 1, 0, null]";
+  auto subsecond = "[123456789, 999999999, 1001001, 0, null]";
+
+  CheckScalarUnary("year", unit, times, int64(), year);
+  CheckScalarUnary("month", unit, times, int64(), month);
+  CheckScalarUnary("day", unit, times, int64(), day);
+  CheckScalarUnary("day_of_week", unit, times, int64(), day_of_week);
+  CheckScalarUnary("day_of_year", unit, times, int64(), day_of_year);
+  CheckScalarUnary("iso_year", unit, times, int64(), iso_year);
+  CheckScalarUnary("iso_week", unit, times, int64(), iso_week);
+  CheckScalarUnary("iso_calendar", timestamps, iso_calendar);
+  CheckScalarUnary("quarter", unit, times, int64(), quarter);
+  CheckScalarUnary("hour", unit, times, int64(), hour);
+  CheckScalarUnary("minute", unit, times, int64(), minute);
+  CheckScalarUnary("second", unit, times, float64(), second);
+  CheckScalarUnary("millisecond", unit, times, int64(), millisecond);
+  CheckScalarUnary("microsecond", unit, times, int64(), microsecond);
+  CheckScalarUnary("nanosecond", unit, times, int64(), nanosecond);
+  CheckScalarUnary("subsecond", unit, times, int64(), subsecond);
+}
+
+TEST(ScalarTemporalTest, TestZonedTemporalComponentExtraction) {
+  std::string timezone = "Etc/UTC-2";
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO, timezone);
+  auto timestamps = ArrayFromJSON(unit, times);
+
+  ASSERT_RAISES(Invalid, Year(timestamps));
+  ASSERT_RAISES(Invalid, Month(timestamps));
+  ASSERT_RAISES(Invalid, Day(timestamps));
+  ASSERT_RAISES(Invalid, DayOfWeek(timestamps));
+  ASSERT_RAISES(Invalid, DayOfYear(timestamps));
+  ASSERT_RAISES(Invalid, ISOYear(timestamps));
+  ASSERT_RAISES(Invalid, ISOWeek(timestamps));
+  ASSERT_RAISES(Invalid, ISOCalendar(timestamps));
+  ASSERT_RAISES(Invalid, Quarter(timestamps));
+  ASSERT_RAISES(Invalid, Hour(timestamps));
+  ASSERT_RAISES(Invalid, Minute(timestamps));
+  ASSERT_RAISES(Invalid, Second(timestamps));
+  ASSERT_RAISES(Invalid, Millisecond(timestamps));
+  ASSERT_RAISES(Invalid, Microsecond(timestamps));
+  ASSERT_RAISES(Invalid, Nanosecond(timestamps));
+  ASSERT_RAISES(Invalid, Subsecond(timestamps));

Review comment:
       Why are these invalid? IMHO, the timezone should simply ignored.

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -462,5 +462,177 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOYear returns ISO year value for each element of `values`

Review comment:
       What is a "ISO year"?

##########
File path: docs/source/cpp/compute.rst
##########
@@ -637,6 +637,54 @@ String extraction
   e.g. 'letter' and 'digit' for the regular expression
   ``(?P<letter>[ab])(?P<digit>\\d)``.
 
+Temporal component extraction
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+These functions extract datetime components (year, month, day, etc) from timestamp type.
+Note: timezone information is currently ignored if present.
+
++--------------------+------------+-------------------+-----------------+--------+
+| Function name      | Arity      | Input types       | Output type     | Notes  |
++====================+============+===================+=================+========+
+| year               | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| month              | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| day                | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| day_of_week        | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| day_of_year        | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| iso_year           | Unary      | Temporal          | Numeric         | \(1)   |
++--------------------+------------+-------------------+-----------------+--------+
+| iso_week           | Unary      | Temporal          | Numeric         | \(1)   |
++--------------------+------------+-------------------+-----------------+--------+
+| iso_calendar       | Unary      | Temporal          | Scalar Struct   | \(2)   |
++--------------------+------------+-------------------+-----------------+--------+
+| quarter            | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| hour               | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| minute             | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| second             | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| millisecond        | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| microsecond        | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| nanosecond         | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| subsecond          | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+
+* \(1) The ISO 8601 definition for week 01 is the week with the first Thursday
+  of the Gregorian year (i.e. of January) in it.
+  .. _Wikipedia ISO Week date: https://en.wikipedia.org/wiki/ISO_week_date#First_week
+
+* \(2) Struct with fields ISO year, ISO week number, and weekday.

Review comment:
       Can you give the field names precisely? You'll find other examples of this in this documentation file.

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};

Review comment:
       I'm curious: why do you use `floor<days>(sys_time<Duration>(Duration{arg}))` above and `sys_days{floor<days>(Duration{arg})}` here? What's the difference?

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return static_cast<T>((sd - sys_days(year_month_day(sd).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day{sys_days{floor<days>(Duration{arg})} + days{3}}.year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto dp = sys_days{floor<days>(Duration{arg})};
+    auto y = year_month_day{dp + days{3}}.year();
+    auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (dp < start) {
+      --y;
+      start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(dp - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        std::chrono::duration<double>(t - floor<std::chrono::minutes>(t)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::seconds>(t)) /
+                          std::chrono::nanoseconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+template <typename Duration, typename OutType>
+struct ISOCalendar {
+  using T = typename OutType::c_type;
+
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    using ScalarType = typename TypeTraits<OutType>::ScalarType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    if (!in.is_valid) {
+      out->is_valid = false;
+    } else {
+      const std::shared_ptr<DataType> iso_calendar_type =
+          struct_({field("iso_year", out_type), field("iso_week", out_type),
+                   field("weekday", out_type)});
+
+      const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+      const auto dp = sys_days{floor<days>(Duration{in_val})};
+      const auto ymd = year_month_day(dp);
+      auto y = year_month_day{dp + days{3}}.year();
+      auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      if (dp < start) {
+        --y;
+        start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      }
+
+      std::vector<std::shared_ptr<Scalar>> values = {
+          std::make_shared<ScalarType>(static_cast<T>(static_cast<int32_t>(ymd.year()))),
+          std::make_shared<ScalarType>(
+              static_cast<T>(trunc<weeks>(dp - start).count() + 1)),
+          std::make_shared<ScalarType>(static_cast<T>(weekday(ymd).iso_encoding()))};
+      *checked_cast<StructScalar*>(out) = StructScalar(values, iso_calendar_type);
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    using BuilderType = typename TypeTraits<OutType>::BuilderType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    const std::shared_ptr<DataType> iso_calendar_type =
+        struct_({field("iso_year", out_type), field("iso_week", out_type),
+                 field("weekday", out_type)});
+
+    std::unique_ptr<ArrayBuilder> array_builder;
+    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), iso_calendar_type, &array_builder));
+    StructBuilder* struct_builder = checked_cast<StructBuilder*>(array_builder.get());
+
+    std::vector<BuilderType*> field_builders;
+    field_builders.reserve(3);
+    for (int i = 0; i < 3; i++) {
+      field_builders.push_back(
+          checked_cast<BuilderType*>(struct_builder->field_builder(i)));
+    }
+    auto visit_null = [&]() { return struct_builder->AppendNull(); };
+    auto visit_value = [&](int64_t arg) {
+      const auto dp = sys_days{floor<days>(Duration{arg})};
+      const auto ymd = year_month_day(dp);
+      auto y = year_month_day{dp + days{3}}.year();
+      auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      if (dp < start) {
+        --y;
+        start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      }
+
+      RETURN_NOT_OK(
+          field_builders[0]->Append(static_cast<T>(static_cast<int32_t>(ymd.year()))));
+      RETURN_NOT_OK(field_builders[1]->Append(
+          static_cast<T>(trunc<weeks>(dp - start).count() + 1)));
+      RETURN_NOT_OK(
+          field_builders[2]->Append(static_cast<T>(weekday(ymd).iso_encoding())));
+
+      return struct_builder->Append();
+    };
+    RETURN_NOT_OK(VisitArrayDataInline<OutType>(in, visit_value, visit_null));
+
+    std::shared_ptr<Array> out_array;
+    RETURN_NOT_OK(struct_builder->Finish(&out_array));
+    *out = *std::move(out_array->data());
+
+    return Status::OK();
+  }
+};
+
+// Generate a kernel given an arithmetic functor
+template <template <typename... Args> class KernelGenerator,
+          template <typename... Args> class Op, typename Duration>
+ArrayKernelExec ExecFromOp(detail::GetTypeId get_id) {
+  switch (get_id.id) {

Review comment:
       Why is this useful? The only input type accepted is timestamp, right?

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return static_cast<T>((sd - sys_days(year_month_day(sd).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day{sys_days{floor<days>(Duration{arg})} + days{3}}.year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto dp = sys_days{floor<days>(Duration{arg})};
+    auto y = year_month_day{dp + days{3}}.year();
+    auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (dp < start) {
+      --y;
+      start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(dp - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        std::chrono::duration<double>(t - floor<std::chrono::minutes>(t)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::seconds>(t)) /
+                          std::chrono::nanoseconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+template <typename Duration, typename OutType>
+struct ISOCalendar {
+  using T = typename OutType::c_type;
+
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    using ScalarType = typename TypeTraits<OutType>::ScalarType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    if (!in.is_valid) {
+      out->is_valid = false;
+    } else {
+      const std::shared_ptr<DataType> iso_calendar_type =
+          struct_({field("iso_year", out_type), field("iso_week", out_type),
+                   field("weekday", out_type)});
+
+      const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+      const auto dp = sys_days{floor<days>(Duration{in_val})};
+      const auto ymd = year_month_day(dp);
+      auto y = year_month_day{dp + days{3}}.year();
+      auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      if (dp < start) {
+        --y;
+        start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      }

Review comment:
       Can you try to factor out code that's repeated between the `Scalar` and `ArrayData` implementations?

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return static_cast<T>((sd - sys_days(year_month_day(sd).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day{sys_days{floor<days>(Duration{arg})} + days{3}}.year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto dp = sys_days{floor<days>(Duration{arg})};
+    auto y = year_month_day{dp + days{3}}.year();
+    auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (dp < start) {
+      --y;
+      start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(dp - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        std::chrono::duration<double>(t - floor<std::chrono::minutes>(t)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::seconds>(t)) /
+                          std::chrono::nanoseconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+template <typename Duration, typename OutType>
+struct ISOCalendar {
+  using T = typename OutType::c_type;
+
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    using ScalarType = typename TypeTraits<OutType>::ScalarType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    if (!in.is_valid) {
+      out->is_valid = false;
+    } else {
+      const std::shared_ptr<DataType> iso_calendar_type =
+          struct_({field("iso_year", out_type), field("iso_week", out_type),
+                   field("weekday", out_type)});
+
+      const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+      const auto dp = sys_days{floor<days>(Duration{in_val})};
+      const auto ymd = year_month_day(dp);
+      auto y = year_month_day{dp + days{3}}.year();
+      auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      if (dp < start) {
+        --y;
+        start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      }
+
+      std::vector<std::shared_ptr<Scalar>> values = {
+          std::make_shared<ScalarType>(static_cast<T>(static_cast<int32_t>(ymd.year()))),
+          std::make_shared<ScalarType>(
+              static_cast<T>(trunc<weeks>(dp - start).count() + 1)),
+          std::make_shared<ScalarType>(static_cast<T>(weekday(ymd).iso_encoding()))};
+      *checked_cast<StructScalar*>(out) = StructScalar(values, iso_calendar_type);
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    using BuilderType = typename TypeTraits<OutType>::BuilderType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    const std::shared_ptr<DataType> iso_calendar_type =
+        struct_({field("iso_year", out_type), field("iso_week", out_type),
+                 field("weekday", out_type)});
+
+    std::unique_ptr<ArrayBuilder> array_builder;
+    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), iso_calendar_type, &array_builder));
+    StructBuilder* struct_builder = checked_cast<StructBuilder*>(array_builder.get());

Review comment:
       You can also call `Reserve` here to presize the result array. 




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

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -481,5 +481,184 @@ ARROW_EXPORT
 Result<Datum> IfElse(const Datum& cond, const Datum& left, const Datum& right,
                      ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month for each element of `values`.
+/// Month is encoded as January=1, December=12
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day number for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns number of the day of the week value for each element of
+/// `values`. Week starts on Monday denoted by 0 and ends on Sunday denoted by 6.
+///
+/// \param[in] values input to extract number of the day of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns number of day of the year for each element of `values`.
+/// January 1st maps to day number 1, February 1st to 32, etc.
+///
+/// \param[in] values input to extract number of day of the year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOYear returns ISO year number for each element of `values`.
+/// First week of an ISO year has the majority (4 or more) of it's days in January.

Review comment:
       ```suggestion
   /// First week of an ISO year has the majority (4 or more) of its days in January.
   ```

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -481,5 +481,184 @@ ARROW_EXPORT
 Result<Datum> IfElse(const Datum& cond, const Datum& left, const Datum& right,
                      ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month for each element of `values`.
+/// Month is encoded as January=1, December=12
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day number for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns number of the day of the week value for each element of
+/// `values`. Week starts on Monday denoted by 0 and ends on Sunday denoted by 6.
+///
+/// \param[in] values input to extract number of the day of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns number of day of the year for each element of `values`.
+/// January 1st maps to day number 1, February 1st to 32, etc.
+///
+/// \param[in] values input to extract number of day of the year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOYear returns ISO year number for each element of `values`.
+/// First week of an ISO year has the majority (4 or more) of it's days in January.
+///
+/// \param[in] values input to extract ISO year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> ISOYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOWeek returns ISO week of year number for each element of `values`.
+/// First ISO week has the majority (4 or more) of it's days in January.

Review comment:
       ```suggestion
   /// First ISO week has the majority (4 or more) of its days in January.
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,151 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO);
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("day_of_week", int64())});
+
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar =
+      ArrayFromJSON(iso_calendar_type,
+                    R"([{"iso_year": 1970, "iso_week": 1, "day_of_week": 4},
+                        {"iso_year": 2000, "iso_week": 9, "day_of_week": 2},
+                        {"iso_year": 1899, "iso_week": 52, "day_of_week": 7},
+                        {"iso_year": 2033, "iso_week": 20, "day_of_week": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59.123456789, 23.999999999, 20.001001001, 20.0, null]";
+  auto millisecond = "[123, 999, 1, 0, null]";
+  auto microsecond = "[456, 999, 1, 0, null]";
+  auto nanosecond = "[789, 999, 1, 0, null]";
+  auto subsecond = "[123456789, 999999999, 1001001, 0, null]";

Review comment:
       I would have expected a float here, but of course an integer is possible as well (see my question above on the docstring of this function for asking to clarify it, but so my inline suggestion there was assuming it would be a float).
   
   Now, if we return an integer, what is this number exactly? Does it then depend on the unit of the Timestamp, or are it always nanoseconds (even when the unit is eg milliseconds) 
   Returning it as a float would avoid the ambiguity (but of course for certain applications an integer might be more useful).

##########
File path: docs/source/cpp/compute.rst
##########
@@ -637,6 +637,55 @@ String extraction
   e.g. 'letter' and 'digit' for the regular expression
   ``(?P<letter>[ab])(?P<digit>\\d)``.
 
+Temporal component extraction
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+These functions extract datetime components (year, month, day, etc) from timestamp type.
+Note: timezone information is currently ignored if present.
+
++--------------------+------------+-------------------+-----------------+--------+
+| Function name      | Arity      | Input types       | Output type     | Notes  |
++====================+============+===================+=================+========+
+| year               | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| month              | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| day                | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| day_of_week        | Unary      | Temporal          | Numeric         | \(1)   |
++--------------------+------------+-------------------+-----------------+--------+
+| day_of_year        | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| iso_year           | Unary      | Temporal          | Numeric         | \(2)   |
++--------------------+------------+-------------------+-----------------+--------+
+| iso_week           | Unary      | Temporal          | Numeric         | \(2)   |
++--------------------+------------+-------------------+-----------------+--------+
+| iso_calendar       | Unary      | Temporal          | Scalar Struct   | \(3)   |
++--------------------+------------+-------------------+-----------------+--------+
+| quarter            | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| hour               | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| minute             | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| second             | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| millisecond        | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| microsecond        | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| nanosecond         | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| subsecond          | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+
+* \(1) Outputs the number of the day of the week. Week begins on Monday and is denoted
+  by 0 and ends on Sunday denoted by 6.
+* \(2) First ISO week has the majority (4 or more) of it's days in January. ISO year
+  starts with the first ISO week.
+  See `ISO 8601 week date definition`_ for more details.
+* \(3) Output is a ``{"iso_year": output type, "iso_week": output type, "day_of_week":  output type}`` Struct.
+.. _ISO 8601 week date definition: https://en.wikipedia.org/wiki/ISO_week_date#First_week

Review comment:
       Can you leave a blank line above this line? (to separate the list from the link target definition; in general rst often requires blank lines to separate different blocks, although not fully sure it's needed here)

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -481,5 +481,184 @@ ARROW_EXPORT
 Result<Datum> IfElse(const Datum& cond, const Datum& left, const Datum& right,
                      ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month for each element of `values`.
+/// Month is encoded as January=1, December=12
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day number for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns number of the day of the week value for each element of
+/// `values`. Week starts on Monday denoted by 0 and ends on Sunday denoted by 6.
+///
+/// \param[in] values input to extract number of the day of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns number of day of the year for each element of `values`.
+/// January 1st maps to day number 1, February 1st to 32, etc.
+///
+/// \param[in] values input to extract number of day of the year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOYear returns ISO year number for each element of `values`.
+/// First week of an ISO year has the majority (4 or more) of it's days in January.
+///
+/// \param[in] values input to extract ISO year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> ISOYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOWeek returns ISO week of year number for each element of `values`.
+/// First ISO week has the majority (4 or more) of it's days in January.
+/// Week of the year starts with 1 and can run up to 53.
+///
+/// \param[in] values input to extract ISO week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> ISOWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOCalendar returns a (ISO year, ISO week, weekday) struct for each element of
+/// `values`
+///
+/// \param[in] values input to ISO calendar struct from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> ISOCalendar(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns the quarter of year number for each element of `values`
+/// First quarter maps to 1 and forth quarter maps to 4.

Review comment:
       ```suggestion
   /// First quarter maps to 1 and fourth quarter maps to 4.
   ```

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -481,5 +481,184 @@ ARROW_EXPORT
 Result<Datum> IfElse(const Datum& cond, const Datum& left, const Datum& right,
                      ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0

Review comment:
       All those "since 4.0.0" will have to be search/replaced with "since 5.0.0" now

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,151 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO);
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("day_of_week", int64())});
+
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar =
+      ArrayFromJSON(iso_calendar_type,
+                    R"([{"iso_year": 1970, "iso_week": 1, "day_of_week": 4},
+                        {"iso_year": 2000, "iso_week": 9, "day_of_week": 2},
+                        {"iso_year": 1899, "iso_week": 52, "day_of_week": 7},
+                        {"iso_year": 2033, "iso_week": 20, "day_of_week": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59.123456789, 23.999999999, 20.001001001, 20.0, null]";
+  auto millisecond = "[123, 999, 1, 0, null]";
+  auto microsecond = "[456, 999, 1, 0, null]";
+  auto nanosecond = "[789, 999, 1, 0, null]";
+  auto subsecond = "[123456789, 999999999, 1001001, 0, null]";
+
+  CheckScalarUnary("year", unit, times, int64(), year);
+  CheckScalarUnary("month", unit, times, int64(), month);
+  CheckScalarUnary("day", unit, times, int64(), day);
+  CheckScalarUnary("day_of_week", unit, times, int64(), day_of_week);
+  CheckScalarUnary("day_of_year", unit, times, int64(), day_of_year);
+  CheckScalarUnary("iso_year", unit, times, int64(), iso_year);
+  CheckScalarUnary("iso_week", unit, times, int64(), iso_week);
+  CheckScalarUnary("iso_calendar", ArrayFromJSON(unit, times), iso_calendar);
+  CheckScalarUnary("quarter", unit, times, int64(), quarter);
+  CheckScalarUnary("hour", unit, times, int64(), hour);
+  CheckScalarUnary("minute", unit, times, int64(), minute);
+  CheckScalarUnary("second", unit, times, float64(), second);
+  CheckScalarUnary("millisecond", unit, times, int64(), millisecond);
+  CheckScalarUnary("microsecond", unit, times, int64(), microsecond);
+  CheckScalarUnary("nanosecond", unit, times, int64(), nanosecond);
+  CheckScalarUnary("subsecond", unit, times, int64(), subsecond);
+}
+
+TEST(ScalarTemporalTest, TestTemporalComponentExtractionWithDifferentUnits) {
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("day_of_week", int64())});
+  const char* times =
+      R"(["1970-01-01T00:00:59","2000-02-29T23:23:23",
+            "1899-01-01T00:59:20","2033-05-18T03:33:20", null])";
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar =
+      ArrayFromJSON(iso_calendar_type,
+                    R"([{"iso_year": 1970, "iso_week": 1, "day_of_week": 4},
+                          {"iso_year": 2000, "iso_week": 9, "day_of_week": 2},
+                          {"iso_year": 1899, "iso_week": 52, "day_of_week": 7},
+                          {"iso_year": 2033, "iso_week": 20, "day_of_week": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59, 23, 20, 20, null]";
+
+  //  for (auto u : internal::AllTimeUnits()) {
+  for (auto u : {TimeUnit::NANO}) {

Review comment:
       This is not yet testing the different units?

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,151 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO);
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("day_of_week", int64())});
+
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar =
+      ArrayFromJSON(iso_calendar_type,
+                    R"([{"iso_year": 1970, "iso_week": 1, "day_of_week": 4},
+                        {"iso_year": 2000, "iso_week": 9, "day_of_week": 2},
+                        {"iso_year": 1899, "iso_week": 52, "day_of_week": 7},
+                        {"iso_year": 2033, "iso_week": 20, "day_of_week": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59.123456789, 23.999999999, 20.001001001, 20.0, null]";

Review comment:
       I would have expected an integer here, which is the number for second in standard ISO string representation (just like the hour and minute etc)
   
   (so at least this should be documented more clearly, if we keep this behaviour)

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -481,5 +481,184 @@ ARROW_EXPORT
 Result<Datum> IfElse(const Datum& cond, const Datum& left, const Datum& right,
                      ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month for each element of `values`.
+/// Month is encoded as January=1, December=12
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day number for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns number of the day of the week value for each element of
+/// `values`. Week starts on Monday denoted by 0 and ends on Sunday denoted by 6.
+///
+/// \param[in] values input to extract number of the day of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns number of day of the year for each element of `values`.
+/// January 1st maps to day number 1, February 1st to 32, etc.
+///
+/// \param[in] values input to extract number of day of the year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOYear returns ISO year number for each element of `values`.
+/// First week of an ISO year has the majority (4 or more) of it's days in January.
+///
+/// \param[in] values input to extract ISO year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> ISOYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOWeek returns ISO week of year number for each element of `values`.
+/// First ISO week has the majority (4 or more) of it's days in January.
+/// Week of the year starts with 1 and can run up to 53.
+///
+/// \param[in] values input to extract ISO week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> ISOWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOCalendar returns a (ISO year, ISO week, weekday) struct for each element of
+/// `values`
+///
+/// \param[in] values input to ISO calendar struct from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> ISOCalendar(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns the quarter of year number for each element of `values`
+/// First quarter maps to 1 and forth quarter maps to 4.
+///
+/// \param[in] values input to extract quarter of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Quarter(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Hour returns hour value for each element of `values`
+///
+/// \param[in] values input to extract hour from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Hour(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Minute returns minutes value for each element of `values`
+///
+/// \param[in] values input to extract minutes from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Minute(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Second returns seconds value for each element of `values`
+///
+/// \param[in] values input to extract seconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Second(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Millisecond returns milliseconds value for each element of `values`
+///
+/// \param[in] values input to extract milliseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Millisecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Microsecond returns microseconds value for each element of `values`

Review comment:
       I think here (and for nanoseconds below as well), we should clarify what this "microseconds" value is exactly

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,107 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestSimpleTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";

Review comment:
       It would also be good to add a case where `iso_year` is not equal to `year` to cover that boundary case

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,151 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO);
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("day_of_week", int64())});
+
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar =
+      ArrayFromJSON(iso_calendar_type,
+                    R"([{"iso_year": 1970, "iso_week": 1, "day_of_week": 4},
+                        {"iso_year": 2000, "iso_week": 9, "day_of_week": 2},
+                        {"iso_year": 1899, "iso_week": 52, "day_of_week": 7},
+                        {"iso_year": 2033, "iso_week": 20, "day_of_week": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59.123456789, 23.999999999, 20.001001001, 20.0, null]";
+  auto millisecond = "[123, 999, 1, 0, null]";
+  auto microsecond = "[456, 999, 1, 0, null]";
+  auto nanosecond = "[789, 999, 1, 0, null]";
+  auto subsecond = "[123456789, 999999999, 1001001, 0, null]";
+
+  CheckScalarUnary("year", unit, times, int64(), year);
+  CheckScalarUnary("month", unit, times, int64(), month);
+  CheckScalarUnary("day", unit, times, int64(), day);
+  CheckScalarUnary("day_of_week", unit, times, int64(), day_of_week);
+  CheckScalarUnary("day_of_year", unit, times, int64(), day_of_year);
+  CheckScalarUnary("iso_year", unit, times, int64(), iso_year);
+  CheckScalarUnary("iso_week", unit, times, int64(), iso_week);
+  CheckScalarUnary("iso_calendar", ArrayFromJSON(unit, times), iso_calendar);
+  CheckScalarUnary("quarter", unit, times, int64(), quarter);
+  CheckScalarUnary("hour", unit, times, int64(), hour);
+  CheckScalarUnary("minute", unit, times, int64(), minute);
+  CheckScalarUnary("second", unit, times, float64(), second);
+  CheckScalarUnary("millisecond", unit, times, int64(), millisecond);
+  CheckScalarUnary("microsecond", unit, times, int64(), microsecond);
+  CheckScalarUnary("nanosecond", unit, times, int64(), nanosecond);
+  CheckScalarUnary("subsecond", unit, times, int64(), subsecond);
+}
+
+TEST(ScalarTemporalTest, TestTemporalComponentExtractionWithDifferentUnits) {
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("day_of_week", int64())});
+  const char* times =
+      R"(["1970-01-01T00:00:59","2000-02-29T23:23:23",
+            "1899-01-01T00:59:20","2033-05-18T03:33:20", null])";
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar =
+      ArrayFromJSON(iso_calendar_type,
+                    R"([{"iso_year": 1970, "iso_week": 1, "day_of_week": 4},
+                          {"iso_year": 2000, "iso_week": 9, "day_of_week": 2},
+                          {"iso_year": 1899, "iso_week": 52, "day_of_week": 7},
+                          {"iso_year": 2033, "iso_week": 20, "day_of_week": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59, 23, 20, 20, null]";

Review comment:
       Can you also test the other sub-second componenents here? (even for a unit of "second", those components should be defined I think?)

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -481,5 +481,184 @@ ARROW_EXPORT
 Result<Datum> IfElse(const Datum& cond, const Datum& left, const Datum& right,
                      ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month for each element of `values`.
+/// Month is encoded as January=1, December=12
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day number for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns number of the day of the week value for each element of
+/// `values`. Week starts on Monday denoted by 0 and ends on Sunday denoted by 6.
+///
+/// \param[in] values input to extract number of the day of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns number of day of the year for each element of `values`.
+/// January 1st maps to day number 1, February 1st to 32, etc.
+///
+/// \param[in] values input to extract number of day of the year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOYear returns ISO year number for each element of `values`.
+/// First week of an ISO year has the majority (4 or more) of it's days in January.
+///
+/// \param[in] values input to extract ISO year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> ISOYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOWeek returns ISO week of year number for each element of `values`.
+/// First ISO week has the majority (4 or more) of it's days in January.
+/// Week of the year starts with 1 and can run up to 53.
+///
+/// \param[in] values input to extract ISO week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> ISOWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOCalendar returns a (ISO year, ISO week, weekday) struct for each element of

Review comment:
       ```suggestion
   /// \brief ISOCalendar returns a (ISO year, ISO week, dayofweek) struct for each element of
   ```
   
   (both are kind of synonyms, but I think our kernel is called DayOfWeek, so best to be consistent?)

##########
File path: docs/source/cpp/compute.rst
##########
@@ -637,6 +637,55 @@ String extraction
   e.g. 'letter' and 'digit' for the regular expression
   ``(?P<letter>[ab])(?P<digit>\\d)``.
 
+Temporal component extraction
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+These functions extract datetime components (year, month, day, etc) from timestamp type.
+Note: timezone information is currently ignored if present.

Review comment:
       This can be updated to indicate that if there is timezone present, this is currently not yet supported

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -481,5 +481,184 @@ ARROW_EXPORT
 Result<Datum> IfElse(const Datum& cond, const Datum& left, const Datum& right,
                      ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month for each element of `values`.
+/// Month is encoded as January=1, December=12
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day number for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns number of the day of the week value for each element of
+/// `values`. Week starts on Monday denoted by 0 and ends on Sunday denoted by 6.
+///
+/// \param[in] values input to extract number of the day of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns number of day of the year for each element of `values`.
+/// January 1st maps to day number 1, February 1st to 32, etc.
+///
+/// \param[in] values input to extract number of day of the year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOYear returns ISO year number for each element of `values`.
+/// First week of an ISO year has the majority (4 or more) of it's days in January.
+///
+/// \param[in] values input to extract ISO year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> ISOYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOWeek returns ISO week of year number for each element of `values`.
+/// First ISO week has the majority (4 or more) of it's days in January.
+/// Week of the year starts with 1 and can run up to 53.
+///
+/// \param[in] values input to extract ISO week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> ISOWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief ISOCalendar returns a (ISO year, ISO week, weekday) struct for each element of
+/// `values`
+///
+/// \param[in] values input to ISO calendar struct from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> ISOCalendar(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns the quarter of year number for each element of `values`
+/// First quarter maps to 1 and forth quarter maps to 4.
+///
+/// \param[in] values input to extract quarter of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Quarter(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Hour returns hour value for each element of `values`
+///
+/// \param[in] values input to extract hour from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Hour(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Minute returns minutes value for each element of `values`
+///
+/// \param[in] values input to extract minutes from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Minute(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Second returns seconds value for each element of `values`
+///
+/// \param[in] values input to extract seconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Second(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Millisecond returns milliseconds value for each element of `values`
+///
+/// \param[in] values input to extract milliseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Millisecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Microsecond returns microseconds value for each element of `values`
+///
+/// \param[in] values input to extract microseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Microsecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Nanosecond returns nanoseconds value for each element of `values`
+///
+/// \param[in] values input to extract nanoseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Nanosecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Subsecond returns subsecond time fraction since last second for each element of

Review comment:
       ```suggestion
   /// \brief Subsecond returns subsecond time fraction since last second as a float for each element of
   ```
   
   (to be explicit, since it could also have been returned as an integer)

##########
File path: docs/source/cpp/compute.rst
##########
@@ -637,6 +637,54 @@ String extraction
   e.g. 'letter' and 'digit' for the regular expression
   ``(?P<letter>[ab])(?P<digit>\\d)``.
 
+Temporal component extraction
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+These functions extract datetime components (year, month, day, etc) from timestamp type.
+Note: timezone information is currently ignored if present.
+
++--------------------+------------+-------------------+-----------------+--------+
+| Function name      | Arity      | Input types       | Output type     | Notes  |
++====================+============+===================+=================+========+
+| year               | Unary      | Temporal          | Numeric         |        |

Review comment:
       (this isn't done yet? Or you didn't yet push your latest edits?)




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

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -450,5 +450,145 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Week returns week of year value for each element of `values`
+///
+/// \param[in] values input to extract week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Week(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns quarter of year value for each element of `values`
+///
+/// \param[in] values input to extract quarter of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Quarter(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Hour returns hour value for each element of `values`
+///
+/// \param[in] values input to extract hour from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Hour(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Minute returns minutes value for each element of `values`
+///
+/// \param[in] values input to extract minutes from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Minute(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Second returns seconds value for each element of `values`
+///
+/// \param[in] values input to extract seconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Second(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Millisecond returns milliseconds value for each element of `values`
+///
+/// \param[in] values input to extract milliseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Millisecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Microsecond returns microseconds value for each element of `values`
+///
+/// \param[in] values input to extract microseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Microsecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Nanosecond returns nanoseconds value for each element of `values`
+///
+/// \param[in] values input to extract nanoseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Nanosecond(const Datum& values, ExecContext* ctx = NULLPTR);

Review comment:
       Seeing the tests, it's clearly the first (additive components), which I think is good (and the most useful). I am only wondering if we can make that clearer in the terminology of the docstring




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

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -450,5 +450,145 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Week returns week of year value for each element of `values`
+///
+/// \param[in] values input to extract week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Week(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns quarter of year value for each element of `values`
+///
+/// \param[in] values input to extract quarter of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Quarter(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Hour returns hour value for each element of `values`
+///
+/// \param[in] values input to extract hour from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Hour(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Minute returns minutes value for each element of `values`
+///
+/// \param[in] values input to extract minutes from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Minute(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Second returns seconds value for each element of `values`
+///
+/// \param[in] values input to extract seconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Second(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Millisecond returns milliseconds value for each element of `values`
+///
+/// \param[in] values input to extract milliseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Millisecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Microsecond returns microseconds value for each element of `values`
+///
+/// \param[in] values input to extract microseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Microsecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Nanosecond returns nanoseconds value for each element of `values`
+///
+/// \param[in] values input to extract nanoseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Nanosecond(const Datum& values, ExecContext* ctx = NULLPTR);

Review comment:
       Are this the total nanoseconds, or only the nanoseconds part after the microseconds? (i.e. are microseconds and nanoseconds additive components, or rather both representing the full fractional part of the second using a different resolution?)




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

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



[GitHub] [arrow] rok edited a comment on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   @jorisvandenbossche - I found the cause of the unit issue - it was [ the missing`break`](https://github.com/apache/arrow/pull/10176/files#diff-53850537a2f3f7977473b30241fdd3ddb35aa97269284c06471d9b0c599e2c59R425)s in a switch statement. Typical. :)
   
   I think all comments were addressed now but more eyeballs == better code :).


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

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



[GitHub] [arrow] rok commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   Just noticed I missed this:
   > @jorisvandenbossche
   > Also, it would maybe be good to add a test for a timestamp that doesn't fit into the nanosecond range?
   
   Do you mind if I move it to [ARROW-12980](https://github.com/apache/arrow/pull/10457)?


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

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,348 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+        .iso_encoding();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return (sd - sys_days(year_month_day(sd).year() / jan / 0)).count();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct Week {

Review comment:
       A struct? (with year, week, day fields)




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

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



[GitHub] [arrow] pitrou closed pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   


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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,239 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestSimpleTemporalComponentExtraction) {
+  const char* json =
+      R"(["1970-01-01T00:00:59","2000-02-29T23:23:23",
+          "3989-07-14T18:04:01","1900-01-01T01:59:20","2033-05-18T03:33:20"])";
+  auto time_points = ArrayFromJSON(timestamp(TimeUnit::SECOND), json);
+
+  auto year = ArrayFromJSON(int64(), "[1970, 2000, 3989, 1900, 2033]");
+  auto month = ArrayFromJSON(int64(), "[1, 2, 7, 1, 5]");
+  auto day = ArrayFromJSON(int64(), "[1, 29, 14, 1, 18]");
+  auto day_of_year = ArrayFromJSON(int64(), "[1, 60, 195, 1, 138]");
+  auto week = ArrayFromJSON(int64(), "[1, 9, 28, 1, 20]");
+  auto quarter = ArrayFromJSON(int64(), "[1, 1, 3, 1, 2]");
+  auto day_of_week = ArrayFromJSON(int64(), "[4, 2, 5, 1, 3]");
+  auto hour = ArrayFromJSON(int64(), "[0, 23, 18, 1, 3]");
+  auto minute = ArrayFromJSON(int64(), "[0, 23, 4, 59, 33]");
+  auto second = ArrayFromJSON(int64(), "[59, 23, 1, 20, 20]");
+  auto millisecond = ArrayFromJSON(int64(), "[0, 0, 0, 0, 0]");
+  auto microsecond = ArrayFromJSON(int64(), "[0, 0, 0, 0, 0]");
+  auto nanosecond = ArrayFromJSON(int64(), "[0, 0, 0, 0, 0]");
+
+  ASSERT_OK_AND_ASSIGN(Datum actual_year, Year(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_month, Month(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_day, Day(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_day_of_year, DayOfYear(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_week, Week(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_quarter, Quarter(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_day_of_week, DayOfWeek(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_hour, Hour(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_minute, Minute(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_second, Second(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_millisecond, Millisecond(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_microsecond, Microsecond(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_nanosecond, Nanosecond(time_points));
+
+  ASSERT_EQ(actual_year, year);
+  ASSERT_EQ(actual_month, month);
+  ASSERT_EQ(actual_day, day);
+  ASSERT_EQ(actual_day_of_year, day_of_year);
+  ASSERT_EQ(actual_week, week);
+  ASSERT_EQ(actual_quarter, quarter);
+  ASSERT_EQ(actual_day_of_week, day_of_week);
+  ASSERT_EQ(actual_hour, hour);
+  ASSERT_EQ(actual_minute, minute);
+  ASSERT_EQ(actual_second, second);
+  ASSERT_EQ(actual_millisecond, millisecond);
+  ASSERT_EQ(actual_microsecond, microsecond);
+  ASSERT_EQ(actual_nanosecond, nanosecond);
+
+  CheckScalarUnary("year", time_points, year);
+  CheckScalarUnary("month", time_points, month);
+  CheckScalarUnary("day", time_points, day);
+  CheckScalarUnary("day_of_year", time_points, day_of_year);
+  CheckScalarUnary("week", time_points, week);
+  CheckScalarUnary("quarter", time_points, quarter);
+  CheckScalarUnary("day_of_week", time_points, day_of_week);
+  CheckScalarUnary("hour", time_points, hour);
+  CheckScalarUnary("minute", time_points, minute);
+  CheckScalarUnary("second", time_points, second);
+  CheckScalarUnary("millisecond", time_points, millisecond);
+  CheckScalarUnary("microsecond", time_points, microsecond);
+  CheckScalarUnary("nanosecond", time_points, nanosecond);
+}
+
+TEST(ScalarTemporalTest, TestTemporalComponentExtraction) {
+  const char* json_second = "[59, 951866603, -2208981640, 2000000000]";
+  const char* json_milli = "[59000, 951866603000, -2208981640000, 2000000000000]";
+  const char* json_micro =
+      "[59000000, 951866603000000, -2208981640000000, 2000000000000000]";
+  const char* json_nano =
+      "[59000000000, 951866603000000000, -2208981640000000000, 2000000000000000000]";
+
+  auto time_points_second = ArrayFromJSON(timestamp(TimeUnit::SECOND), json_second);
+  auto time_points_milli = ArrayFromJSON(timestamp(TimeUnit::MILLI), json_milli);
+  auto time_points_micro = ArrayFromJSON(timestamp(TimeUnit::MICRO), json_micro);
+  auto time_points_nano = ArrayFromJSON(timestamp(TimeUnit::NANO), json_nano);
+
+  auto year = ArrayFromJSON(int64(), "[1970, 2000, 1900, 2033]");
+  auto month = ArrayFromJSON(int64(), "[1, 2, 1, 5]");
+  auto day = ArrayFromJSON(int64(), "[1, 29, 1, 18]");
+  auto day_of_year = ArrayFromJSON(int64(), "[1, 60, 1, 138]");
+  auto week = ArrayFromJSON(int64(), "[1, 9, 1, 20]");
+  auto quarter = ArrayFromJSON(int64(), "[1, 1, 1, 2]");
+  auto day_of_week = ArrayFromJSON(int64(), "[4, 2, 1, 3]");
+  auto hour = ArrayFromJSON(int64(), "[0, 23, 1, 3]");
+  auto minute = ArrayFromJSON(int64(), "[0, 23, 59, 33]");
+  auto second = ArrayFromJSON(int64(), "[59, 23, 20, 20]");
+  auto millisecond = ArrayFromJSON(int64(), "[0, 0, 0, 0]");
+  auto microsecond = ArrayFromJSON(int64(), "[0, 0, 0, 0]");
+  auto nanosecond = ArrayFromJSON(int64(), "[0, 0, 0, 0]");
+
+  for (auto time_points :
+       {time_points_second, time_points_milli, time_points_micro, time_points_nano}) {
+    CheckScalarUnary("year", time_points, year);
+    CheckScalarUnary("month", time_points, month);
+    CheckScalarUnary("day", time_points, day);
+    CheckScalarUnary("day_of_year", time_points, day_of_year);
+    CheckScalarUnary("week", time_points, week);
+    CheckScalarUnary("quarter", time_points, quarter);
+    CheckScalarUnary("day_of_week", time_points, day_of_week);
+    CheckScalarUnary("hour", time_points, hour);
+    CheckScalarUnary("minute", time_points, minute);
+    CheckScalarUnary("second", time_points, second);
+    CheckScalarUnary("millisecond", time_points, millisecond);
+    CheckScalarUnary("microsecond", time_points, microsecond);
+    CheckScalarUnary("nanosecond", time_points, nanosecond);
+  }
+
+  std::string in = "[123, 999, 1, 31231000]";
+  auto out = ArrayFromJSON(int64(), "[123, 999, 1, 0]");
+
+  auto tp_milli = ArrayFromJSON(timestamp(TimeUnit::MILLI), in);
+  auto tp_milli_zoned = ArrayFromJSON(timestamp(TimeUnit::MILLI, "Etc/GMT+2"), in);
+  CheckScalarUnary("millisecond", tp_milli, out);
+  CheckScalarUnary("millisecond", tp_milli, out);
+
+  auto tp_micro = ArrayFromJSON(timestamp(TimeUnit::MICRO), in);
+  auto tp_micro_zoned = ArrayFromJSON(timestamp(TimeUnit::MICRO, "Etc/GMT+2"), in);
+  CheckScalarUnary("microsecond", tp_micro, out);
+  CheckScalarUnary("microsecond", tp_micro_zoned, out);
+
+  auto tp_nano = ArrayFromJSON(timestamp(TimeUnit::NANO), in);
+  auto tp_nano_zoned = ArrayFromJSON(timestamp(TimeUnit::NANO, "Etc/GMT+2"), in);
+  CheckScalarUnary("nanosecond", tp_nano, out);
+  CheckScalarUnary("nanosecond", tp_nano_zoned, out);
+}
+
+TEST(ScalarTemporalTest, TestSimpleZonedTemporalComponentExtraction) {
+  const char* json =
+      R"(["1970-01-01T00:00:59","2000-02-29T23:23:23",
+          "3989-07-14T18:04:01","1900-01-01T01:59:20","2033-05-18T03:33:20"])";
+  auto time_points = ArrayFromJSON(timestamp(TimeUnit::SECOND, "Etc/GMT+2"), json);
+
+  auto year = ArrayFromJSON(int64(), "[1969, 2000, 3989, 1899, 2033]");

Review comment:
       Yeah, I suppose this is taking `UTC` year instead of `Etc/GMT+2` year. Will fix.




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

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



[GitHub] [arrow] rok commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   Actually @jorisvandenbossche what do you mean by "doesn't fit into the nanosecond range"?
   Like ` "1970-01-01T00:00:59.1234567890123"?


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

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



[GitHub] [arrow] rok commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   @jorisvandenbossche - I found the cause of the unit issue - it was [ the missing`break`](https://github.com/apache/arrow/pull/10176/files#diff-53850537a2f3f7977473b30241fdd3ddb35aa97269284c06471d9b0c599e2c59R425)s in a switch statement. Typical. :)
   
   I think all comments were addressed now but more eyballs == better code :).


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

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



[GitHub] [arrow] jorisvandenbossche commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   To recap some spread inline comments (original discussion at https://github.com/apache/arrow/pull/10176#discussion_r633497393) about the sub-second / fractional components. 
   
   Assume the timestamp `2012-03-26 01:02:03.123456789`.
   
   Currently, I think the PR does:
   
   * second -> 3.123456789 (float)
   * subsecond -> 123456789 (int)
   
   Based on a naive reading of the docstrings and my biased expectations, I would have thought they return:
   
   * second -> 3 (int)
   * subsecond -> 0.123456789 (float)
   
   
   Now, in our previous discussion you proposed to implement `subsecond` as "total units since last second", so that indeed matches with what is in the PR now. But, that also means that the interpretation of this value critically depends on the unit of the timestamp type. While a float value would not have that problem.
   
   In addition, I personally think it would be useful to have some form of `second` that is just the integer the non-fractional component in " 01:02:03.123...".
   
   ---
   
   In addition, we also have the other components (for which eg Antoine questioned whether they are needed):
   
   * microsecond -> 123
   * millisecond -> 456
   * nanosecond -> 789


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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return static_cast<T>((sd - sys_days(year_month_day(sd).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day{sys_days{floor<days>(Duration{arg})} + days{3}}.year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto dp = sys_days{floor<days>(Duration{arg})};
+    auto y = year_month_day{dp + days{3}}.year();
+    auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (dp < start) {
+      --y;
+      start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(dp - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of quarter from timestamp

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.

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



[GitHub] [arrow] pitrou commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   Thank you for this PR @rok ! I believe this is ready for merging. I'll just wait for CI.


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

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



[GitHub] [arrow] pitrou commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,107 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestSimpleTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO);
+  auto timestamps = ArrayFromJSON(unit, times);
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("weekday", int64())});
+
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar = ArrayFromJSON(iso_calendar_type,
+                                    R"([{"iso_year": 1970, "iso_week": 1, "weekday": 4},
+                        {"iso_year": 2000, "iso_week": 9, "weekday": 2},
+                        {"iso_year": 1899, "iso_week": 52, "weekday": 7},
+                        {"iso_year": 2033, "iso_week": 20, "weekday": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59.123456789, 23.999999999, 20.001001001, 20.0, null]";
+  auto millisecond = "[123, 999, 1, 0, null]";
+  auto microsecond = "[456, 999, 1, 0, null]";
+  auto nanosecond = "[789, 999, 1, 0, null]";
+  auto subsecond = "[123456789, 999999999, 1001001, 0, null]";
+
+  CheckScalarUnary("year", unit, times, int64(), year);
+  CheckScalarUnary("month", unit, times, int64(), month);
+  CheckScalarUnary("day", unit, times, int64(), day);
+  CheckScalarUnary("day_of_week", unit, times, int64(), day_of_week);
+  CheckScalarUnary("day_of_year", unit, times, int64(), day_of_year);
+  CheckScalarUnary("iso_year", unit, times, int64(), iso_year);
+  CheckScalarUnary("iso_week", unit, times, int64(), iso_week);
+  CheckScalarUnary("iso_calendar", timestamps, iso_calendar);
+  CheckScalarUnary("quarter", unit, times, int64(), quarter);
+  CheckScalarUnary("hour", unit, times, int64(), hour);
+  CheckScalarUnary("minute", unit, times, int64(), minute);
+  CheckScalarUnary("second", unit, times, float64(), second);
+  CheckScalarUnary("millisecond", unit, times, int64(), millisecond);
+  CheckScalarUnary("microsecond", unit, times, int64(), microsecond);
+  CheckScalarUnary("nanosecond", unit, times, int64(), nanosecond);
+  CheckScalarUnary("subsecond", unit, times, int64(), subsecond);
+}
+
+TEST(ScalarTemporalTest, TestZonedTemporalComponentExtraction) {
+  std::string timezone = "Etc/UTC-2";
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO, timezone);
+  auto timestamps = ArrayFromJSON(unit, times);
+
+  ASSERT_RAISES(Invalid, Year(timestamps));
+  ASSERT_RAISES(Invalid, Month(timestamps));
+  ASSERT_RAISES(Invalid, Day(timestamps));
+  ASSERT_RAISES(Invalid, DayOfWeek(timestamps));
+  ASSERT_RAISES(Invalid, DayOfYear(timestamps));
+  ASSERT_RAISES(Invalid, ISOYear(timestamps));
+  ASSERT_RAISES(Invalid, ISOWeek(timestamps));
+  ASSERT_RAISES(Invalid, ISOCalendar(timestamps));
+  ASSERT_RAISES(Invalid, Quarter(timestamps));
+  ASSERT_RAISES(Invalid, Hour(timestamps));
+  ASSERT_RAISES(Invalid, Minute(timestamps));
+  ASSERT_RAISES(Invalid, Second(timestamps));
+  ASSERT_RAISES(Invalid, Millisecond(timestamps));
+  ASSERT_RAISES(Invalid, Microsecond(timestamps));
+  ASSERT_RAISES(Invalid, Nanosecond(timestamps));
+  ASSERT_RAISES(Invalid, Subsecond(timestamps));

Review comment:
       In any case, this is mostly a documentation issue. If the timestamp value is always in UTC, then "year" and friends will return the year expressed in the UTC timezone. I don't think there is any reason to forbid the operation, just document it properly.




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

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,107 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestSimpleTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO);
+  auto timestamps = ArrayFromJSON(unit, times);
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("weekday", int64())});
+
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar = ArrayFromJSON(iso_calendar_type,
+                                    R"([{"iso_year": 1970, "iso_week": 1, "weekday": 4},
+                        {"iso_year": 2000, "iso_week": 9, "weekday": 2},
+                        {"iso_year": 1899, "iso_week": 52, "weekday": 7},
+                        {"iso_year": 2033, "iso_week": 20, "weekday": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59.123456789, 23.999999999, 20.001001001, 20.0, null]";
+  auto millisecond = "[123, 999, 1, 0, null]";
+  auto microsecond = "[456, 999, 1, 0, null]";
+  auto nanosecond = "[789, 999, 1, 0, null]";
+  auto subsecond = "[123456789, 999999999, 1001001, 0, null]";
+
+  CheckScalarUnary("year", unit, times, int64(), year);
+  CheckScalarUnary("month", unit, times, int64(), month);
+  CheckScalarUnary("day", unit, times, int64(), day);
+  CheckScalarUnary("day_of_week", unit, times, int64(), day_of_week);
+  CheckScalarUnary("day_of_year", unit, times, int64(), day_of_year);
+  CheckScalarUnary("iso_year", unit, times, int64(), iso_year);
+  CheckScalarUnary("iso_week", unit, times, int64(), iso_week);
+  CheckScalarUnary("iso_calendar", timestamps, iso_calendar);
+  CheckScalarUnary("quarter", unit, times, int64(), quarter);
+  CheckScalarUnary("hour", unit, times, int64(), hour);
+  CheckScalarUnary("minute", unit, times, int64(), minute);
+  CheckScalarUnary("second", unit, times, float64(), second);
+  CheckScalarUnary("millisecond", unit, times, int64(), millisecond);
+  CheckScalarUnary("microsecond", unit, times, int64(), microsecond);
+  CheckScalarUnary("nanosecond", unit, times, int64(), nanosecond);
+  CheckScalarUnary("subsecond", unit, times, int64(), subsecond);
+}
+
+TEST(ScalarTemporalTest, TestZonedTemporalComponentExtraction) {
+  std::string timezone = "Etc/UTC-2";
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO, timezone);
+  auto timestamps = ArrayFromJSON(unit, times);
+
+  ASSERT_RAISES(Invalid, Year(timestamps));
+  ASSERT_RAISES(Invalid, Month(timestamps));
+  ASSERT_RAISES(Invalid, Day(timestamps));
+  ASSERT_RAISES(Invalid, DayOfWeek(timestamps));
+  ASSERT_RAISES(Invalid, DayOfYear(timestamps));
+  ASSERT_RAISES(Invalid, ISOYear(timestamps));
+  ASSERT_RAISES(Invalid, ISOWeek(timestamps));
+  ASSERT_RAISES(Invalid, ISOCalendar(timestamps));
+  ASSERT_RAISES(Invalid, Quarter(timestamps));
+  ASSERT_RAISES(Invalid, Hour(timestamps));
+  ASSERT_RAISES(Invalid, Minute(timestamps));
+  ASSERT_RAISES(Invalid, Second(timestamps));
+  ASSERT_RAISES(Invalid, Millisecond(timestamps));
+  ASSERT_RAISES(Invalid, Microsecond(timestamps));
+  ASSERT_RAISES(Invalid, Nanosecond(timestamps));
+  ASSERT_RAISES(Invalid, Subsecond(timestamps));

Review comment:
       Indeed, the storage model of timezone-aware data is different compared to stdlib python, we are always storing the UTC timestamp (and not the localized timestamp).
   
   > In any case, this is mostly a documentation issue. If the timestamp value is always in UTC, then "year" and friends will return the year expressed in the UTC timezone. I don't think there is any reason to forbid the operation, just document it properly.
   
   I personally disagree with this. If you have a timezone-aware timestamp, as a user I would expect to get the result in *local*time. If I want to have the UTC version, I can easily first convert to UTC (which is a no-op). 
   For "year" it most of the time doesn't change much (only the hours around New Year), but if I have a timestamp "2021-06-01 09:00:00+02:00" (CEST), and I ask for "hour", I expect to get 9 and not 7.




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

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



[GitHub] [arrow] rok closed pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   


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

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



[GitHub] [arrow] pitrou commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,107 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestSimpleTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO);
+  auto timestamps = ArrayFromJSON(unit, times);
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("weekday", int64())});
+
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar = ArrayFromJSON(iso_calendar_type,
+                                    R"([{"iso_year": 1970, "iso_week": 1, "weekday": 4},
+                        {"iso_year": 2000, "iso_week": 9, "weekday": 2},
+                        {"iso_year": 1899, "iso_week": 52, "weekday": 7},
+                        {"iso_year": 2033, "iso_week": 20, "weekday": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59.123456789, 23.999999999, 20.001001001, 20.0, null]";
+  auto millisecond = "[123, 999, 1, 0, null]";
+  auto microsecond = "[456, 999, 1, 0, null]";
+  auto nanosecond = "[789, 999, 1, 0, null]";
+  auto subsecond = "[123456789, 999999999, 1001001, 0, null]";
+
+  CheckScalarUnary("year", unit, times, int64(), year);
+  CheckScalarUnary("month", unit, times, int64(), month);
+  CheckScalarUnary("day", unit, times, int64(), day);
+  CheckScalarUnary("day_of_week", unit, times, int64(), day_of_week);
+  CheckScalarUnary("day_of_year", unit, times, int64(), day_of_year);
+  CheckScalarUnary("iso_year", unit, times, int64(), iso_year);
+  CheckScalarUnary("iso_week", unit, times, int64(), iso_week);
+  CheckScalarUnary("iso_calendar", timestamps, iso_calendar);
+  CheckScalarUnary("quarter", unit, times, int64(), quarter);
+  CheckScalarUnary("hour", unit, times, int64(), hour);
+  CheckScalarUnary("minute", unit, times, int64(), minute);
+  CheckScalarUnary("second", unit, times, float64(), second);
+  CheckScalarUnary("millisecond", unit, times, int64(), millisecond);
+  CheckScalarUnary("microsecond", unit, times, int64(), microsecond);
+  CheckScalarUnary("nanosecond", unit, times, int64(), nanosecond);
+  CheckScalarUnary("subsecond", unit, times, int64(), subsecond);
+}
+
+TEST(ScalarTemporalTest, TestZonedTemporalComponentExtraction) {
+  std::string timezone = "Etc/UTC-2";
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO, timezone);
+  auto timestamps = ArrayFromJSON(unit, times);
+
+  ASSERT_RAISES(Invalid, Year(timestamps));
+  ASSERT_RAISES(Invalid, Month(timestamps));
+  ASSERT_RAISES(Invalid, Day(timestamps));
+  ASSERT_RAISES(Invalid, DayOfWeek(timestamps));
+  ASSERT_RAISES(Invalid, DayOfYear(timestamps));
+  ASSERT_RAISES(Invalid, ISOYear(timestamps));
+  ASSERT_RAISES(Invalid, ISOWeek(timestamps));
+  ASSERT_RAISES(Invalid, ISOCalendar(timestamps));
+  ASSERT_RAISES(Invalid, Quarter(timestamps));
+  ASSERT_RAISES(Invalid, Hour(timestamps));
+  ASSERT_RAISES(Invalid, Minute(timestamps));
+  ASSERT_RAISES(Invalid, Second(timestamps));
+  ASSERT_RAISES(Invalid, Millisecond(timestamps));
+  ASSERT_RAISES(Invalid, Microsecond(timestamps));
+  ASSERT_RAISES(Invalid, Nanosecond(timestamps));
+  ASSERT_RAISES(Invalid, Subsecond(timestamps));

Review comment:
       Ah... It seems you're right indeed. 
   ```
             /// * If the time zone is set to a valid value, values can be displayed as
             ///   "localized" to that time zone, even though the underlying 64-bit
             ///   integers are identical to the same data stored in UTC. Converting
             ///   between time zones is a metadata-only operation and does not change the
             ///   underlying values
   ```
   
   I will ask on the ML for clarification.




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,151 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO);
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("day_of_week", int64())});
+
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar =
+      ArrayFromJSON(iso_calendar_type,
+                    R"([{"iso_year": 1970, "iso_week": 1, "day_of_week": 4},
+                        {"iso_year": 2000, "iso_week": 9, "day_of_week": 2},
+                        {"iso_year": 1899, "iso_week": 52, "day_of_week": 7},
+                        {"iso_year": 2033, "iso_week": 20, "day_of_week": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59.123456789, 23.999999999, 20.001001001, 20.0, null]";
+  auto millisecond = "[123, 999, 1, 0, null]";
+  auto microsecond = "[456, 999, 1, 0, null]";
+  auto nanosecond = "[789, 999, 1, 0, null]";
+  auto subsecond = "[123456789, 999999999, 1001001, 0, null]";
+
+  CheckScalarUnary("year", unit, times, int64(), year);
+  CheckScalarUnary("month", unit, times, int64(), month);
+  CheckScalarUnary("day", unit, times, int64(), day);
+  CheckScalarUnary("day_of_week", unit, times, int64(), day_of_week);
+  CheckScalarUnary("day_of_year", unit, times, int64(), day_of_year);
+  CheckScalarUnary("iso_year", unit, times, int64(), iso_year);
+  CheckScalarUnary("iso_week", unit, times, int64(), iso_week);
+  CheckScalarUnary("iso_calendar", ArrayFromJSON(unit, times), iso_calendar);
+  CheckScalarUnary("quarter", unit, times, int64(), quarter);
+  CheckScalarUnary("hour", unit, times, int64(), hour);
+  CheckScalarUnary("minute", unit, times, int64(), minute);
+  CheckScalarUnary("second", unit, times, float64(), second);
+  CheckScalarUnary("millisecond", unit, times, int64(), millisecond);
+  CheckScalarUnary("microsecond", unit, times, int64(), microsecond);
+  CheckScalarUnary("nanosecond", unit, times, int64(), nanosecond);
+  CheckScalarUnary("subsecond", unit, times, int64(), subsecond);
+}
+
+TEST(ScalarTemporalTest, TestTemporalComponentExtractionWithDifferentUnits) {
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("day_of_week", int64())});
+  const char* times =
+      R"(["1970-01-01T00:00:59","2000-02-29T23:23:23",
+            "1899-01-01T00:59:20","2033-05-18T03:33:20", null])";
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar =
+      ArrayFromJSON(iso_calendar_type,
+                    R"([{"iso_year": 1970, "iso_week": 1, "day_of_week": 4},
+                          {"iso_year": 2000, "iso_week": 9, "day_of_week": 2},
+                          {"iso_year": 1899, "iso_week": 52, "day_of_week": 7},
+                          {"iso_year": 2033, "iso_week": 20, "day_of_week": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59, 23, 20, 20, null]";

Review comment:
       Sure, I'll add a block like:
   ```
       CheckScalarUnary("millisecond", unit, times, int64(), zeros);
       CheckScalarUnary("microsecond", unit, times, int64(), zeros);
       CheckScalarUnary("nanosecond", unit, times, int64(), zeros);
       CheckScalarUnary("subsecond", unit, times, float64(), zeros);
   ```
   
   Where:
   ```
   auto zeros = "[0, 0, 0, 0, null]";
   ```




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

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



[GitHub] [arrow] rok commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   > There seems to be something wrong with the ISO calendar (year) implementation.
   
   Will take a look. Still need to add test cases too.
   
   > Also in general, for non-nanosecond resolutions, you get wrong results for all kernels. It seems that it is interpreting the non-nanosecond epoch as nanosecond epoch. That's related to your previous comment that tests for other units are still failing?
   
   Yeah. It seems that currently all timestamps are interpreted as if having nanosecond unit. I suspect it's a kernel matching problem.


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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,151 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO);
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("day_of_week", int64())});
+
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar =
+      ArrayFromJSON(iso_calendar_type,
+                    R"([{"iso_year": 1970, "iso_week": 1, "day_of_week": 4},
+                        {"iso_year": 2000, "iso_week": 9, "day_of_week": 2},
+                        {"iso_year": 1899, "iso_week": 52, "day_of_week": 7},
+                        {"iso_year": 2033, "iso_week": 20, "day_of_week": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59.123456789, 23.999999999, 20.001001001, 20.0, null]";
+  auto millisecond = "[123, 999, 1, 0, null]";
+  auto microsecond = "[456, 999, 1, 0, null]";
+  auto nanosecond = "[789, 999, 1, 0, null]";
+  auto subsecond = "[123456789, 999999999, 1001001, 0, null]";
+
+  CheckScalarUnary("year", unit, times, int64(), year);
+  CheckScalarUnary("month", unit, times, int64(), month);
+  CheckScalarUnary("day", unit, times, int64(), day);
+  CheckScalarUnary("day_of_week", unit, times, int64(), day_of_week);
+  CheckScalarUnary("day_of_year", unit, times, int64(), day_of_year);
+  CheckScalarUnary("iso_year", unit, times, int64(), iso_year);
+  CheckScalarUnary("iso_week", unit, times, int64(), iso_week);
+  CheckScalarUnary("iso_calendar", ArrayFromJSON(unit, times), iso_calendar);
+  CheckScalarUnary("quarter", unit, times, int64(), quarter);
+  CheckScalarUnary("hour", unit, times, int64(), hour);
+  CheckScalarUnary("minute", unit, times, int64(), minute);
+  CheckScalarUnary("second", unit, times, float64(), second);
+  CheckScalarUnary("millisecond", unit, times, int64(), millisecond);
+  CheckScalarUnary("microsecond", unit, times, int64(), microsecond);
+  CheckScalarUnary("nanosecond", unit, times, int64(), nanosecond);
+  CheckScalarUnary("subsecond", unit, times, int64(), subsecond);
+}
+
+TEST(ScalarTemporalTest, TestTemporalComponentExtractionWithDifferentUnits) {
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("day_of_week", int64())});
+  const char* times =
+      R"(["1970-01-01T00:00:59","2000-02-29T23:23:23",
+            "1899-01-01T00:59:20","2033-05-18T03:33:20", null])";
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar =
+      ArrayFromJSON(iso_calendar_type,
+                    R"([{"iso_year": 1970, "iso_week": 1, "day_of_week": 4},
+                          {"iso_year": 2000, "iso_week": 9, "day_of_week": 2},
+                          {"iso_year": 1899, "iso_week": 52, "day_of_week": 7},
+                          {"iso_year": 2033, "iso_week": 20, "day_of_week": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59, 23, 20, 20, null]";
+
+  //  for (auto u : internal::AllTimeUnits()) {
+  for (auto u : {TimeUnit::NANO}) {

Review comment:
       As stated above:
   > This test fails for every unit except for nanosecond even though we [register kernels for all units](https://github.com/apache/arrow/pull/10176/commits/efd6b9e746dadaaaa6d1cc69614e6145b30b8037#diff-53850537a2f3f7977473b30241fdd3ddb35aa97269284c06471d9b0c599e2c59R415).
   Is this expected? We can get timestamp units at runtime but matching would be preferable I think.




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return static_cast<T>((sd - sys_days(year_month_day(sd).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day{sys_days{floor<days>(Duration{arg})} + days{3}}.year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto dp = sys_days{floor<days>(Duration{arg})};
+    auto y = year_month_day{dp + days{3}}.year();
+    auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (dp < start) {
+      --y;
+      start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(dp - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        std::chrono::duration<double>(t - floor<std::chrono::minutes>(t)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::seconds>(t)) /
+                          std::chrono::nanoseconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+template <typename Duration, typename OutType>
+struct ISOCalendar {
+  using T = typename OutType::c_type;
+
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    using ScalarType = typename TypeTraits<OutType>::ScalarType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    if (!in.is_valid) {
+      out->is_valid = false;
+    } else {
+      const std::shared_ptr<DataType> iso_calendar_type =
+          struct_({field("iso_year", out_type), field("iso_week", out_type),
+                   field("weekday", out_type)});
+
+      const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+      const auto dp = sys_days{floor<days>(Duration{in_val})};
+      const auto ymd = year_month_day(dp);
+      auto y = year_month_day{dp + days{3}}.year();
+      auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      if (dp < start) {
+        --y;
+        start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      }
+
+      std::vector<std::shared_ptr<Scalar>> values = {
+          std::make_shared<ScalarType>(static_cast<T>(static_cast<int32_t>(ymd.year()))),
+          std::make_shared<ScalarType>(
+              static_cast<T>(trunc<weeks>(dp - start).count() + 1)),
+          std::make_shared<ScalarType>(static_cast<T>(weekday(ymd).iso_encoding()))};
+      *checked_cast<StructScalar*>(out) = StructScalar(values, iso_calendar_type);
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    using BuilderType = typename TypeTraits<OutType>::BuilderType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    const std::shared_ptr<DataType> iso_calendar_type =
+        struct_({field("iso_year", out_type), field("iso_week", out_type),
+                 field("weekday", out_type)});
+
+    std::unique_ptr<ArrayBuilder> array_builder;
+    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), iso_calendar_type, &array_builder));
+    StructBuilder* struct_builder = checked_cast<StructBuilder*>(array_builder.get());

Review comment:
       Done.

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return static_cast<T>((sd - sys_days(year_month_day(sd).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day{sys_days{floor<days>(Duration{arg})} + days{3}}.year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto dp = sys_days{floor<days>(Duration{arg})};
+    auto y = year_month_day{dp + days{3}}.year();
+    auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (dp < start) {
+      --y;
+      start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(dp - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        std::chrono::duration<double>(t - floor<std::chrono::minutes>(t)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::seconds>(t)) /
+                          std::chrono::nanoseconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+template <typename Duration, typename OutType>
+struct ISOCalendar {
+  using T = typename OutType::c_type;
+
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    using ScalarType = typename TypeTraits<OutType>::ScalarType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    if (!in.is_valid) {
+      out->is_valid = false;
+    } else {
+      const std::shared_ptr<DataType> iso_calendar_type =
+          struct_({field("iso_year", out_type), field("iso_week", out_type),
+                   field("weekday", out_type)});
+
+      const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+      const auto dp = sys_days{floor<days>(Duration{in_val})};
+      const auto ymd = year_month_day(dp);
+      auto y = year_month_day{dp + days{3}}.year();
+      auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      if (dp < start) {
+        --y;
+        start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      }
+
+      std::vector<std::shared_ptr<Scalar>> values = {
+          std::make_shared<ScalarType>(static_cast<T>(static_cast<int32_t>(ymd.year()))),
+          std::make_shared<ScalarType>(
+              static_cast<T>(trunc<weeks>(dp - start).count() + 1)),
+          std::make_shared<ScalarType>(static_cast<T>(weekday(ymd).iso_encoding()))};
+      *checked_cast<StructScalar*>(out) = StructScalar(values, iso_calendar_type);
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    using BuilderType = typename TypeTraits<OutType>::BuilderType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    const std::shared_ptr<DataType> iso_calendar_type =
+        struct_({field("iso_year", out_type), field("iso_week", out_type),
+                 field("weekday", out_type)});
+
+    std::unique_ptr<ArrayBuilder> array_builder;
+    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), iso_calendar_type, &array_builder));
+    StructBuilder* struct_builder = checked_cast<StructBuilder*>(array_builder.get());
+
+    std::vector<BuilderType*> field_builders;
+    field_builders.reserve(3);
+    for (int i = 0; i < 3; i++) {
+      field_builders.push_back(
+          checked_cast<BuilderType*>(struct_builder->field_builder(i)));

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.

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



[GitHub] [arrow] rok commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   > I did a quick naive check of the performance of this branch, and comparing the simple components, they are consistently 6 to 25x faster compared to the pandas implementation (6x for the year/month/day, 25-30x for hour/minute/second). That aspect seems to be good!
   > (I don't know how good the pandas implementation is, so it might not be the most relevant comparison (comparing with eg clickhouse might be more interesting), but it at least says something)
   
   Nice! Thanks for measuring that. That's a dramatic improvement!
   Maybe we could still optimize somewhat but that would require a bit of a study of `date.h` and benchmarking as changes are tried.
   
   I've fixed the `iso_year` function and added some tests [borrowed from pandas](https://github.com/pandas-dev/pandas/blob/059c8bac51e47d6eaaa3e36d6a293a22312925e6/pandas/tests/tslibs/test_ccalendar.py#L39).
   
   
   I think the last thing left is the matching to proper time unit.


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

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -450,5 +450,145 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Week returns week of year value for each element of `values`
+///
+/// \param[in] values input to extract week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Week(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns quarter of year value for each element of `values`
+///
+/// \param[in] values input to extract quarter of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Quarter(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Hour returns hour value for each element of `values`
+///
+/// \param[in] values input to extract hour from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Hour(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Minute returns minutes value for each element of `values`
+///
+/// \param[in] values input to extract minutes from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Minute(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Second returns seconds value for each element of `values`
+///
+/// \param[in] values input to extract seconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Second(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Millisecond returns milliseconds value for each element of `values`
+///
+/// \param[in] values input to extract milliseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Millisecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Microsecond returns microseconds value for each element of `values`
+///
+/// \param[in] values input to extract microseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Microsecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Nanosecond returns nanoseconds value for each element of `values`
+///
+/// \param[in] values input to extract nanoseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Nanosecond(const Datum& values, ExecContext* ctx = NULLPTR);

Review comment:
       Seeing the tests, it's clearly the first (additive components), which I think is good. I am only wondering if we can make that clearer in the terminology of the docstring




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return static_cast<T>((sd - sys_days(year_month_day(sd).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day{sys_days{floor<days>(Duration{arg})} + days{3}}.year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto dp = sys_days{floor<days>(Duration{arg})};
+    auto y = year_month_day{dp + days{3}}.year();
+    auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (dp < start) {
+      --y;
+      start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(dp - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        std::chrono::duration<double>(t - floor<std::chrono::minutes>(t)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::seconds>(t)) /
+                          std::chrono::nanoseconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+template <typename Duration, typename OutType>
+struct ISOCalendar {
+  using T = typename OutType::c_type;
+
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    using ScalarType = typename TypeTraits<OutType>::ScalarType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    if (!in.is_valid) {
+      out->is_valid = false;
+    } else {
+      const std::shared_ptr<DataType> iso_calendar_type =
+          struct_({field("iso_year", out_type), field("iso_week", out_type),
+                   field("weekday", out_type)});
+
+      const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+      const auto dp = sys_days{floor<days>(Duration{in_val})};
+      const auto ymd = year_month_day(dp);
+      auto y = year_month_day{dp + days{3}}.year();
+      auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      if (dp < start) {
+        --y;
+        start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      }
+
+      std::vector<std::shared_ptr<Scalar>> values = {
+          std::make_shared<ScalarType>(static_cast<T>(static_cast<int32_t>(ymd.year()))),
+          std::make_shared<ScalarType>(
+              static_cast<T>(trunc<weeks>(dp - start).count() + 1)),
+          std::make_shared<ScalarType>(static_cast<T>(weekday(ymd).iso_encoding()))};
+      *checked_cast<StructScalar*>(out) = StructScalar(values, iso_calendar_type);
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    using BuilderType = typename TypeTraits<OutType>::BuilderType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    const std::shared_ptr<DataType> iso_calendar_type =
+        struct_({field("iso_year", out_type), field("iso_week", out_type),
+                 field("weekday", out_type)});
+
+    std::unique_ptr<ArrayBuilder> array_builder;
+    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), iso_calendar_type, &array_builder));
+    StructBuilder* struct_builder = checked_cast<StructBuilder*>(array_builder.get());
+
+    std::vector<BuilderType*> field_builders;
+    field_builders.reserve(3);
+    for (int i = 0; i < 3; i++) {
+      field_builders.push_back(
+          checked_cast<BuilderType*>(struct_builder->field_builder(i)));
+    }
+    auto visit_null = [&]() { return struct_builder->AppendNull(); };
+    auto visit_value = [&](int64_t arg) {
+      const auto dp = sys_days{floor<days>(Duration{arg})};
+      const auto ymd = year_month_day(dp);
+      auto y = year_month_day{dp + days{3}}.year();
+      auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      if (dp < start) {
+        --y;
+        start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      }
+
+      RETURN_NOT_OK(
+          field_builders[0]->Append(static_cast<T>(static_cast<int32_t>(ymd.year()))));
+      RETURN_NOT_OK(field_builders[1]->Append(
+          static_cast<T>(trunc<weeks>(dp - start).count() + 1)));
+      RETURN_NOT_OK(
+          field_builders[2]->Append(static_cast<T>(weekday(ymd).iso_encoding())));
+
+      return struct_builder->Append();
+    };
+    RETURN_NOT_OK(VisitArrayDataInline<OutType>(in, visit_value, visit_null));
+
+    std::shared_ptr<Array> out_array;
+    RETURN_NOT_OK(struct_builder->Finish(&out_array));
+    *out = *std::move(out_array->data());
+
+    return Status::OK();
+  }
+};
+
+// Generate a kernel given an arithmetic functor
+template <template <typename... Args> class KernelGenerator,
+          template <typename... Args> class Op, typename Duration>
+ArrayKernelExec ExecFromOp(detail::GetTypeId get_id) {
+  switch (get_id.id) {

Review comment:
       Inputs are timestamps but I'd like to control type of output (as proposed [here](https://github.com/apache/arrow/pull/10176#discussion_r629457595)). I'm not sure output type of a kernel can be matched?




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

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



[GitHub] [arrow] jorisvandenbossche commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   Cool, this is a nice start for datetime kernels! 
   @thisisnic opened a JIRA for expsosing those in R as well (https://issues.apache.org/jira/browse/ARROW-13022)
   
   > what do you mean by "doesn't fit into the nanosecond range"?
   
   I meant a date that falls outside the *date* range of 1677-09-21 - 2262-04-11 (the range that ns resolution can cover).


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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,151 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO);
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("day_of_week", int64())});
+
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar =
+      ArrayFromJSON(iso_calendar_type,
+                    R"([{"iso_year": 1970, "iso_week": 1, "day_of_week": 4},
+                        {"iso_year": 2000, "iso_week": 9, "day_of_week": 2},
+                        {"iso_year": 1899, "iso_week": 52, "day_of_week": 7},
+                        {"iso_year": 2033, "iso_week": 20, "day_of_week": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59.123456789, 23.999999999, 20.001001001, 20.0, null]";

Review comment:
       Switching to integer.




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

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



[GitHub] [arrow] pitrou commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: docs/source/cpp/compute.rst
##########
@@ -665,6 +665,57 @@ String component extraction
   e.g. 'letter' and 'digit' for the regular expression
   ``(?P<letter>[ab])(?P<digit>\\d)``.
 
+Temporal component extraction

Review comment:
       Nit: can you move these below other string-related sections? 




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -450,5 +450,145 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Week returns week of year value for each element of `values`
+///
+/// \param[in] values input to extract week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Week(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns quarter of year value for each element of `values`
+///
+/// \param[in] values input to extract quarter of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Quarter(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Hour returns hour value for each element of `values`
+///
+/// \param[in] values input to extract hour from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Hour(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Minute returns minutes value for each element of `values`
+///
+/// \param[in] values input to extract minutes from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Minute(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Second returns seconds value for each element of `values`
+///
+/// \param[in] values input to extract seconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Second(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Millisecond returns milliseconds value for each element of `values`
+///
+/// \param[in] values input to extract milliseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Millisecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Microsecond returns microseconds value for each element of `values`
+///
+/// \param[in] values input to extract microseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Microsecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Nanosecond returns nanoseconds value for each element of `values`
+///
+/// \param[in] values input to extract nanoseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Nanosecond(const Datum& values, ExecContext* ctx = NULLPTR);

Review comment:
       Well this is not ideal :).
   I propose to implement this as `total units since midnight` and then add an exception in python for nanoseconds.




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,509 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return static_cast<T>((sd - sys_days(year_month_day(sd).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day{sys_days{floor<days>(Duration{arg})} + days{3}}.year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    const auto dp = sys_days{floor<days>(Duration{arg})};
+    auto y = year_month_day{dp + days{3}}.year();
+    auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (dp < start) {
+      --y;
+      start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(dp - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+const std::shared_ptr<DataType> iso_calendar_type = struct_(
+    {field("iso_year", int64()), field("iso_week", int64()), field("weekday", int64())});
+
+template <typename Duration>
+struct ISOCalendar {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    using ScalarType = typename TypeTraits<Int64Type>::ScalarType;
+
+    const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+    const auto dp = sys_days{floor<days>(Duration{in_val})};
+    const auto ymd = year_month_day(dp);
+    auto y = year_month_day{dp + days{3}}.year();
+    auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (dp < start) {
+      --y;
+      start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+
+    std::vector<std::shared_ptr<Scalar>> values = {
+        std::make_shared<ScalarType>(
+            static_cast<int64_t>(static_cast<int32_t>(ymd.year()))),
+        std::make_shared<ScalarType>(
+            static_cast<int64_t>(trunc<weeks>(dp - start).count() + 1)),
+        std::make_shared<ScalarType>(static_cast<int64_t>(weekday(ymd).iso_encoding()))};
+    *checked_cast<StructScalar*>(out) =
+        StructScalar(std::move(values), iso_calendar_type);
+    return Status::OK();
+  }

Review comment:
       @pitrou any idea why this scalar kernel is failing here? The test outputs and [CI log](https://github.com/apache/arrow/pull/10176/checks?check_run_id=2666780178#step:10:1299) is not very telling.




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,239 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestSimpleTemporalComponentExtraction) {
+  const char* json =
+      R"(["1970-01-01T00:00:59","2000-02-29T23:23:23",
+          "3989-07-14T18:04:01","1900-01-01T01:59:20","2033-05-18T03:33:20"])";
+  auto time_points = ArrayFromJSON(timestamp(TimeUnit::SECOND), json);
+
+  auto year = ArrayFromJSON(int64(), "[1970, 2000, 3989, 1900, 2033]");
+  auto month = ArrayFromJSON(int64(), "[1, 2, 7, 1, 5]");
+  auto day = ArrayFromJSON(int64(), "[1, 29, 14, 1, 18]");
+  auto day_of_year = ArrayFromJSON(int64(), "[1, 60, 195, 1, 138]");
+  auto week = ArrayFromJSON(int64(), "[1, 9, 28, 1, 20]");
+  auto quarter = ArrayFromJSON(int64(), "[1, 1, 3, 1, 2]");
+  auto day_of_week = ArrayFromJSON(int64(), "[4, 2, 5, 1, 3]");
+  auto hour = ArrayFromJSON(int64(), "[0, 23, 18, 1, 3]");
+  auto minute = ArrayFromJSON(int64(), "[0, 23, 4, 59, 33]");
+  auto second = ArrayFromJSON(int64(), "[59, 23, 1, 20, 20]");
+  auto millisecond = ArrayFromJSON(int64(), "[0, 0, 0, 0, 0]");
+  auto microsecond = ArrayFromJSON(int64(), "[0, 0, 0, 0, 0]");
+  auto nanosecond = ArrayFromJSON(int64(), "[0, 0, 0, 0, 0]");
+
+  ASSERT_OK_AND_ASSIGN(Datum actual_year, Year(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_month, Month(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_day, Day(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_day_of_year, DayOfYear(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_week, Week(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_quarter, Quarter(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_day_of_week, DayOfWeek(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_hour, Hour(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_minute, Minute(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_second, Second(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_millisecond, Millisecond(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_microsecond, Microsecond(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_nanosecond, Nanosecond(time_points));
+
+  ASSERT_EQ(actual_year, year);
+  ASSERT_EQ(actual_month, month);
+  ASSERT_EQ(actual_day, day);
+  ASSERT_EQ(actual_day_of_year, day_of_year);
+  ASSERT_EQ(actual_week, week);
+  ASSERT_EQ(actual_quarter, quarter);
+  ASSERT_EQ(actual_day_of_week, day_of_week);
+  ASSERT_EQ(actual_hour, hour);
+  ASSERT_EQ(actual_minute, minute);
+  ASSERT_EQ(actual_second, second);
+  ASSERT_EQ(actual_millisecond, millisecond);
+  ASSERT_EQ(actual_microsecond, microsecond);
+  ASSERT_EQ(actual_nanosecond, nanosecond);
+
+  CheckScalarUnary("year", time_points, year);
+  CheckScalarUnary("month", time_points, month);
+  CheckScalarUnary("day", time_points, day);
+  CheckScalarUnary("day_of_year", time_points, day_of_year);
+  CheckScalarUnary("week", time_points, week);
+  CheckScalarUnary("quarter", time_points, quarter);
+  CheckScalarUnary("day_of_week", time_points, day_of_week);
+  CheckScalarUnary("hour", time_points, hour);
+  CheckScalarUnary("minute", time_points, minute);
+  CheckScalarUnary("second", time_points, second);
+  CheckScalarUnary("millisecond", time_points, millisecond);
+  CheckScalarUnary("microsecond", time_points, microsecond);
+  CheckScalarUnary("nanosecond", time_points, nanosecond);
+}
+
+TEST(ScalarTemporalTest, TestTemporalComponentExtraction) {
+  const char* json_second = "[59, 951866603, -2208981640, 2000000000]";
+  const char* json_milli = "[59000, 951866603000, -2208981640000, 2000000000000]";
+  const char* json_micro =
+      "[59000000, 951866603000000, -2208981640000000, 2000000000000000]";
+  const char* json_nano =
+      "[59000000000, 951866603000000000, -2208981640000000000, 2000000000000000000]";
+
+  auto time_points_second = ArrayFromJSON(timestamp(TimeUnit::SECOND), json_second);
+  auto time_points_milli = ArrayFromJSON(timestamp(TimeUnit::MILLI), json_milli);
+  auto time_points_micro = ArrayFromJSON(timestamp(TimeUnit::MICRO), json_micro);
+  auto time_points_nano = ArrayFromJSON(timestamp(TimeUnit::NANO), json_nano);
+
+  auto year = ArrayFromJSON(int64(), "[1970, 2000, 1900, 2033]");
+  auto month = ArrayFromJSON(int64(), "[1, 2, 1, 5]");
+  auto day = ArrayFromJSON(int64(), "[1, 29, 1, 18]");
+  auto day_of_year = ArrayFromJSON(int64(), "[1, 60, 1, 138]");
+  auto week = ArrayFromJSON(int64(), "[1, 9, 1, 20]");
+  auto quarter = ArrayFromJSON(int64(), "[1, 1, 1, 2]");
+  auto day_of_week = ArrayFromJSON(int64(), "[4, 2, 1, 3]");
+  auto hour = ArrayFromJSON(int64(), "[0, 23, 1, 3]");
+  auto minute = ArrayFromJSON(int64(), "[0, 23, 59, 33]");
+  auto second = ArrayFromJSON(int64(), "[59, 23, 20, 20]");
+  auto millisecond = ArrayFromJSON(int64(), "[0, 0, 0, 0]");
+  auto microsecond = ArrayFromJSON(int64(), "[0, 0, 0, 0]");
+  auto nanosecond = ArrayFromJSON(int64(), "[0, 0, 0, 0]");
+
+  for (auto time_points :
+       {time_points_second, time_points_milli, time_points_micro, time_points_nano}) {
+    CheckScalarUnary("year", time_points, year);
+    CheckScalarUnary("month", time_points, month);
+    CheckScalarUnary("day", time_points, day);
+    CheckScalarUnary("day_of_year", time_points, day_of_year);
+    CheckScalarUnary("week", time_points, week);
+    CheckScalarUnary("quarter", time_points, quarter);
+    CheckScalarUnary("day_of_week", time_points, day_of_week);
+    CheckScalarUnary("hour", time_points, hour);
+    CheckScalarUnary("minute", time_points, minute);
+    CheckScalarUnary("second", time_points, second);
+    CheckScalarUnary("millisecond", time_points, millisecond);
+    CheckScalarUnary("microsecond", time_points, microsecond);
+    CheckScalarUnary("nanosecond", time_points, nanosecond);
+  }
+
+  std::string in = "[123, 999, 1, 31231000]";
+  auto out = ArrayFromJSON(int64(), "[123, 999, 1, 0]");
+
+  auto tp_milli = ArrayFromJSON(timestamp(TimeUnit::MILLI), in);
+  auto tp_milli_zoned = ArrayFromJSON(timestamp(TimeUnit::MILLI, "Etc/GMT+2"), in);
+  CheckScalarUnary("millisecond", tp_milli, out);
+  CheckScalarUnary("millisecond", tp_milli, out);
+
+  auto tp_micro = ArrayFromJSON(timestamp(TimeUnit::MICRO), in);
+  auto tp_micro_zoned = ArrayFromJSON(timestamp(TimeUnit::MICRO, "Etc/GMT+2"), in);
+  CheckScalarUnary("microsecond", tp_micro, out);
+  CheckScalarUnary("microsecond", tp_micro_zoned, out);
+
+  auto tp_nano = ArrayFromJSON(timestamp(TimeUnit::NANO), in);
+  auto tp_nano_zoned = ArrayFromJSON(timestamp(TimeUnit::NANO, "Etc/GMT+2"), in);
+  CheckScalarUnary("nanosecond", tp_nano, out);
+  CheckScalarUnary("nanosecond", tp_nano_zoned, out);
+}
+
+TEST(ScalarTemporalTest, TestSimpleZonedTemporalComponentExtraction) {
+  const char* json =
+      R"(["1970-01-01T00:00:59","2000-02-29T23:23:23",
+          "3989-07-14T18:04:01","1900-01-01T01:59:20","2033-05-18T03:33:20"])";
+  auto time_points = ArrayFromJSON(timestamp(TimeUnit::SECOND, "Etc/GMT+2"), json);
+
+  auto year = ArrayFromJSON(int64(), "[1969, 2000, 3989, 1899, 2033]");

Review comment:
       It turns out `Etc/GMT+X == UTC-X` so this was actually ok. Weird mapping.

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,348 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+        .iso_encoding();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return (sd - sys_days(year_month_day(sd).year() / jan / 0)).count();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct Week {

Review comment:
       I'll do that.
   By the way what would be a good data type for "isocalendar" output? It's a [tuple in Pandas](https://pandas.pydata.org/docs/reference/api/pandas.Timestamp.isocalendar.html).

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -450,5 +450,145 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Week returns week of year value for each element of `values`
+///
+/// \param[in] values input to extract week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Week(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns quarter of year value for each element of `values`
+///
+/// \param[in] values input to extract quarter of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Quarter(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Hour returns hour value for each element of `values`
+///
+/// \param[in] values input to extract hour from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Hour(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Minute returns minutes value for each element of `values`
+///
+/// \param[in] values input to extract minutes from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Minute(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Second returns seconds value for each element of `values`
+///
+/// \param[in] values input to extract seconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Second(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Millisecond returns milliseconds value for each element of `values`
+///
+/// \param[in] values input to extract milliseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Millisecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Microsecond returns microseconds value for each element of `values`
+///
+/// \param[in] values input to extract microseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Microsecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Nanosecond returns nanoseconds value for each element of `values`
+///
+/// \param[in] values input to extract nanoseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Nanosecond(const Datum& values, ExecContext* ctx = NULLPTR);

Review comment:
       Oh yeah indeed. This would behave in the latter way which would be surprising to users. Will fix.

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -450,5 +450,145 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Week returns week of year value for each element of `values`
+///
+/// \param[in] values input to extract week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Week(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns quarter of year value for each element of `values`
+///
+/// \param[in] values input to extract quarter of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Quarter(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Hour returns hour value for each element of `values`
+///
+/// \param[in] values input to extract hour from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Hour(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Minute returns minutes value for each element of `values`
+///
+/// \param[in] values input to extract minutes from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Minute(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Second returns seconds value for each element of `values`
+///
+/// \param[in] values input to extract seconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Second(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Millisecond returns milliseconds value for each element of `values`
+///
+/// \param[in] values input to extract milliseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Millisecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Microsecond returns microseconds value for each element of `values`
+///
+/// \param[in] values input to extract microseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Microsecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Nanosecond returns nanoseconds value for each element of `values`
+///
+/// \param[in] values input to extract nanoseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Nanosecond(const Datum& values, ExecContext* ctx = NULLPTR);

Review comment:
       Do you know by hear what does pandas do? [Documentation](https://pandas.pydata.org/docs/reference/api/pandas.Series.dt.microsecond.html) is not clear.
   Perhaps it would be good to then have a `subseconds` function that would return the total nanoseconds since midnight?

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -450,5 +450,145 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Week returns week of year value for each element of `values`
+///
+/// \param[in] values input to extract week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Week(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns quarter of year value for each element of `values`
+///
+/// \param[in] values input to extract quarter of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Quarter(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Hour returns hour value for each element of `values`
+///
+/// \param[in] values input to extract hour from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Hour(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Minute returns minutes value for each element of `values`
+///
+/// \param[in] values input to extract minutes from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Minute(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Second returns seconds value for each element of `values`
+///
+/// \param[in] values input to extract seconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Second(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Millisecond returns milliseconds value for each element of `values`
+///
+/// \param[in] values input to extract milliseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Millisecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Microsecond returns microseconds value for each element of `values`
+///
+/// \param[in] values input to extract microseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Microsecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Nanosecond returns nanoseconds value for each element of `values`
+///
+/// \param[in] values input to extract nanoseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Nanosecond(const Datum& values, ExecContext* ctx = NULLPTR);

Review comment:
       Do you know by heart what does pandas do? [Documentation](https://pandas.pydata.org/docs/reference/api/pandas.Series.dt.microsecond.html) is not clear.
   Perhaps it would be good to then have a `subseconds` function that would return the total nanoseconds since midnight?

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -450,5 +450,145 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Week returns week of year value for each element of `values`
+///
+/// \param[in] values input to extract week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Week(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns quarter of year value for each element of `values`
+///
+/// \param[in] values input to extract quarter of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Quarter(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Hour returns hour value for each element of `values`
+///
+/// \param[in] values input to extract hour from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Hour(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Minute returns minutes value for each element of `values`
+///
+/// \param[in] values input to extract minutes from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Minute(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Second returns seconds value for each element of `values`
+///
+/// \param[in] values input to extract seconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Second(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Millisecond returns milliseconds value for each element of `values`
+///
+/// \param[in] values input to extract milliseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Millisecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Microsecond returns microseconds value for each element of `values`
+///
+/// \param[in] values input to extract microseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Microsecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Nanosecond returns nanoseconds value for each element of `values`
+///
+/// \param[in] values input to extract nanoseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Nanosecond(const Datum& values, ExecContext* ctx = NULLPTR);

Review comment:
       Well this is not ideal :).
   I propose to implement this as `total units since midnight` and then add an exception in python for nanoseconds.

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -450,5 +450,145 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Week returns week of year value for each element of `values`
+///
+/// \param[in] values input to extract week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Week(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns quarter of year value for each element of `values`
+///
+/// \param[in] values input to extract quarter of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Quarter(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Hour returns hour value for each element of `values`
+///
+/// \param[in] values input to extract hour from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Hour(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Minute returns minutes value for each element of `values`
+///
+/// \param[in] values input to extract minutes from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Minute(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Second returns seconds value for each element of `values`
+///
+/// \param[in] values input to extract seconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Second(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Millisecond returns milliseconds value for each element of `values`
+///
+/// \param[in] values input to extract milliseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Millisecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Microsecond returns microseconds value for each element of `values`
+///
+/// \param[in] values input to extract microseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Microsecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Nanosecond returns nanoseconds value for each element of `values`
+///
+/// \param[in] values input to extract nanoseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Nanosecond(const Datum& values, ExecContext* ctx = NULLPTR);

Review comment:
       Well this is not ideal :).
   I propose to implement this as `total units since last second` and then add an exception in python for nanoseconds.




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

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



[GitHub] [arrow] pitrou commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,239 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestSimpleTemporalComponentExtraction) {
+  const char* json =
+      R"(["1970-01-01T00:00:59","2000-02-29T23:23:23",
+          "3989-07-14T18:04:01","1900-01-01T01:59:20","2033-05-18T03:33:20"])";
+  auto time_points = ArrayFromJSON(timestamp(TimeUnit::SECOND), json);
+
+  auto year = ArrayFromJSON(int64(), "[1970, 2000, 3989, 1900, 2033]");
+  auto month = ArrayFromJSON(int64(), "[1, 2, 7, 1, 5]");
+  auto day = ArrayFromJSON(int64(), "[1, 29, 14, 1, 18]");
+  auto day_of_year = ArrayFromJSON(int64(), "[1, 60, 195, 1, 138]");
+  auto week = ArrayFromJSON(int64(), "[1, 9, 28, 1, 20]");
+  auto quarter = ArrayFromJSON(int64(), "[1, 1, 3, 1, 2]");
+  auto day_of_week = ArrayFromJSON(int64(), "[4, 2, 5, 1, 3]");
+  auto hour = ArrayFromJSON(int64(), "[0, 23, 18, 1, 3]");
+  auto minute = ArrayFromJSON(int64(), "[0, 23, 4, 59, 33]");
+  auto second = ArrayFromJSON(int64(), "[59, 23, 1, 20, 20]");
+  auto millisecond = ArrayFromJSON(int64(), "[0, 0, 0, 0, 0]");
+  auto microsecond = ArrayFromJSON(int64(), "[0, 0, 0, 0, 0]");
+  auto nanosecond = ArrayFromJSON(int64(), "[0, 0, 0, 0, 0]");
+
+  ASSERT_OK_AND_ASSIGN(Datum actual_year, Year(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_month, Month(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_day, Day(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_day_of_year, DayOfYear(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_week, Week(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_quarter, Quarter(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_day_of_week, DayOfWeek(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_hour, Hour(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_minute, Minute(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_second, Second(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_millisecond, Millisecond(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_microsecond, Microsecond(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_nanosecond, Nanosecond(time_points));
+
+  ASSERT_EQ(actual_year, year);
+  ASSERT_EQ(actual_month, month);
+  ASSERT_EQ(actual_day, day);
+  ASSERT_EQ(actual_day_of_year, day_of_year);
+  ASSERT_EQ(actual_week, week);
+  ASSERT_EQ(actual_quarter, quarter);
+  ASSERT_EQ(actual_day_of_week, day_of_week);
+  ASSERT_EQ(actual_hour, hour);
+  ASSERT_EQ(actual_minute, minute);
+  ASSERT_EQ(actual_second, second);
+  ASSERT_EQ(actual_millisecond, millisecond);
+  ASSERT_EQ(actual_microsecond, microsecond);
+  ASSERT_EQ(actual_nanosecond, nanosecond);
+
+  CheckScalarUnary("year", time_points, year);
+  CheckScalarUnary("month", time_points, month);
+  CheckScalarUnary("day", time_points, day);
+  CheckScalarUnary("day_of_year", time_points, day_of_year);
+  CheckScalarUnary("week", time_points, week);
+  CheckScalarUnary("quarter", time_points, quarter);
+  CheckScalarUnary("day_of_week", time_points, day_of_week);
+  CheckScalarUnary("hour", time_points, hour);
+  CheckScalarUnary("minute", time_points, minute);
+  CheckScalarUnary("second", time_points, second);
+  CheckScalarUnary("millisecond", time_points, millisecond);
+  CheckScalarUnary("microsecond", time_points, microsecond);
+  CheckScalarUnary("nanosecond", time_points, nanosecond);
+}
+
+TEST(ScalarTemporalTest, TestTemporalComponentExtraction) {
+  const char* json_second = "[59, 951866603, -2208981640, 2000000000]";
+  const char* json_milli = "[59000, 951866603000, -2208981640000, 2000000000000]";
+  const char* json_micro =
+      "[59000000, 951866603000000, -2208981640000000, 2000000000000000]";
+  const char* json_nano =
+      "[59000000000, 951866603000000000, -2208981640000000000, 2000000000000000000]";
+
+  auto time_points_second = ArrayFromJSON(timestamp(TimeUnit::SECOND), json_second);
+  auto time_points_milli = ArrayFromJSON(timestamp(TimeUnit::MILLI), json_milli);
+  auto time_points_micro = ArrayFromJSON(timestamp(TimeUnit::MICRO), json_micro);
+  auto time_points_nano = ArrayFromJSON(timestamp(TimeUnit::NANO), json_nano);
+
+  auto year = ArrayFromJSON(int64(), "[1970, 2000, 1900, 2033]");
+  auto month = ArrayFromJSON(int64(), "[1, 2, 1, 5]");
+  auto day = ArrayFromJSON(int64(), "[1, 29, 1, 18]");
+  auto day_of_year = ArrayFromJSON(int64(), "[1, 60, 1, 138]");
+  auto week = ArrayFromJSON(int64(), "[1, 9, 1, 20]");
+  auto quarter = ArrayFromJSON(int64(), "[1, 1, 1, 2]");
+  auto day_of_week = ArrayFromJSON(int64(), "[4, 2, 1, 3]");
+  auto hour = ArrayFromJSON(int64(), "[0, 23, 1, 3]");
+  auto minute = ArrayFromJSON(int64(), "[0, 23, 59, 33]");
+  auto second = ArrayFromJSON(int64(), "[59, 23, 20, 20]");
+  auto millisecond = ArrayFromJSON(int64(), "[0, 0, 0, 0]");
+  auto microsecond = ArrayFromJSON(int64(), "[0, 0, 0, 0]");
+  auto nanosecond = ArrayFromJSON(int64(), "[0, 0, 0, 0]");

Review comment:
       Hmm... so you never test with a non-zero number of milli/micro/nanoseconds?
   Also, how about testing with null values as well?

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,239 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestSimpleTemporalComponentExtraction) {
+  const char* json =
+      R"(["1970-01-01T00:00:59","2000-02-29T23:23:23",
+          "3989-07-14T18:04:01","1900-01-01T01:59:20","2033-05-18T03:33:20"])";
+  auto time_points = ArrayFromJSON(timestamp(TimeUnit::SECOND), json);
+
+  auto year = ArrayFromJSON(int64(), "[1970, 2000, 3989, 1900, 2033]");
+  auto month = ArrayFromJSON(int64(), "[1, 2, 7, 1, 5]");
+  auto day = ArrayFromJSON(int64(), "[1, 29, 14, 1, 18]");
+  auto day_of_year = ArrayFromJSON(int64(), "[1, 60, 195, 1, 138]");
+  auto week = ArrayFromJSON(int64(), "[1, 9, 28, 1, 20]");
+  auto quarter = ArrayFromJSON(int64(), "[1, 1, 3, 1, 2]");
+  auto day_of_week = ArrayFromJSON(int64(), "[4, 2, 5, 1, 3]");
+  auto hour = ArrayFromJSON(int64(), "[0, 23, 18, 1, 3]");
+  auto minute = ArrayFromJSON(int64(), "[0, 23, 4, 59, 33]");
+  auto second = ArrayFromJSON(int64(), "[59, 23, 1, 20, 20]");
+  auto millisecond = ArrayFromJSON(int64(), "[0, 0, 0, 0, 0]");
+  auto microsecond = ArrayFromJSON(int64(), "[0, 0, 0, 0, 0]");
+  auto nanosecond = ArrayFromJSON(int64(), "[0, 0, 0, 0, 0]");
+
+  ASSERT_OK_AND_ASSIGN(Datum actual_year, Year(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_month, Month(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_day, Day(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_day_of_year, DayOfYear(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_week, Week(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_quarter, Quarter(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_day_of_week, DayOfWeek(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_hour, Hour(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_minute, Minute(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_second, Second(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_millisecond, Millisecond(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_microsecond, Microsecond(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_nanosecond, Nanosecond(time_points));
+
+  ASSERT_EQ(actual_year, year);
+  ASSERT_EQ(actual_month, month);
+  ASSERT_EQ(actual_day, day);
+  ASSERT_EQ(actual_day_of_year, day_of_year);
+  ASSERT_EQ(actual_week, week);
+  ASSERT_EQ(actual_quarter, quarter);
+  ASSERT_EQ(actual_day_of_week, day_of_week);
+  ASSERT_EQ(actual_hour, hour);
+  ASSERT_EQ(actual_minute, minute);
+  ASSERT_EQ(actual_second, second);
+  ASSERT_EQ(actual_millisecond, millisecond);
+  ASSERT_EQ(actual_microsecond, microsecond);
+  ASSERT_EQ(actual_nanosecond, nanosecond);
+
+  CheckScalarUnary("year", time_points, year);
+  CheckScalarUnary("month", time_points, month);
+  CheckScalarUnary("day", time_points, day);
+  CheckScalarUnary("day_of_year", time_points, day_of_year);
+  CheckScalarUnary("week", time_points, week);
+  CheckScalarUnary("quarter", time_points, quarter);
+  CheckScalarUnary("day_of_week", time_points, day_of_week);
+  CheckScalarUnary("hour", time_points, hour);
+  CheckScalarUnary("minute", time_points, minute);
+  CheckScalarUnary("second", time_points, second);
+  CheckScalarUnary("millisecond", time_points, millisecond);
+  CheckScalarUnary("microsecond", time_points, microsecond);
+  CheckScalarUnary("nanosecond", time_points, nanosecond);
+}
+
+TEST(ScalarTemporalTest, TestTemporalComponentExtraction) {

Review comment:
       Is this test different from the previous one?

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,239 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestSimpleTemporalComponentExtraction) {
+  const char* json =
+      R"(["1970-01-01T00:00:59","2000-02-29T23:23:23",
+          "3989-07-14T18:04:01","1900-01-01T01:59:20","2033-05-18T03:33:20"])";
+  auto time_points = ArrayFromJSON(timestamp(TimeUnit::SECOND), json);
+
+  auto year = ArrayFromJSON(int64(), "[1970, 2000, 3989, 1900, 2033]");
+  auto month = ArrayFromJSON(int64(), "[1, 2, 7, 1, 5]");
+  auto day = ArrayFromJSON(int64(), "[1, 29, 14, 1, 18]");
+  auto day_of_year = ArrayFromJSON(int64(), "[1, 60, 195, 1, 138]");
+  auto week = ArrayFromJSON(int64(), "[1, 9, 28, 1, 20]");
+  auto quarter = ArrayFromJSON(int64(), "[1, 1, 3, 1, 2]");
+  auto day_of_week = ArrayFromJSON(int64(), "[4, 2, 5, 1, 3]");
+  auto hour = ArrayFromJSON(int64(), "[0, 23, 18, 1, 3]");
+  auto minute = ArrayFromJSON(int64(), "[0, 23, 4, 59, 33]");
+  auto second = ArrayFromJSON(int64(), "[59, 23, 1, 20, 20]");
+  auto millisecond = ArrayFromJSON(int64(), "[0, 0, 0, 0, 0]");
+  auto microsecond = ArrayFromJSON(int64(), "[0, 0, 0, 0, 0]");
+  auto nanosecond = ArrayFromJSON(int64(), "[0, 0, 0, 0, 0]");

Review comment:
       Is it desirable for all these values to be `int64`? A number of hours, minutes or seconds should fit in a `int8`, for example.

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,239 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestSimpleTemporalComponentExtraction) {
+  const char* json =
+      R"(["1970-01-01T00:00:59","2000-02-29T23:23:23",
+          "3989-07-14T18:04:01","1900-01-01T01:59:20","2033-05-18T03:33:20"])";
+  auto time_points = ArrayFromJSON(timestamp(TimeUnit::SECOND), json);
+
+  auto year = ArrayFromJSON(int64(), "[1970, 2000, 3989, 1900, 2033]");
+  auto month = ArrayFromJSON(int64(), "[1, 2, 7, 1, 5]");
+  auto day = ArrayFromJSON(int64(), "[1, 29, 14, 1, 18]");
+  auto day_of_year = ArrayFromJSON(int64(), "[1, 60, 195, 1, 138]");
+  auto week = ArrayFromJSON(int64(), "[1, 9, 28, 1, 20]");
+  auto quarter = ArrayFromJSON(int64(), "[1, 1, 3, 1, 2]");
+  auto day_of_week = ArrayFromJSON(int64(), "[4, 2, 5, 1, 3]");
+  auto hour = ArrayFromJSON(int64(), "[0, 23, 18, 1, 3]");
+  auto minute = ArrayFromJSON(int64(), "[0, 23, 4, 59, 33]");
+  auto second = ArrayFromJSON(int64(), "[59, 23, 1, 20, 20]");
+  auto millisecond = ArrayFromJSON(int64(), "[0, 0, 0, 0, 0]");
+  auto microsecond = ArrayFromJSON(int64(), "[0, 0, 0, 0, 0]");
+  auto nanosecond = ArrayFromJSON(int64(), "[0, 0, 0, 0, 0]");
+
+  ASSERT_OK_AND_ASSIGN(Datum actual_year, Year(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_month, Month(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_day, Day(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_day_of_year, DayOfYear(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_week, Week(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_quarter, Quarter(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_day_of_week, DayOfWeek(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_hour, Hour(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_minute, Minute(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_second, Second(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_millisecond, Millisecond(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_microsecond, Microsecond(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_nanosecond, Nanosecond(time_points));
+
+  ASSERT_EQ(actual_year, year);
+  ASSERT_EQ(actual_month, month);
+  ASSERT_EQ(actual_day, day);
+  ASSERT_EQ(actual_day_of_year, day_of_year);
+  ASSERT_EQ(actual_week, week);
+  ASSERT_EQ(actual_quarter, quarter);
+  ASSERT_EQ(actual_day_of_week, day_of_week);
+  ASSERT_EQ(actual_hour, hour);
+  ASSERT_EQ(actual_minute, minute);
+  ASSERT_EQ(actual_second, second);
+  ASSERT_EQ(actual_millisecond, millisecond);
+  ASSERT_EQ(actual_microsecond, microsecond);
+  ASSERT_EQ(actual_nanosecond, nanosecond);
+
+  CheckScalarUnary("year", time_points, year);
+  CheckScalarUnary("month", time_points, month);
+  CheckScalarUnary("day", time_points, day);
+  CheckScalarUnary("day_of_year", time_points, day_of_year);
+  CheckScalarUnary("week", time_points, week);
+  CheckScalarUnary("quarter", time_points, quarter);
+  CheckScalarUnary("day_of_week", time_points, day_of_week);
+  CheckScalarUnary("hour", time_points, hour);
+  CheckScalarUnary("minute", time_points, minute);
+  CheckScalarUnary("second", time_points, second);
+  CheckScalarUnary("millisecond", time_points, millisecond);
+  CheckScalarUnary("microsecond", time_points, microsecond);
+  CheckScalarUnary("nanosecond", time_points, nanosecond);
+}
+
+TEST(ScalarTemporalTest, TestTemporalComponentExtraction) {
+  const char* json_second = "[59, 951866603, -2208981640, 2000000000]";
+  const char* json_milli = "[59000, 951866603000, -2208981640000, 2000000000000]";
+  const char* json_micro =
+      "[59000000, 951866603000000, -2208981640000000, 2000000000000000]";
+  const char* json_nano =
+      "[59000000000, 951866603000000000, -2208981640000000000, 2000000000000000000]";
+
+  auto time_points_second = ArrayFromJSON(timestamp(TimeUnit::SECOND), json_second);
+  auto time_points_milli = ArrayFromJSON(timestamp(TimeUnit::MILLI), json_milli);
+  auto time_points_micro = ArrayFromJSON(timestamp(TimeUnit::MICRO), json_micro);
+  auto time_points_nano = ArrayFromJSON(timestamp(TimeUnit::NANO), json_nano);
+
+  auto year = ArrayFromJSON(int64(), "[1970, 2000, 1900, 2033]");
+  auto month = ArrayFromJSON(int64(), "[1, 2, 1, 5]");
+  auto day = ArrayFromJSON(int64(), "[1, 29, 1, 18]");
+  auto day_of_year = ArrayFromJSON(int64(), "[1, 60, 1, 138]");
+  auto week = ArrayFromJSON(int64(), "[1, 9, 1, 20]");
+  auto quarter = ArrayFromJSON(int64(), "[1, 1, 1, 2]");
+  auto day_of_week = ArrayFromJSON(int64(), "[4, 2, 1, 3]");
+  auto hour = ArrayFromJSON(int64(), "[0, 23, 1, 3]");
+  auto minute = ArrayFromJSON(int64(), "[0, 23, 59, 33]");
+  auto second = ArrayFromJSON(int64(), "[59, 23, 20, 20]");
+  auto millisecond = ArrayFromJSON(int64(), "[0, 0, 0, 0]");
+  auto microsecond = ArrayFromJSON(int64(), "[0, 0, 0, 0]");
+  auto nanosecond = ArrayFromJSON(int64(), "[0, 0, 0, 0]");
+
+  for (auto time_points :
+       {time_points_second, time_points_milli, time_points_micro, time_points_nano}) {
+    CheckScalarUnary("year", time_points, year);
+    CheckScalarUnary("month", time_points, month);
+    CheckScalarUnary("day", time_points, day);
+    CheckScalarUnary("day_of_year", time_points, day_of_year);
+    CheckScalarUnary("week", time_points, week);
+    CheckScalarUnary("quarter", time_points, quarter);
+    CheckScalarUnary("day_of_week", time_points, day_of_week);
+    CheckScalarUnary("hour", time_points, hour);
+    CheckScalarUnary("minute", time_points, minute);
+    CheckScalarUnary("second", time_points, second);
+    CheckScalarUnary("millisecond", time_points, millisecond);
+    CheckScalarUnary("microsecond", time_points, microsecond);
+    CheckScalarUnary("nanosecond", time_points, nanosecond);
+  }
+
+  std::string in = "[123, 999, 1, 31231000]";
+  auto out = ArrayFromJSON(int64(), "[123, 999, 1, 0]");
+
+  auto tp_milli = ArrayFromJSON(timestamp(TimeUnit::MILLI), in);
+  auto tp_milli_zoned = ArrayFromJSON(timestamp(TimeUnit::MILLI, "Etc/GMT+2"), in);
+  CheckScalarUnary("millisecond", tp_milli, out);
+  CheckScalarUnary("millisecond", tp_milli, out);
+
+  auto tp_micro = ArrayFromJSON(timestamp(TimeUnit::MICRO), in);
+  auto tp_micro_zoned = ArrayFromJSON(timestamp(TimeUnit::MICRO, "Etc/GMT+2"), in);
+  CheckScalarUnary("microsecond", tp_micro, out);
+  CheckScalarUnary("microsecond", tp_micro_zoned, out);
+
+  auto tp_nano = ArrayFromJSON(timestamp(TimeUnit::NANO), in);
+  auto tp_nano_zoned = ArrayFromJSON(timestamp(TimeUnit::NANO, "Etc/GMT+2"), in);
+  CheckScalarUnary("nanosecond", tp_nano, out);
+  CheckScalarUnary("nanosecond", tp_nano_zoned, out);
+}
+
+TEST(ScalarTemporalTest, TestSimpleZonedTemporalComponentExtraction) {
+  const char* json =
+      R"(["1970-01-01T00:00:59","2000-02-29T23:23:23",
+          "3989-07-14T18:04:01","1900-01-01T01:59:20","2033-05-18T03:33:20"])";
+  auto time_points = ArrayFromJSON(timestamp(TimeUnit::SECOND, "Etc/GMT+2"), json);
+
+  auto year = ArrayFromJSON(int64(), "[1969, 2000, 3989, 1899, 2033]");
+  auto month = ArrayFromJSON(int64(), "[12, 2, 7, 12, 5]");
+  auto day = ArrayFromJSON(int64(), "[31, 29, 14, 31, 18]");
+  auto day_of_year = ArrayFromJSON(int64(), "[365, 60, 195, 365, 138]");
+  auto week = ArrayFromJSON(int64(), "[1, 9, 28, 52, 20]");
+  auto quarter = ArrayFromJSON(int64(), "[4, 1, 3, 4, 2]");
+  auto day_of_week = ArrayFromJSON(int64(), "[3, 2, 5, 7, 3]");
+  auto hour = ArrayFromJSON(int64(), "[22, 21, 16, 23, 1]");
+  auto minute = ArrayFromJSON(int64(), "[0, 23, 4, 59, 33]");
+  auto second = ArrayFromJSON(int64(), "[59, 23, 1, 20, 20]");
+  auto millisecond = ArrayFromJSON(int64(), "[0, 0, 0, 0, 0]");
+  auto microsecond = ArrayFromJSON(int64(), "[0, 0, 0, 0, 0]");
+  auto nanosecond = ArrayFromJSON(int64(), "[0, 0, 0, 0, 0]");
+
+  CheckScalarUnary("year", time_points, year);
+  CheckScalarUnary("month", time_points, month);
+  CheckScalarUnary("day", time_points, day);
+  CheckScalarUnary("day_of_year", time_points, day_of_year);
+  CheckScalarUnary("week", time_points, week);
+  CheckScalarUnary("quarter", time_points, quarter);
+  CheckScalarUnary("day_of_week", time_points, day_of_week);
+  CheckScalarUnary("hour", time_points, hour);
+  CheckScalarUnary("minute", time_points, minute);
+  CheckScalarUnary("second", time_points, second);
+  CheckScalarUnary("millisecond", time_points, millisecond);
+  CheckScalarUnary("microsecond", time_points, microsecond);
+  CheckScalarUnary("nanosecond", time_points, nanosecond);
+}
+
+TEST(ScalarTemporalTest, TestZonedTemporalComponentExtraction) {
+  std::string timezone = "Etc/GMT+2";

Review comment:
       Again, is this testing something different compared to above?

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,632 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::local_days;
+using arrow_vendored::date::locate_zone;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+template <typename Duration>
+inline year_month_day ymd_caster_template(const int64_t data) {
+  return year_month_day(floor<days>(sys_time<Duration>(Duration{data})));
+}
+
+template <typename Duration>
+inline std::function<year_month_day(const int64_t)> ymd_caster_zoned_template(
+    const std::string timezone) {
+  static const arrow_vendored::date::time_zone* tz = locate_zone(timezone);
+  return [](const int64_t data) {
+    return year_month_day(floor<days>(tz->to_local(sys_time<Duration>(Duration{data}))));
+  };
+}
+
+inline std::function<year_month_day(const int64_t)> make_ymd_caster(
+    const std::shared_ptr<DataType> type) {
+  const auto ts_type = std::static_pointer_cast<const TimestampType>(type);
+  const TimeUnit::type unit = ts_type->unit();
+  const std::string timezone = ts_type->timezone();
+
+  if (timezone.empty()) {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return ymd_caster_template<std::chrono::seconds>;
+      case TimeUnit::MILLI:
+        return ymd_caster_template<std::chrono::milliseconds>;
+      case TimeUnit::MICRO:
+        return ymd_caster_template<std::chrono::microseconds>;
+      case TimeUnit::NANO:
+        return ymd_caster_template<std::chrono::nanoseconds>;
+    }
+  } else {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return ymd_caster_zoned_template<std::chrono::seconds>(timezone);
+      case TimeUnit::MILLI:
+        return ymd_caster_zoned_template<std::chrono::milliseconds>(timezone);
+      case TimeUnit::MICRO:
+        return ymd_caster_zoned_template<std::chrono::microseconds>(timezone);
+      case TimeUnit::NANO:
+        return ymd_caster_zoned_template<std::chrono::nanoseconds>(timezone);
+    }
+  }
+  return ymd_caster_template<std::chrono::seconds>;
+}
+
+template <typename DurationIn, typename DurationOut>
+inline hh_mm_ss<DurationOut> hhmmss_caster_template(const int64_t data) {
+  DurationIn t = DurationIn{data};
+  return hh_mm_ss<DurationOut>(
+      std::chrono::duration_cast<DurationOut>(t - floor<days>(t)));
+}
+
+template <typename DurationIn, typename DurationOut>
+inline std::function<hh_mm_ss<DurationOut>(const int64_t)> hhmmss_caster_zoned_template(
+    const std::string timezone) {
+  static const arrow_vendored::date::time_zone* tz = locate_zone(timezone);
+  return [](const int64_t data) {
+    const auto z = sys_time<DurationIn>(DurationIn{data});
+    const auto l = make_zoned(tz, z).get_local_time();
+    return hh_mm_ss<DurationOut>(
+        std::chrono::duration_cast<DurationOut>(l - floor<days>(l)));
+  };
+}
+
+template <typename Duration>
+inline std::function<hh_mm_ss<Duration>(const int64_t)> make_hhmmss_caster(
+    const std::shared_ptr<DataType> type) {
+  const auto ts_type = std::static_pointer_cast<const TimestampType>(type);
+  const TimeUnit::type unit = ts_type->unit();
+  const std::string timezone = ts_type->timezone();
+
+  if (timezone.empty()) {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return hhmmss_caster_template<std::chrono::seconds, Duration>;
+      case TimeUnit::MILLI:
+        return hhmmss_caster_template<std::chrono::milliseconds, Duration>;
+      case TimeUnit::MICRO:
+        return hhmmss_caster_template<std::chrono::microseconds, Duration>;
+      case TimeUnit::NANO:
+        return hhmmss_caster_template<std::chrono::nanoseconds, Duration>;
+    }
+  } else {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return hhmmss_caster_zoned_template<std::chrono::seconds, Duration>(timezone);
+      case TimeUnit::MILLI:
+        return hhmmss_caster_zoned_template<std::chrono::milliseconds, Duration>(
+            timezone);
+      case TimeUnit::MICRO:
+        return hhmmss_caster_zoned_template<std::chrono::microseconds, Duration>(
+            timezone);
+      case TimeUnit::NANO:
+        return hhmmss_caster_zoned_template<std::chrono::nanoseconds, Duration>(timezone);
+    }
+  }
+  return hhmmss_caster_template<std::chrono::seconds, Duration>;
+}
+
+template <typename Duration>
+inline unsigned day_of_year_caster_template(const int64_t data) {
+  const auto sd = sys_days{floor<days>(Duration{data})};
+  const auto y = year_month_day(sd).year();
+  return static_cast<unsigned>((sd - sys_days(y / jan / 0)).count());
+}
+
+template <typename Duration>
+inline std::function<unsigned(const int64_t)> day_of_year_zoned_caster_template(
+    const std::string timezone) {
+  static const arrow_vendored::date::time_zone* tz = locate_zone(timezone);
+  return [](const int64_t data) {
+    auto ld =
+        year_month_day(floor<days>(tz->to_local(sys_time<Duration>(Duration{data}))));
+    return static_cast<unsigned>(
+        (local_days(ld) - local_days(ld.year() / jan / 1) + days{1}).count());
+  };
+}
+
+inline std::function<unsigned(const int64_t)> get_day_of_year_caster(
+    const std::shared_ptr<DataType> type) {
+  const auto ts_type = std::static_pointer_cast<const TimestampType>(type);
+  const TimeUnit::type unit = ts_type->unit();
+  const std::string timezone = ts_type->timezone();
+
+  if (timezone.empty()) {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return day_of_year_caster_template<std::chrono::seconds>;
+      case TimeUnit::MILLI:
+        return day_of_year_caster_template<std::chrono::milliseconds>;
+      case TimeUnit::MICRO:
+        return day_of_year_caster_template<std::chrono::microseconds>;
+      case TimeUnit::NANO:
+        return day_of_year_caster_template<std::chrono::nanoseconds>;
+    }
+  } else {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return day_of_year_zoned_caster_template<std::chrono::seconds>(timezone);
+      case TimeUnit::MILLI:
+        return day_of_year_zoned_caster_template<std::chrono::milliseconds>(timezone);
+      case TimeUnit::MICRO:
+        return day_of_year_zoned_caster_template<std::chrono::microseconds>(timezone);
+      case TimeUnit::NANO:
+        return day_of_year_zoned_caster_template<std::chrono::nanoseconds>(timezone);
+    }
+  }
+  return day_of_year_caster_template<std::chrono::seconds>;
+}
+
+template <typename Duration>
+inline unsigned week_caster_template(const int64_t data) {
+  // Based on
+  // https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+  const auto dp = sys_days{floor<days>(Duration{data})};
+  auto y = year_month_day{dp + days{3}}.year();
+  auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+  if (dp < start) {
+    --y;
+    start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+  }
+  return static_cast<unsigned>(trunc<weeks>(dp - start).count() + 1);
+}
+
+template <typename Duration>
+inline std::function<unsigned(const int64_t)> week_zoned_caster_template(
+    const std::string timezone) {
+  static const arrow_vendored::date::time_zone* tz = locate_zone(timezone);
+  return [](const int64_t data) {
+    const auto ld = floor<days>(tz->to_local(sys_time<Duration>(Duration{data})));
+    auto y = year_month_day{ld + days{3}}.year();
+    auto start = local_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (ld < start) {
+      --y;
+      start = local_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<unsigned>(trunc<weeks>(local_days(ld) - start).count() + 1);
+  };
+}
+
+inline std::function<unsigned(const int64_t)> make_week_caster(
+    const std::shared_ptr<DataType> type) {
+  const auto ts_type = std::static_pointer_cast<const TimestampType>(type);
+  const TimeUnit::type unit = ts_type->unit();
+  const std::string timezone = ts_type->timezone();
+
+  if (timezone.empty()) {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return week_caster_template<std::chrono::seconds>;
+      case TimeUnit::MILLI:
+        return week_caster_template<std::chrono::milliseconds>;
+      case TimeUnit::MICRO:
+        return week_caster_template<std::chrono::microseconds>;
+      case TimeUnit::NANO:
+        return week_caster_template<std::chrono::nanoseconds>;
+    }
+  } else {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return week_zoned_caster_template<std::chrono::seconds>(timezone);
+      case TimeUnit::MILLI:
+        return week_zoned_caster_template<std::chrono::milliseconds>(timezone);
+      case TimeUnit::MICRO:
+        return week_zoned_caster_template<std::chrono::microseconds>(timezone);
+      case TimeUnit::NANO:
+        return week_zoned_caster_template<std::chrono::nanoseconds>(timezone);
+    }
+  }
+  return day_of_year_caster_template<std::chrono::seconds>;
+}
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename out_type>
+struct Year {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const auto& in_data = internal::UnboxScalar<const TimestampType>::Unbox(in);
+    auto ymd_caster = make_ymd_caster(in.type);
+    checked_cast<Int64Scalar*>(out)->value = static_cast<int>(ymd_caster(in_data).year());
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    auto in_data = in.GetValues<uint64_t>(1);
+    auto out_data = out->GetMutableValues<out_type>(1);
+    auto ymd_caster = make_ymd_caster(in.type);
+    for (int64_t i = 0; i < in.length; i++) {
+      out_data[i] = static_cast<int>(ymd_caster(in_data[i]).year());
+    }
+    return Status::OK();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename out_type>
+struct Month {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const auto& in_data = internal::UnboxScalar<const TimestampType>::Unbox(in);
+    auto ymd_caster = make_ymd_caster(in.type);
+    checked_cast<Int64Scalar*>(out)->value =
+        static_cast<unsigned>(ymd_caster(in_data).month());
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    auto in_data = in.GetValues<uint64_t>(1);
+    auto out_data = out->GetMutableValues<out_type>(1);
+    auto ymd_caster = make_ymd_caster(in.type);
+    for (int64_t i = 0; i < in.length; i++) {
+      out_data[i] = static_cast<unsigned>(ymd_caster(in_data[i]).month());
+    }
+    return Status::OK();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename out_type>
+struct Day {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const int64_t& in_data = internal::UnboxScalar<const TimestampType>::Unbox(in);
+    auto ymd_caster = make_ymd_caster(in.type);
+    checked_cast<Int64Scalar*>(out)->value =
+        static_cast<unsigned>(ymd_caster(in_data).day());
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    auto in_data = in.GetValues<uint64_t>(1);
+    auto out_data = out->GetMutableValues<out_type>(1);
+    auto ymd_caster = make_ymd_caster(in.type);
+    for (int64_t i = 0; i < in.length; i++) {
+      out_data[i] = static_cast<unsigned>(ymd_caster(in_data[i]).day());
+    }
+    return Status::OK();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract week from timestamp
+
+template <typename out_type>
+struct Week {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const auto& in_data = internal::UnboxScalar<const TimestampType>::Unbox(in);
+    auto week_caster = make_week_caster(in.type);
+    checked_cast<Int64Scalar*>(out)->value = week_caster(in_data);
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    auto in_data = in.GetValues<uint64_t>(1);
+    auto out_data = out->GetMutableValues<out_type>(1);
+    auto week_caster = make_week_caster(in.type);
+    for (int64_t i = 0; i < in.length; i++) {
+      out_data[i] = week_caster(in_data[i]);
+    }
+    return Status::OK();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract quarter from timestamp
+
+template <typename out_type>
+struct Quarter {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const auto& in_data = internal::UnboxScalar<const TimestampType>::Unbox(in);
+    auto ymd_caster = make_ymd_caster(in.type);
+    checked_cast<Int64Scalar*>(out)->value =
+        (static_cast<unsigned>(ymd_caster(in_data).month()) - 1) / 3 + 1;
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    auto in_data = in.GetValues<uint64_t>(1);
+    auto out_data = out->GetMutableValues<out_type>(1);
+    auto ymd_caster = make_ymd_caster(in.type);
+    for (int64_t i = 0; i < in.length; i++) {
+      out_data[i] = (static_cast<unsigned>(ymd_caster(in_data[i]).month()) - 1) / 3 + 1;
+    }
+    return Status::OK();
+  }
+};

Review comment:
       You're repeating yourself a lot. You basically have specific extraction function (the "caster" functions) and a generic kernel structure that just applies the extraction function. It should be possible to minimize the amount of code copied and pasted in this file.

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,632 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::local_days;
+using arrow_vendored::date::locate_zone;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+template <typename Duration>
+inline year_month_day ymd_caster_template(const int64_t data) {

Review comment:
       Please follow the coding convention guidelines, i.e. `CamelCase` for functions.

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,632 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::local_days;
+using arrow_vendored::date::locate_zone;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+template <typename Duration>
+inline year_month_day ymd_caster_template(const int64_t data) {
+  return year_month_day(floor<days>(sys_time<Duration>(Duration{data})));
+}
+
+template <typename Duration>
+inline std::function<year_month_day(const int64_t)> ymd_caster_zoned_template(
+    const std::string timezone) {
+  static const arrow_vendored::date::time_zone* tz = locate_zone(timezone);
+  return [](const int64_t data) {
+    return year_month_day(floor<days>(tz->to_local(sys_time<Duration>(Duration{data}))));
+  };
+}
+
+inline std::function<year_month_day(const int64_t)> make_ymd_caster(
+    const std::shared_ptr<DataType> type) {
+  const auto ts_type = std::static_pointer_cast<const TimestampType>(type);
+  const TimeUnit::type unit = ts_type->unit();
+  const std::string timezone = ts_type->timezone();
+
+  if (timezone.empty()) {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return ymd_caster_template<std::chrono::seconds>;
+      case TimeUnit::MILLI:
+        return ymd_caster_template<std::chrono::milliseconds>;
+      case TimeUnit::MICRO:
+        return ymd_caster_template<std::chrono::microseconds>;
+      case TimeUnit::NANO:
+        return ymd_caster_template<std::chrono::nanoseconds>;
+    }
+  } else {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return ymd_caster_zoned_template<std::chrono::seconds>(timezone);
+      case TimeUnit::MILLI:
+        return ymd_caster_zoned_template<std::chrono::milliseconds>(timezone);
+      case TimeUnit::MICRO:
+        return ymd_caster_zoned_template<std::chrono::microseconds>(timezone);
+      case TimeUnit::NANO:
+        return ymd_caster_zoned_template<std::chrono::nanoseconds>(timezone);
+    }
+  }
+  return ymd_caster_template<std::chrono::seconds>;
+}
+
+template <typename DurationIn, typename DurationOut>
+inline hh_mm_ss<DurationOut> hhmmss_caster_template(const int64_t data) {
+  DurationIn t = DurationIn{data};
+  return hh_mm_ss<DurationOut>(
+      std::chrono::duration_cast<DurationOut>(t - floor<days>(t)));
+}
+
+template <typename DurationIn, typename DurationOut>
+inline std::function<hh_mm_ss<DurationOut>(const int64_t)> hhmmss_caster_zoned_template(
+    const std::string timezone) {
+  static const arrow_vendored::date::time_zone* tz = locate_zone(timezone);
+  return [](const int64_t data) {
+    const auto z = sys_time<DurationIn>(DurationIn{data});
+    const auto l = make_zoned(tz, z).get_local_time();
+    return hh_mm_ss<DurationOut>(
+        std::chrono::duration_cast<DurationOut>(l - floor<days>(l)));
+  };
+}
+
+template <typename Duration>
+inline std::function<hh_mm_ss<Duration>(const int64_t)> make_hhmmss_caster(
+    const std::shared_ptr<DataType> type) {
+  const auto ts_type = std::static_pointer_cast<const TimestampType>(type);
+  const TimeUnit::type unit = ts_type->unit();
+  const std::string timezone = ts_type->timezone();
+
+  if (timezone.empty()) {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return hhmmss_caster_template<std::chrono::seconds, Duration>;
+      case TimeUnit::MILLI:
+        return hhmmss_caster_template<std::chrono::milliseconds, Duration>;
+      case TimeUnit::MICRO:
+        return hhmmss_caster_template<std::chrono::microseconds, Duration>;
+      case TimeUnit::NANO:
+        return hhmmss_caster_template<std::chrono::nanoseconds, Duration>;
+    }
+  } else {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return hhmmss_caster_zoned_template<std::chrono::seconds, Duration>(timezone);
+      case TimeUnit::MILLI:
+        return hhmmss_caster_zoned_template<std::chrono::milliseconds, Duration>(
+            timezone);
+      case TimeUnit::MICRO:
+        return hhmmss_caster_zoned_template<std::chrono::microseconds, Duration>(
+            timezone);
+      case TimeUnit::NANO:
+        return hhmmss_caster_zoned_template<std::chrono::nanoseconds, Duration>(timezone);
+    }
+  }
+  return hhmmss_caster_template<std::chrono::seconds, Duration>;
+}
+
+template <typename Duration>
+inline unsigned day_of_year_caster_template(const int64_t data) {
+  const auto sd = sys_days{floor<days>(Duration{data})};
+  const auto y = year_month_day(sd).year();
+  return static_cast<unsigned>((sd - sys_days(y / jan / 0)).count());
+}
+
+template <typename Duration>
+inline std::function<unsigned(const int64_t)> day_of_year_zoned_caster_template(
+    const std::string timezone) {
+  static const arrow_vendored::date::time_zone* tz = locate_zone(timezone);
+  return [](const int64_t data) {
+    auto ld =
+        year_month_day(floor<days>(tz->to_local(sys_time<Duration>(Duration{data}))));
+    return static_cast<unsigned>(
+        (local_days(ld) - local_days(ld.year() / jan / 1) + days{1}).count());
+  };
+}
+
+inline std::function<unsigned(const int64_t)> get_day_of_year_caster(
+    const std::shared_ptr<DataType> type) {
+  const auto ts_type = std::static_pointer_cast<const TimestampType>(type);
+  const TimeUnit::type unit = ts_type->unit();
+  const std::string timezone = ts_type->timezone();
+
+  if (timezone.empty()) {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return day_of_year_caster_template<std::chrono::seconds>;
+      case TimeUnit::MILLI:
+        return day_of_year_caster_template<std::chrono::milliseconds>;
+      case TimeUnit::MICRO:
+        return day_of_year_caster_template<std::chrono::microseconds>;
+      case TimeUnit::NANO:
+        return day_of_year_caster_template<std::chrono::nanoseconds>;
+    }
+  } else {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return day_of_year_zoned_caster_template<std::chrono::seconds>(timezone);
+      case TimeUnit::MILLI:
+        return day_of_year_zoned_caster_template<std::chrono::milliseconds>(timezone);
+      case TimeUnit::MICRO:
+        return day_of_year_zoned_caster_template<std::chrono::microseconds>(timezone);
+      case TimeUnit::NANO:
+        return day_of_year_zoned_caster_template<std::chrono::nanoseconds>(timezone);
+    }
+  }
+  return day_of_year_caster_template<std::chrono::seconds>;
+}
+
+template <typename Duration>
+inline unsigned week_caster_template(const int64_t data) {
+  // Based on
+  // https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+  const auto dp = sys_days{floor<days>(Duration{data})};
+  auto y = year_month_day{dp + days{3}}.year();
+  auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+  if (dp < start) {
+    --y;
+    start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+  }
+  return static_cast<unsigned>(trunc<weeks>(dp - start).count() + 1);
+}
+
+template <typename Duration>
+inline std::function<unsigned(const int64_t)> week_zoned_caster_template(
+    const std::string timezone) {
+  static const arrow_vendored::date::time_zone* tz = locate_zone(timezone);
+  return [](const int64_t data) {
+    const auto ld = floor<days>(tz->to_local(sys_time<Duration>(Duration{data})));
+    auto y = year_month_day{ld + days{3}}.year();
+    auto start = local_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (ld < start) {
+      --y;
+      start = local_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<unsigned>(trunc<weeks>(local_days(ld) - start).count() + 1);
+  };
+}
+
+inline std::function<unsigned(const int64_t)> make_week_caster(
+    const std::shared_ptr<DataType> type) {
+  const auto ts_type = std::static_pointer_cast<const TimestampType>(type);
+  const TimeUnit::type unit = ts_type->unit();
+  const std::string timezone = ts_type->timezone();
+
+  if (timezone.empty()) {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return week_caster_template<std::chrono::seconds>;
+      case TimeUnit::MILLI:
+        return week_caster_template<std::chrono::milliseconds>;
+      case TimeUnit::MICRO:
+        return week_caster_template<std::chrono::microseconds>;
+      case TimeUnit::NANO:
+        return week_caster_template<std::chrono::nanoseconds>;
+    }
+  } else {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return week_zoned_caster_template<std::chrono::seconds>(timezone);
+      case TimeUnit::MILLI:
+        return week_zoned_caster_template<std::chrono::milliseconds>(timezone);
+      case TimeUnit::MICRO:
+        return week_zoned_caster_template<std::chrono::microseconds>(timezone);
+      case TimeUnit::NANO:
+        return week_zoned_caster_template<std::chrono::nanoseconds>(timezone);
+    }
+  }
+  return day_of_year_caster_template<std::chrono::seconds>;
+}
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename out_type>

Review comment:
       As per the coding conventions, this should probably `OutType`.

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,239 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestSimpleTemporalComponentExtraction) {
+  const char* json =
+      R"(["1970-01-01T00:00:59","2000-02-29T23:23:23",
+          "3989-07-14T18:04:01","1900-01-01T01:59:20","2033-05-18T03:33:20"])";
+  auto time_points = ArrayFromJSON(timestamp(TimeUnit::SECOND), json);
+
+  auto year = ArrayFromJSON(int64(), "[1970, 2000, 3989, 1900, 2033]");
+  auto month = ArrayFromJSON(int64(), "[1, 2, 7, 1, 5]");
+  auto day = ArrayFromJSON(int64(), "[1, 29, 14, 1, 18]");
+  auto day_of_year = ArrayFromJSON(int64(), "[1, 60, 195, 1, 138]");
+  auto week = ArrayFromJSON(int64(), "[1, 9, 28, 1, 20]");
+  auto quarter = ArrayFromJSON(int64(), "[1, 1, 3, 1, 2]");
+  auto day_of_week = ArrayFromJSON(int64(), "[4, 2, 5, 1, 3]");
+  auto hour = ArrayFromJSON(int64(), "[0, 23, 18, 1, 3]");
+  auto minute = ArrayFromJSON(int64(), "[0, 23, 4, 59, 33]");
+  auto second = ArrayFromJSON(int64(), "[59, 23, 1, 20, 20]");
+  auto millisecond = ArrayFromJSON(int64(), "[0, 0, 0, 0, 0]");
+  auto microsecond = ArrayFromJSON(int64(), "[0, 0, 0, 0, 0]");
+  auto nanosecond = ArrayFromJSON(int64(), "[0, 0, 0, 0, 0]");
+
+  ASSERT_OK_AND_ASSIGN(Datum actual_year, Year(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_month, Month(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_day, Day(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_day_of_year, DayOfYear(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_week, Week(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_quarter, Quarter(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_day_of_week, DayOfWeek(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_hour, Hour(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_minute, Minute(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_second, Second(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_millisecond, Millisecond(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_microsecond, Microsecond(time_points));
+  ASSERT_OK_AND_ASSIGN(Datum actual_nanosecond, Nanosecond(time_points));
+
+  ASSERT_EQ(actual_year, year);
+  ASSERT_EQ(actual_month, month);
+  ASSERT_EQ(actual_day, day);
+  ASSERT_EQ(actual_day_of_year, day_of_year);
+  ASSERT_EQ(actual_week, week);
+  ASSERT_EQ(actual_quarter, quarter);
+  ASSERT_EQ(actual_day_of_week, day_of_week);
+  ASSERT_EQ(actual_hour, hour);
+  ASSERT_EQ(actual_minute, minute);
+  ASSERT_EQ(actual_second, second);
+  ASSERT_EQ(actual_millisecond, millisecond);
+  ASSERT_EQ(actual_microsecond, microsecond);
+  ASSERT_EQ(actual_nanosecond, nanosecond);
+
+  CheckScalarUnary("year", time_points, year);
+  CheckScalarUnary("month", time_points, month);
+  CheckScalarUnary("day", time_points, day);
+  CheckScalarUnary("day_of_year", time_points, day_of_year);
+  CheckScalarUnary("week", time_points, week);
+  CheckScalarUnary("quarter", time_points, quarter);
+  CheckScalarUnary("day_of_week", time_points, day_of_week);
+  CheckScalarUnary("hour", time_points, hour);
+  CheckScalarUnary("minute", time_points, minute);
+  CheckScalarUnary("second", time_points, second);
+  CheckScalarUnary("millisecond", time_points, millisecond);
+  CheckScalarUnary("microsecond", time_points, microsecond);
+  CheckScalarUnary("nanosecond", time_points, nanosecond);
+}
+
+TEST(ScalarTemporalTest, TestTemporalComponentExtraction) {
+  const char* json_second = "[59, 951866603, -2208981640, 2000000000]";
+  const char* json_milli = "[59000, 951866603000, -2208981640000, 2000000000000]";
+  const char* json_micro =
+      "[59000000, 951866603000000, -2208981640000000, 2000000000000000]";
+  const char* json_nano =
+      "[59000000000, 951866603000000000, -2208981640000000000, 2000000000000000000]";
+
+  auto time_points_second = ArrayFromJSON(timestamp(TimeUnit::SECOND), json_second);
+  auto time_points_milli = ArrayFromJSON(timestamp(TimeUnit::MILLI), json_milli);
+  auto time_points_micro = ArrayFromJSON(timestamp(TimeUnit::MICRO), json_micro);
+  auto time_points_nano = ArrayFromJSON(timestamp(TimeUnit::NANO), json_nano);
+
+  auto year = ArrayFromJSON(int64(), "[1970, 2000, 1900, 2033]");
+  auto month = ArrayFromJSON(int64(), "[1, 2, 1, 5]");
+  auto day = ArrayFromJSON(int64(), "[1, 29, 1, 18]");
+  auto day_of_year = ArrayFromJSON(int64(), "[1, 60, 1, 138]");
+  auto week = ArrayFromJSON(int64(), "[1, 9, 1, 20]");
+  auto quarter = ArrayFromJSON(int64(), "[1, 1, 1, 2]");
+  auto day_of_week = ArrayFromJSON(int64(), "[4, 2, 1, 3]");
+  auto hour = ArrayFromJSON(int64(), "[0, 23, 1, 3]");
+  auto minute = ArrayFromJSON(int64(), "[0, 23, 59, 33]");
+  auto second = ArrayFromJSON(int64(), "[59, 23, 20, 20]");
+  auto millisecond = ArrayFromJSON(int64(), "[0, 0, 0, 0]");
+  auto microsecond = ArrayFromJSON(int64(), "[0, 0, 0, 0]");
+  auto nanosecond = ArrayFromJSON(int64(), "[0, 0, 0, 0]");
+
+  for (auto time_points :
+       {time_points_second, time_points_milli, time_points_micro, time_points_nano}) {
+    CheckScalarUnary("year", time_points, year);
+    CheckScalarUnary("month", time_points, month);
+    CheckScalarUnary("day", time_points, day);
+    CheckScalarUnary("day_of_year", time_points, day_of_year);
+    CheckScalarUnary("week", time_points, week);
+    CheckScalarUnary("quarter", time_points, quarter);
+    CheckScalarUnary("day_of_week", time_points, day_of_week);
+    CheckScalarUnary("hour", time_points, hour);
+    CheckScalarUnary("minute", time_points, minute);
+    CheckScalarUnary("second", time_points, second);
+    CheckScalarUnary("millisecond", time_points, millisecond);
+    CheckScalarUnary("microsecond", time_points, microsecond);
+    CheckScalarUnary("nanosecond", time_points, nanosecond);
+  }
+
+  std::string in = "[123, 999, 1, 31231000]";
+  auto out = ArrayFromJSON(int64(), "[123, 999, 1, 0]");
+
+  auto tp_milli = ArrayFromJSON(timestamp(TimeUnit::MILLI), in);
+  auto tp_milli_zoned = ArrayFromJSON(timestamp(TimeUnit::MILLI, "Etc/GMT+2"), in);
+  CheckScalarUnary("millisecond", tp_milli, out);
+  CheckScalarUnary("millisecond", tp_milli, out);
+
+  auto tp_micro = ArrayFromJSON(timestamp(TimeUnit::MICRO), in);
+  auto tp_micro_zoned = ArrayFromJSON(timestamp(TimeUnit::MICRO, "Etc/GMT+2"), in);
+  CheckScalarUnary("microsecond", tp_micro, out);
+  CheckScalarUnary("microsecond", tp_micro_zoned, out);
+
+  auto tp_nano = ArrayFromJSON(timestamp(TimeUnit::NANO), in);
+  auto tp_nano_zoned = ArrayFromJSON(timestamp(TimeUnit::NANO, "Etc/GMT+2"), in);
+  CheckScalarUnary("nanosecond", tp_nano, out);
+  CheckScalarUnary("nanosecond", tp_nano_zoned, out);
+}
+
+TEST(ScalarTemporalTest, TestSimpleZonedTemporalComponentExtraction) {
+  const char* json =
+      R"(["1970-01-01T00:00:59","2000-02-29T23:23:23",
+          "3989-07-14T18:04:01","1900-01-01T01:59:20","2033-05-18T03:33:20"])";
+  auto time_points = ArrayFromJSON(timestamp(TimeUnit::SECOND, "Etc/GMT+2"), json);
+
+  auto year = ArrayFromJSON(int64(), "[1969, 2000, 3989, 1899, 2033]");

Review comment:
       Hmm... can you explain the reasoning here? If the timestamp is situated in 1970, I would expect the year to be 1970.




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

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



[GitHub] [arrow] rok commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   It seems tests on windows are not able to download IANA timezone database or access it post-download (https://howardhinnant.github.io/date/tz.html#Installation). I'll solve it but meanwhile I I'd like to ask for review.
   
   Open questions I have at the moment:
   - Is it possible to match kernels to zoned timestamps when registering or do we have to keep matching at runtime [like so](https://github.com/apache/arrow/blob/2765cefc737ac8b5c364a20f8ff94fc158b91b37/cpp/src/arrow/compute/kernels/scalar_temporal.cc#L58).
   - What would be a good array type for isodate?


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

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



[GitHub] [arrow] rok edited a comment on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   CI issues don't seem related.
   I've also created a [followup Jira](https://issues.apache.org/jira/browse/ARROW-12980) and a [PR](https://github.com/apache/arrow/pull/10457) for the timezone aware component extraction.


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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -450,5 +450,145 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Week returns week of year value for each element of `values`
+///
+/// \param[in] values input to extract week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Week(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns quarter of year value for each element of `values`
+///
+/// \param[in] values input to extract quarter of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Quarter(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Hour returns hour value for each element of `values`
+///
+/// \param[in] values input to extract hour from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Hour(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Minute returns minutes value for each element of `values`
+///
+/// \param[in] values input to extract minutes from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Minute(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Second returns seconds value for each element of `values`
+///
+/// \param[in] values input to extract seconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Second(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Millisecond returns milliseconds value for each element of `values`
+///
+/// \param[in] values input to extract milliseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Millisecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Microsecond returns microseconds value for each element of `values`
+///
+/// \param[in] values input to extract microseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Microsecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Nanosecond returns nanoseconds value for each element of `values`
+///
+/// \param[in] values input to extract nanoseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Nanosecond(const Datum& values, ExecContext* ctx = NULLPTR);

Review comment:
       How about for `2012-03-26 01:02:03.123456789`:
   * subsecond -> 123456789
   * second[int] -> 3
   * second[float] -> 3.123456789
   * microsecond -> 123
   * millisecond -> 456
   * nanosecond -> 789
   
   This way we have minimal "postprocessing" in languages other than c++




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,632 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::local_days;
+using arrow_vendored::date::locate_zone;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+template <typename Duration>
+inline year_month_day ymd_caster_template(const int64_t data) {

Review comment:
       Removed functions so this is probably fixed 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.

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,348 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+        .iso_encoding();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return (sd - sys_days(year_month_day(sd).year() / jan / 0)).count();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct Week {

Review comment:
       I'll do that.
   By the way what would be a good data type for "isocalendar" output? It's a [tuple in Pandas](https://pandas.pydata.org/docs/reference/api/pandas.Timestamp.isocalendar.html).




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,107 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestSimpleTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO);
+  auto timestamps = ArrayFromJSON(unit, times);
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("weekday", int64())});
+
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar = ArrayFromJSON(iso_calendar_type,
+                                    R"([{"iso_year": 1970, "iso_week": 1, "weekday": 4},
+                        {"iso_year": 2000, "iso_week": 9, "weekday": 2},
+                        {"iso_year": 1899, "iso_week": 52, "weekday": 7},
+                        {"iso_year": 2033, "iso_week": 20, "weekday": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59.123456789, 23.999999999, 20.001001001, 20.0, null]";
+  auto millisecond = "[123, 999, 1, 0, null]";
+  auto microsecond = "[456, 999, 1, 0, null]";
+  auto nanosecond = "[789, 999, 1, 0, null]";
+  auto subsecond = "[123456789, 999999999, 1001001, 0, null]";
+
+  CheckScalarUnary("year", unit, times, int64(), year);
+  CheckScalarUnary("month", unit, times, int64(), month);
+  CheckScalarUnary("day", unit, times, int64(), day);
+  CheckScalarUnary("day_of_week", unit, times, int64(), day_of_week);
+  CheckScalarUnary("day_of_year", unit, times, int64(), day_of_year);
+  CheckScalarUnary("iso_year", unit, times, int64(), iso_year);
+  CheckScalarUnary("iso_week", unit, times, int64(), iso_week);
+  CheckScalarUnary("iso_calendar", timestamps, iso_calendar);
+  CheckScalarUnary("quarter", unit, times, int64(), quarter);
+  CheckScalarUnary("hour", unit, times, int64(), hour);
+  CheckScalarUnary("minute", unit, times, int64(), minute);
+  CheckScalarUnary("second", unit, times, float64(), second);
+  CheckScalarUnary("millisecond", unit, times, int64(), millisecond);
+  CheckScalarUnary("microsecond", unit, times, int64(), microsecond);
+  CheckScalarUnary("nanosecond", unit, times, int64(), nanosecond);
+  CheckScalarUnary("subsecond", unit, times, int64(), subsecond);
+}
+
+TEST(ScalarTemporalTest, TestZonedTemporalComponentExtraction) {
+  std::string timezone = "Etc/UTC-2";
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO, timezone);
+  auto timestamps = ArrayFromJSON(unit, times);
+
+  ASSERT_RAISES(Invalid, Year(timestamps));
+  ASSERT_RAISES(Invalid, Month(timestamps));
+  ASSERT_RAISES(Invalid, Day(timestamps));
+  ASSERT_RAISES(Invalid, DayOfWeek(timestamps));
+  ASSERT_RAISES(Invalid, DayOfYear(timestamps));
+  ASSERT_RAISES(Invalid, ISOYear(timestamps));
+  ASSERT_RAISES(Invalid, ISOWeek(timestamps));
+  ASSERT_RAISES(Invalid, ISOCalendar(timestamps));
+  ASSERT_RAISES(Invalid, Quarter(timestamps));
+  ASSERT_RAISES(Invalid, Hour(timestamps));
+  ASSERT_RAISES(Invalid, Minute(timestamps));
+  ASSERT_RAISES(Invalid, Second(timestamps));
+  ASSERT_RAISES(Invalid, Millisecond(timestamps));
+  ASSERT_RAISES(Invalid, Microsecond(timestamps));
+  ASSERT_RAISES(Invalid, Nanosecond(timestamps));
+  ASSERT_RAISES(Invalid, Subsecond(timestamps));

Review comment:
       Let's leave it as is then. I'll open a followup JIRA to add timezone processing.




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

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -450,5 +450,145 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Week returns week of year value for each element of `values`
+///
+/// \param[in] values input to extract week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Week(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns quarter of year value for each element of `values`
+///
+/// \param[in] values input to extract quarter of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Quarter(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Hour returns hour value for each element of `values`
+///
+/// \param[in] values input to extract hour from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Hour(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Minute returns minutes value for each element of `values`
+///
+/// \param[in] values input to extract minutes from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Minute(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Second returns seconds value for each element of `values`
+///
+/// \param[in] values input to extract seconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Second(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Millisecond returns milliseconds value for each element of `values`
+///
+/// \param[in] values input to extract milliseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Millisecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Microsecond returns microseconds value for each element of `values`
+///
+/// \param[in] values input to extract microseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Microsecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Nanosecond returns nanoseconds value for each element of `values`
+///
+/// \param[in] values input to extract nanoseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Nanosecond(const Datum& values, ExecContext* ctx = NULLPTR);

Review comment:
       We don't need to match exactly what pandas does to be clear. pyarrow.compute should just follow Arrow's behaviour.
   
   So for Arrow the question is, what is most useful: the total units vs the component? (or do we need both?)
   
   As comparison, the R lubridate package only has component getter functions up to `second()`, and not for `milli/micro/..second`, but the `second` function returns the fractional part of the second as well:
   
   ```R
   > library(lubridate)
   > x <- ymd_hms("2012-03-26 01:02:03.123456")
   > second(x)
   [1] 3.123456
   ```
   
   
   




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return static_cast<T>((sd - sys_days(year_month_day(sd).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day{sys_days{floor<days>(Duration{arg})} + days{3}}.year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto dp = sys_days{floor<days>(Duration{arg})};
+    auto y = year_month_day{dp + days{3}}.year();
+    auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (dp < start) {
+      --y;
+      start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(dp - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        std::chrono::duration<double>(t - floor<std::chrono::minutes>(t)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::seconds>(t)) /
+                          std::chrono::nanoseconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+template <typename Duration, typename OutType>
+struct ISOCalendar {
+  using T = typename OutType::c_type;
+
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    using ScalarType = typename TypeTraits<OutType>::ScalarType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    if (!in.is_valid) {
+      out->is_valid = false;
+    } else {
+      const std::shared_ptr<DataType> iso_calendar_type =
+          struct_({field("iso_year", out_type), field("iso_week", out_type),
+                   field("weekday", out_type)});
+
+      const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+      const auto dp = sys_days{floor<days>(Duration{in_val})};
+      const auto ymd = year_month_day(dp);
+      auto y = year_month_day{dp + days{3}}.year();
+      auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      if (dp < start) {
+        --y;
+        start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      }
+
+      std::vector<std::shared_ptr<Scalar>> values = {
+          std::make_shared<ScalarType>(static_cast<T>(static_cast<int32_t>(ymd.year()))),
+          std::make_shared<ScalarType>(
+              static_cast<T>(trunc<weeks>(dp - start).count() + 1)),
+          std::make_shared<ScalarType>(static_cast<T>(weekday(ymd).iso_encoding()))};
+      *checked_cast<StructScalar*>(out) = StructScalar(values, iso_calendar_type);
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    using BuilderType = typename TypeTraits<OutType>::BuilderType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    const std::shared_ptr<DataType> iso_calendar_type =
+        struct_({field("iso_year", out_type), field("iso_week", out_type),
+                 field("weekday", out_type)});
+
+    std::unique_ptr<ArrayBuilder> array_builder;
+    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), iso_calendar_type, &array_builder));
+    StructBuilder* struct_builder = checked_cast<StructBuilder*>(array_builder.get());
+
+    std::vector<BuilderType*> field_builders;
+    field_builders.reserve(3);
+    for (int i = 0; i < 3; i++) {
+      field_builders.push_back(
+          checked_cast<BuilderType*>(struct_builder->field_builder(i)));
+    }
+    auto visit_null = [&]() { return struct_builder->AppendNull(); };
+    auto visit_value = [&](int64_t arg) {
+      const auto dp = sys_days{floor<days>(Duration{arg})};
+      const auto ymd = year_month_day(dp);
+      auto y = year_month_day{dp + days{3}}.year();
+      auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      if (dp < start) {
+        --y;
+        start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      }
+
+      RETURN_NOT_OK(
+          field_builders[0]->Append(static_cast<T>(static_cast<int32_t>(ymd.year()))));
+      RETURN_NOT_OK(field_builders[1]->Append(
+          static_cast<T>(trunc<weeks>(dp - start).count() + 1)));
+      RETURN_NOT_OK(
+          field_builders[2]->Append(static_cast<T>(weekday(ymd).iso_encoding())));
+
+      return struct_builder->Append();
+    };
+    RETURN_NOT_OK(VisitArrayDataInline<OutType>(in, visit_value, visit_null));
+
+    std::shared_ptr<Array> out_array;
+    RETURN_NOT_OK(struct_builder->Finish(&out_array));
+    *out = *std::move(out_array->data());
+
+    return Status::OK();
+  }
+};
+
+// Generate a kernel given an arithmetic functor
+template <template <typename... Args> class KernelGenerator,
+          template <typename... Args> class Op, typename Duration>
+ArrayKernelExec ExecFromOp(detail::GetTypeId get_id) {
+  switch (get_id.id) {
+    case Type::INT8:
+      return KernelGenerator<Int8Type, Op<Duration>>::Exec;
+    case Type::UINT8:
+      return KernelGenerator<UInt8Type, Op<Duration>>::Exec;
+    case Type::INT16:
+      return KernelGenerator<Int16Type, Op<Duration>>::Exec;
+    case Type::UINT16:
+      return KernelGenerator<UInt16Type, Op<Duration>>::Exec;
+    case Type::INT32:
+      return KernelGenerator<Int32Type, Op<Duration>>::Exec;
+    case Type::UINT32:
+      return KernelGenerator<UInt32Type, Op<Duration>>::Exec;
+    case Type::INT64:
+    case Type::TIMESTAMP:
+      return KernelGenerator<Int64Type, Op<Duration>>::Exec;
+    case Type::UINT64:
+      return KernelGenerator<UInt64Type, Op<Duration>>::Exec;
+    case Type::FLOAT:
+      return KernelGenerator<FloatType, Op<Duration>>::Exec;
+    case Type::DOUBLE:
+      return KernelGenerator<DoubleType, Op<Duration>>::Exec;
+    default:
+      DCHECK(false);
+      return ExecFail;
+  }
+}
+
+template <template <typename...> class Op>
+std::shared_ptr<ScalarFunction> MakeTemporalFunction(
+    std::string name, const FunctionDoc* doc,
+    std::vector<std::shared_ptr<DataType>> types) {
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+
+  for (auto ty : types) {
+    for (auto unit : AllTimeUnits()) {
+      ArrayKernelExec exec;
+      switch (unit) {
+        case TimeUnit::SECOND: {
+          exec = ExecFromOp<ScalarUnaryTemporal, Op, std::chrono::seconds>(ty);
+        }
+        case TimeUnit::MILLI: {
+          exec = ExecFromOp<ScalarUnaryTemporal, Op, std::chrono::milliseconds>(ty);
+        }
+        case TimeUnit::MICRO: {
+          exec = ExecFromOp<ScalarUnaryTemporal, Op, std::chrono::microseconds>(ty);
+        }
+        case TimeUnit::NANO: {
+          exec = ExecFromOp<ScalarUnaryTemporal, Op, std::chrono::nanoseconds>(ty);
+        }
+      }
+      ScalarKernel kernel =
+          ScalarKernel({match::TimestampTypeUnit(unit)}, OutputType(ty), exec);
+      DCHECK_OK(func->AddKernel(kernel));
+    }
+  }
+  return func;
+}
+
+// Generate a kernel given an arithmetic functor
+template <template <typename... Args> class Op, typename Duration>
+ArrayKernelExec SimpleUnaryFromOp(detail::GetTypeId get_id) {
+  switch (get_id.id) {
+    case Type::INT32:
+      return SimpleUnary<Op<Duration, Int32Type>>;
+    case Type::INT64:
+      return SimpleUnary<Op<Duration, Int64Type>>;
+    default:
+      DCHECK(false);
+      return ExecFail;
+  }
+}
+
+template <template <typename...> class Op>
+std::shared_ptr<ScalarFunction> MakeSimpleUnaryTemporalFunction(
+    std::string name, const FunctionDoc* doc,
+    std::vector<std::shared_ptr<DataType>> types) {
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+
+  for (auto ty : types) {
+    for (auto unit : AllTimeUnits()) {
+      ArrayKernelExec exec;
+      switch (unit) {
+        case TimeUnit::SECOND: {
+          exec = SimpleUnaryFromOp<Op, std::chrono::seconds>(ty);
+        }
+        case TimeUnit::MILLI: {
+          exec = SimpleUnaryFromOp<Op, std::chrono::milliseconds>(ty);
+        }
+        case TimeUnit::MICRO: {
+          exec = SimpleUnaryFromOp<Op, std::chrono::microseconds>(ty);
+        }
+        case TimeUnit::NANO: {
+          exec = SimpleUnaryFromOp<Op, std::chrono::nanoseconds>(ty);
+        }
+      }
+      auto output_type =
+          struct_({field("iso_year", ty), field("iso_week", ty), field("weekday", ty)});
+      ScalarKernel kernel =
+          ScalarKernel({match::TimestampTypeUnit(unit)}, OutputType(output_type), exec);
+      DCHECK_OK(func->AddKernel(kernel));
+    }
+  }
+  return func;
+}
+
+const FunctionDoc year_doc{"Extract year values", "", {"values"}};
+const FunctionDoc month_doc{"Extract month values", "", {"values"}};
+const FunctionDoc day_doc{"Extract day values", "", {"values"}};
+const FunctionDoc day_of_week_doc{"Extract day of week values", "", {"values"}};
+const FunctionDoc day_of_year_doc{"Extract day of year values", "", {"values"}};
+const FunctionDoc iso_year_doc{"Extract ISO year values", "", {"values"}};
+const FunctionDoc iso_week_doc{"Extract ISO week values", "", {"values"}};
+const FunctionDoc iso_calendar_doc{"Extract ISO calendar values", "", {"values"}};
+const FunctionDoc quarter_doc{"Extract quarter values", "", {"values"}};
+const FunctionDoc hour_doc{"Extract hour values", "", {"values"}};
+const FunctionDoc minute_doc{"Extract minute values", "", {"values"}};
+const FunctionDoc second_doc{"Extract second values", "", {"values"}};
+const FunctionDoc millisecond_doc{"Extract millisecond values", "", {"values"}};
+const FunctionDoc microsecond_doc{"Extract microsecond values", "", {"values"}};
+const FunctionDoc nanosecond_doc{"Extract nanosecond values", "", {"values"}};
+const FunctionDoc subsecond_doc{"Extract subsecond values", "", {"values"}};

Review comment:
       Done.
   I feel very awkward about my English right 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.

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



[GitHub] [arrow] pitrou commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return static_cast<T>((sd - sys_days(year_month_day(sd).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day{sys_days{floor<days>(Duration{arg})} + days{3}}.year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto dp = sys_days{floor<days>(Duration{arg})};
+    auto y = year_month_day{dp + days{3}}.year();
+    auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (dp < start) {
+      --y;
+      start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(dp - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        std::chrono::duration<double>(t - floor<std::chrono::minutes>(t)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::seconds>(t)) /
+                          std::chrono::nanoseconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+template <typename Duration, typename OutType>
+struct ISOCalendar {
+  using T = typename OutType::c_type;
+
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    using ScalarType = typename TypeTraits<OutType>::ScalarType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    if (!in.is_valid) {
+      out->is_valid = false;
+    } else {
+      const std::shared_ptr<DataType> iso_calendar_type =
+          struct_({field("iso_year", out_type), field("iso_week", out_type),
+                   field("weekday", out_type)});
+
+      const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+      const auto dp = sys_days{floor<days>(Duration{in_val})};
+      const auto ymd = year_month_day(dp);
+      auto y = year_month_day{dp + days{3}}.year();
+      auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      if (dp < start) {
+        --y;
+        start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      }
+
+      std::vector<std::shared_ptr<Scalar>> values = {
+          std::make_shared<ScalarType>(static_cast<T>(static_cast<int32_t>(ymd.year()))),
+          std::make_shared<ScalarType>(
+              static_cast<T>(trunc<weeks>(dp - start).count() + 1)),
+          std::make_shared<ScalarType>(static_cast<T>(weekday(ymd).iso_encoding()))};
+      *checked_cast<StructScalar*>(out) = StructScalar(values, iso_calendar_type);
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    using BuilderType = typename TypeTraits<OutType>::BuilderType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    const std::shared_ptr<DataType> iso_calendar_type =
+        struct_({field("iso_year", out_type), field("iso_week", out_type),
+                 field("weekday", out_type)});
+
+    std::unique_ptr<ArrayBuilder> array_builder;
+    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), iso_calendar_type, &array_builder));
+    StructBuilder* struct_builder = checked_cast<StructBuilder*>(array_builder.get());
+
+    std::vector<BuilderType*> field_builders;
+    field_builders.reserve(3);
+    for (int i = 0; i < 3; i++) {
+      field_builders.push_back(
+          checked_cast<BuilderType*>(struct_builder->field_builder(i)));
+    }
+    auto visit_null = [&]() { return struct_builder->AppendNull(); };
+    auto visit_value = [&](int64_t arg) {
+      const auto dp = sys_days{floor<days>(Duration{arg})};
+      const auto ymd = year_month_day(dp);
+      auto y = year_month_day{dp + days{3}}.year();
+      auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      if (dp < start) {
+        --y;
+        start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      }
+
+      RETURN_NOT_OK(
+          field_builders[0]->Append(static_cast<T>(static_cast<int32_t>(ymd.year()))));
+      RETURN_NOT_OK(field_builders[1]->Append(
+          static_cast<T>(trunc<weeks>(dp - start).count() + 1)));
+      RETURN_NOT_OK(
+          field_builders[2]->Append(static_cast<T>(weekday(ymd).iso_encoding())));
+
+      return struct_builder->Append();
+    };
+    RETURN_NOT_OK(VisitArrayDataInline<OutType>(in, visit_value, visit_null));
+
+    std::shared_ptr<Array> out_array;
+    RETURN_NOT_OK(struct_builder->Finish(&out_array));
+    *out = *std::move(out_array->data());
+
+    return Status::OK();
+  }
+};
+
+// Generate a kernel given an arithmetic functor
+template <template <typename... Args> class KernelGenerator,
+          template <typename... Args> class Op, typename Duration>
+ArrayKernelExec ExecFromOp(detail::GetTypeId get_id) {
+  switch (get_id.id) {

Review comment:
       I don't think there's a point in trying to control the output type. Also this will generate a lot of additional code.




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return static_cast<T>((sd - sys_days(year_month_day(sd).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day{sys_days{floor<days>(Duration{arg})} + days{3}}.year()));

Review comment:
       Added comment above.




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

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



[GitHub] [arrow] jorisvandenbossche commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   > (Sorry for amend commits).
   
   Don't be sorry for amending commits. Personally that's actually my strong preference that you amend commits during the review process, then we can see what changed since a previous 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.

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,151 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO);
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("day_of_week", int64())});
+
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar =
+      ArrayFromJSON(iso_calendar_type,
+                    R"([{"iso_year": 1970, "iso_week": 1, "day_of_week": 4},
+                        {"iso_year": 2000, "iso_week": 9, "day_of_week": 2},
+                        {"iso_year": 1899, "iso_week": 52, "day_of_week": 7},
+                        {"iso_year": 2033, "iso_week": 20, "day_of_week": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59.123456789, 23.999999999, 20.001001001, 20.0, null]";
+  auto millisecond = "[123, 999, 1, 0, null]";
+  auto microsecond = "[456, 999, 1, 0, null]";
+  auto nanosecond = "[789, 999, 1, 0, null]";
+  auto subsecond = "[123456789, 999999999, 1001001, 0, null]";
+
+  CheckScalarUnary("year", unit, times, int64(), year);
+  CheckScalarUnary("month", unit, times, int64(), month);
+  CheckScalarUnary("day", unit, times, int64(), day);
+  CheckScalarUnary("day_of_week", unit, times, int64(), day_of_week);
+  CheckScalarUnary("day_of_year", unit, times, int64(), day_of_year);
+  CheckScalarUnary("iso_year", unit, times, int64(), iso_year);
+  CheckScalarUnary("iso_week", unit, times, int64(), iso_week);
+  CheckScalarUnary("iso_calendar", ArrayFromJSON(unit, times), iso_calendar);
+  CheckScalarUnary("quarter", unit, times, int64(), quarter);
+  CheckScalarUnary("hour", unit, times, int64(), hour);
+  CheckScalarUnary("minute", unit, times, int64(), minute);
+  CheckScalarUnary("second", unit, times, float64(), second);
+  CheckScalarUnary("millisecond", unit, times, int64(), millisecond);
+  CheckScalarUnary("microsecond", unit, times, int64(), microsecond);
+  CheckScalarUnary("nanosecond", unit, times, int64(), nanosecond);
+  CheckScalarUnary("subsecond", unit, times, int64(), subsecond);
+}
+
+TEST(ScalarTemporalTest, TestTemporalComponentExtractionWithDifferentUnits) {
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("day_of_week", int64())});
+  const char* times =
+      R"(["1970-01-01T00:00:59","2000-02-29T23:23:23",
+            "1899-01-01T00:59:20","2033-05-18T03:33:20", null])";
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar =
+      ArrayFromJSON(iso_calendar_type,
+                    R"([{"iso_year": 1970, "iso_week": 1, "day_of_week": 4},
+                          {"iso_year": 2000, "iso_week": 9, "day_of_week": 2},
+                          {"iso_year": 1899, "iso_week": 52, "day_of_week": 7},
+                          {"iso_year": 2033, "iso_week": 20, "day_of_week": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59, 23, 20, 20, null]";

Review comment:
       The original timestamps don't need to be more detailed (to start with), you can leave those as is. But then I would still expect the subcomponents to work and return 0 ?




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

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



[GitHub] [arrow] rok commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   Missing features at moment:
   * isocalendar
   * millisecond
   * microsecond
   * nanosecond
   
   Different input / output types should also be handled.
   Does this need to be timezone aware?


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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,632 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::local_days;
+using arrow_vendored::date::locate_zone;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+template <typename Duration>
+inline year_month_day ymd_caster_template(const int64_t data) {
+  return year_month_day(floor<days>(sys_time<Duration>(Duration{data})));
+}
+
+template <typename Duration>
+inline std::function<year_month_day(const int64_t)> ymd_caster_zoned_template(
+    const std::string timezone) {
+  static const arrow_vendored::date::time_zone* tz = locate_zone(timezone);
+  return [](const int64_t data) {
+    return year_month_day(floor<days>(tz->to_local(sys_time<Duration>(Duration{data}))));
+  };
+}
+
+inline std::function<year_month_day(const int64_t)> make_ymd_caster(
+    const std::shared_ptr<DataType> type) {
+  const auto ts_type = std::static_pointer_cast<const TimestampType>(type);
+  const TimeUnit::type unit = ts_type->unit();
+  const std::string timezone = ts_type->timezone();
+
+  if (timezone.empty()) {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return ymd_caster_template<std::chrono::seconds>;
+      case TimeUnit::MILLI:
+        return ymd_caster_template<std::chrono::milliseconds>;
+      case TimeUnit::MICRO:
+        return ymd_caster_template<std::chrono::microseconds>;
+      case TimeUnit::NANO:
+        return ymd_caster_template<std::chrono::nanoseconds>;
+    }
+  } else {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return ymd_caster_zoned_template<std::chrono::seconds>(timezone);
+      case TimeUnit::MILLI:
+        return ymd_caster_zoned_template<std::chrono::milliseconds>(timezone);
+      case TimeUnit::MICRO:
+        return ymd_caster_zoned_template<std::chrono::microseconds>(timezone);
+      case TimeUnit::NANO:
+        return ymd_caster_zoned_template<std::chrono::nanoseconds>(timezone);
+    }
+  }
+  return ymd_caster_template<std::chrono::seconds>;
+}
+
+template <typename DurationIn, typename DurationOut>
+inline hh_mm_ss<DurationOut> hhmmss_caster_template(const int64_t data) {
+  DurationIn t = DurationIn{data};
+  return hh_mm_ss<DurationOut>(
+      std::chrono::duration_cast<DurationOut>(t - floor<days>(t)));
+}
+
+template <typename DurationIn, typename DurationOut>
+inline std::function<hh_mm_ss<DurationOut>(const int64_t)> hhmmss_caster_zoned_template(
+    const std::string timezone) {
+  static const arrow_vendored::date::time_zone* tz = locate_zone(timezone);
+  return [](const int64_t data) {
+    const auto z = sys_time<DurationIn>(DurationIn{data});
+    const auto l = make_zoned(tz, z).get_local_time();
+    return hh_mm_ss<DurationOut>(
+        std::chrono::duration_cast<DurationOut>(l - floor<days>(l)));
+  };
+}
+
+template <typename Duration>
+inline std::function<hh_mm_ss<Duration>(const int64_t)> make_hhmmss_caster(
+    const std::shared_ptr<DataType> type) {
+  const auto ts_type = std::static_pointer_cast<const TimestampType>(type);
+  const TimeUnit::type unit = ts_type->unit();
+  const std::string timezone = ts_type->timezone();
+
+  if (timezone.empty()) {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return hhmmss_caster_template<std::chrono::seconds, Duration>;
+      case TimeUnit::MILLI:
+        return hhmmss_caster_template<std::chrono::milliseconds, Duration>;
+      case TimeUnit::MICRO:
+        return hhmmss_caster_template<std::chrono::microseconds, Duration>;
+      case TimeUnit::NANO:
+        return hhmmss_caster_template<std::chrono::nanoseconds, Duration>;
+    }
+  } else {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return hhmmss_caster_zoned_template<std::chrono::seconds, Duration>(timezone);
+      case TimeUnit::MILLI:
+        return hhmmss_caster_zoned_template<std::chrono::milliseconds, Duration>(
+            timezone);
+      case TimeUnit::MICRO:
+        return hhmmss_caster_zoned_template<std::chrono::microseconds, Duration>(
+            timezone);
+      case TimeUnit::NANO:
+        return hhmmss_caster_zoned_template<std::chrono::nanoseconds, Duration>(timezone);
+    }
+  }
+  return hhmmss_caster_template<std::chrono::seconds, Duration>;
+}
+
+template <typename Duration>
+inline unsigned day_of_year_caster_template(const int64_t data) {
+  const auto sd = sys_days{floor<days>(Duration{data})};
+  const auto y = year_month_day(sd).year();
+  return static_cast<unsigned>((sd - sys_days(y / jan / 0)).count());
+}
+
+template <typename Duration>
+inline std::function<unsigned(const int64_t)> day_of_year_zoned_caster_template(
+    const std::string timezone) {
+  static const arrow_vendored::date::time_zone* tz = locate_zone(timezone);
+  return [](const int64_t data) {
+    auto ld =
+        year_month_day(floor<days>(tz->to_local(sys_time<Duration>(Duration{data}))));
+    return static_cast<unsigned>(
+        (local_days(ld) - local_days(ld.year() / jan / 1) + days{1}).count());
+  };
+}
+
+inline std::function<unsigned(const int64_t)> get_day_of_year_caster(
+    const std::shared_ptr<DataType> type) {
+  const auto ts_type = std::static_pointer_cast<const TimestampType>(type);
+  const TimeUnit::type unit = ts_type->unit();
+  const std::string timezone = ts_type->timezone();
+
+  if (timezone.empty()) {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return day_of_year_caster_template<std::chrono::seconds>;
+      case TimeUnit::MILLI:
+        return day_of_year_caster_template<std::chrono::milliseconds>;
+      case TimeUnit::MICRO:
+        return day_of_year_caster_template<std::chrono::microseconds>;
+      case TimeUnit::NANO:
+        return day_of_year_caster_template<std::chrono::nanoseconds>;
+    }
+  } else {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return day_of_year_zoned_caster_template<std::chrono::seconds>(timezone);
+      case TimeUnit::MILLI:
+        return day_of_year_zoned_caster_template<std::chrono::milliseconds>(timezone);
+      case TimeUnit::MICRO:
+        return day_of_year_zoned_caster_template<std::chrono::microseconds>(timezone);
+      case TimeUnit::NANO:
+        return day_of_year_zoned_caster_template<std::chrono::nanoseconds>(timezone);
+    }
+  }
+  return day_of_year_caster_template<std::chrono::seconds>;
+}
+
+template <typename Duration>
+inline unsigned week_caster_template(const int64_t data) {
+  // Based on
+  // https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+  const auto dp = sys_days{floor<days>(Duration{data})};
+  auto y = year_month_day{dp + days{3}}.year();
+  auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+  if (dp < start) {
+    --y;
+    start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+  }
+  return static_cast<unsigned>(trunc<weeks>(dp - start).count() + 1);
+}
+
+template <typename Duration>
+inline std::function<unsigned(const int64_t)> week_zoned_caster_template(
+    const std::string timezone) {
+  static const arrow_vendored::date::time_zone* tz = locate_zone(timezone);
+  return [](const int64_t data) {
+    const auto ld = floor<days>(tz->to_local(sys_time<Duration>(Duration{data})));
+    auto y = year_month_day{ld + days{3}}.year();
+    auto start = local_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (ld < start) {
+      --y;
+      start = local_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<unsigned>(trunc<weeks>(local_days(ld) - start).count() + 1);
+  };
+}
+
+inline std::function<unsigned(const int64_t)> make_week_caster(
+    const std::shared_ptr<DataType> type) {
+  const auto ts_type = std::static_pointer_cast<const TimestampType>(type);
+  const TimeUnit::type unit = ts_type->unit();
+  const std::string timezone = ts_type->timezone();
+
+  if (timezone.empty()) {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return week_caster_template<std::chrono::seconds>;
+      case TimeUnit::MILLI:
+        return week_caster_template<std::chrono::milliseconds>;
+      case TimeUnit::MICRO:
+        return week_caster_template<std::chrono::microseconds>;
+      case TimeUnit::NANO:
+        return week_caster_template<std::chrono::nanoseconds>;
+    }
+  } else {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return week_zoned_caster_template<std::chrono::seconds>(timezone);
+      case TimeUnit::MILLI:
+        return week_zoned_caster_template<std::chrono::milliseconds>(timezone);
+      case TimeUnit::MICRO:
+        return week_zoned_caster_template<std::chrono::microseconds>(timezone);
+      case TimeUnit::NANO:
+        return week_zoned_caster_template<std::chrono::nanoseconds>(timezone);
+    }
+  }
+  return day_of_year_caster_template<std::chrono::seconds>;
+}
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename out_type>
+struct Year {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const auto& in_data = internal::UnboxScalar<const TimestampType>::Unbox(in);
+    auto ymd_caster = make_ymd_caster(in.type);
+    checked_cast<Int64Scalar*>(out)->value = static_cast<int>(ymd_caster(in_data).year());
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    auto in_data = in.GetValues<uint64_t>(1);
+    auto out_data = out->GetMutableValues<out_type>(1);
+    auto ymd_caster = make_ymd_caster(in.type);
+    for (int64_t i = 0; i < in.length; i++) {
+      out_data[i] = static_cast<int>(ymd_caster(in_data[i]).year());
+    }
+    return Status::OK();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename out_type>
+struct Month {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const auto& in_data = internal::UnboxScalar<const TimestampType>::Unbox(in);
+    auto ymd_caster = make_ymd_caster(in.type);
+    checked_cast<Int64Scalar*>(out)->value =
+        static_cast<unsigned>(ymd_caster(in_data).month());
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    auto in_data = in.GetValues<uint64_t>(1);
+    auto out_data = out->GetMutableValues<out_type>(1);
+    auto ymd_caster = make_ymd_caster(in.type);
+    for (int64_t i = 0; i < in.length; i++) {
+      out_data[i] = static_cast<unsigned>(ymd_caster(in_data[i]).month());
+    }
+    return Status::OK();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename out_type>
+struct Day {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const int64_t& in_data = internal::UnboxScalar<const TimestampType>::Unbox(in);
+    auto ymd_caster = make_ymd_caster(in.type);
+    checked_cast<Int64Scalar*>(out)->value =
+        static_cast<unsigned>(ymd_caster(in_data).day());
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    auto in_data = in.GetValues<uint64_t>(1);
+    auto out_data = out->GetMutableValues<out_type>(1);
+    auto ymd_caster = make_ymd_caster(in.type);
+    for (int64_t i = 0; i < in.length; i++) {
+      out_data[i] = static_cast<unsigned>(ymd_caster(in_data[i]).day());
+    }
+    return Status::OK();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract week from timestamp
+
+template <typename out_type>
+struct Week {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const auto& in_data = internal::UnboxScalar<const TimestampType>::Unbox(in);
+    auto week_caster = make_week_caster(in.type);
+    checked_cast<Int64Scalar*>(out)->value = week_caster(in_data);
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    auto in_data = in.GetValues<uint64_t>(1);
+    auto out_data = out->GetMutableValues<out_type>(1);
+    auto week_caster = make_week_caster(in.type);
+    for (int64_t i = 0; i < in.length; i++) {
+      out_data[i] = week_caster(in_data[i]);
+    }
+    return Status::OK();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract quarter from timestamp
+
+template <typename out_type>
+struct Quarter {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const auto& in_data = internal::UnboxScalar<const TimestampType>::Unbox(in);
+    auto ymd_caster = make_ymd_caster(in.type);
+    checked_cast<Int64Scalar*>(out)->value =
+        (static_cast<unsigned>(ymd_caster(in_data).month()) - 1) / 3 + 1;
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    auto in_data = in.GetValues<uint64_t>(1);
+    auto out_data = out->GetMutableValues<out_type>(1);
+    auto ymd_caster = make_ymd_caster(in.type);
+    for (int64_t i = 0; i < in.length; i++) {
+      out_data[i] = (static_cast<unsigned>(ymd_caster(in_data[i]).month()) - 1) / 3 + 1;
+    }
+    return Status::OK();
+  }
+};

Review comment:
       I've abstracted the "caster maker" function. I'll try to do the same with the generic kernel.




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,632 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::local_days;
+using arrow_vendored::date::locate_zone;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+template <typename Duration>
+inline year_month_day ymd_caster_template(const int64_t data) {
+  return year_month_day(floor<days>(sys_time<Duration>(Duration{data})));
+}
+
+template <typename Duration>
+inline std::function<year_month_day(const int64_t)> ymd_caster_zoned_template(
+    const std::string timezone) {
+  static const arrow_vendored::date::time_zone* tz = locate_zone(timezone);
+  return [](const int64_t data) {
+    return year_month_day(floor<days>(tz->to_local(sys_time<Duration>(Duration{data}))));
+  };
+}
+
+inline std::function<year_month_day(const int64_t)> make_ymd_caster(
+    const std::shared_ptr<DataType> type) {
+  const auto ts_type = std::static_pointer_cast<const TimestampType>(type);
+  const TimeUnit::type unit = ts_type->unit();
+  const std::string timezone = ts_type->timezone();
+
+  if (timezone.empty()) {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return ymd_caster_template<std::chrono::seconds>;
+      case TimeUnit::MILLI:
+        return ymd_caster_template<std::chrono::milliseconds>;
+      case TimeUnit::MICRO:
+        return ymd_caster_template<std::chrono::microseconds>;
+      case TimeUnit::NANO:
+        return ymd_caster_template<std::chrono::nanoseconds>;
+    }
+  } else {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return ymd_caster_zoned_template<std::chrono::seconds>(timezone);
+      case TimeUnit::MILLI:
+        return ymd_caster_zoned_template<std::chrono::milliseconds>(timezone);
+      case TimeUnit::MICRO:
+        return ymd_caster_zoned_template<std::chrono::microseconds>(timezone);
+      case TimeUnit::NANO:
+        return ymd_caster_zoned_template<std::chrono::nanoseconds>(timezone);
+    }
+  }
+  return ymd_caster_template<std::chrono::seconds>;
+}
+
+template <typename DurationIn, typename DurationOut>
+inline hh_mm_ss<DurationOut> hhmmss_caster_template(const int64_t data) {
+  DurationIn t = DurationIn{data};
+  return hh_mm_ss<DurationOut>(
+      std::chrono::duration_cast<DurationOut>(t - floor<days>(t)));
+}
+
+template <typename DurationIn, typename DurationOut>
+inline std::function<hh_mm_ss<DurationOut>(const int64_t)> hhmmss_caster_zoned_template(
+    const std::string timezone) {
+  static const arrow_vendored::date::time_zone* tz = locate_zone(timezone);
+  return [](const int64_t data) {
+    const auto z = sys_time<DurationIn>(DurationIn{data});
+    const auto l = make_zoned(tz, z).get_local_time();
+    return hh_mm_ss<DurationOut>(
+        std::chrono::duration_cast<DurationOut>(l - floor<days>(l)));
+  };
+}
+
+template <typename Duration>
+inline std::function<hh_mm_ss<Duration>(const int64_t)> make_hhmmss_caster(
+    const std::shared_ptr<DataType> type) {
+  const auto ts_type = std::static_pointer_cast<const TimestampType>(type);
+  const TimeUnit::type unit = ts_type->unit();
+  const std::string timezone = ts_type->timezone();
+
+  if (timezone.empty()) {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return hhmmss_caster_template<std::chrono::seconds, Duration>;
+      case TimeUnit::MILLI:
+        return hhmmss_caster_template<std::chrono::milliseconds, Duration>;
+      case TimeUnit::MICRO:
+        return hhmmss_caster_template<std::chrono::microseconds, Duration>;
+      case TimeUnit::NANO:
+        return hhmmss_caster_template<std::chrono::nanoseconds, Duration>;
+    }
+  } else {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return hhmmss_caster_zoned_template<std::chrono::seconds, Duration>(timezone);
+      case TimeUnit::MILLI:
+        return hhmmss_caster_zoned_template<std::chrono::milliseconds, Duration>(
+            timezone);
+      case TimeUnit::MICRO:
+        return hhmmss_caster_zoned_template<std::chrono::microseconds, Duration>(
+            timezone);
+      case TimeUnit::NANO:
+        return hhmmss_caster_zoned_template<std::chrono::nanoseconds, Duration>(timezone);
+    }
+  }
+  return hhmmss_caster_template<std::chrono::seconds, Duration>;
+}
+
+template <typename Duration>
+inline unsigned day_of_year_caster_template(const int64_t data) {
+  const auto sd = sys_days{floor<days>(Duration{data})};
+  const auto y = year_month_day(sd).year();
+  return static_cast<unsigned>((sd - sys_days(y / jan / 0)).count());
+}
+
+template <typename Duration>
+inline std::function<unsigned(const int64_t)> day_of_year_zoned_caster_template(
+    const std::string timezone) {
+  static const arrow_vendored::date::time_zone* tz = locate_zone(timezone);
+  return [](const int64_t data) {
+    auto ld =
+        year_month_day(floor<days>(tz->to_local(sys_time<Duration>(Duration{data}))));
+    return static_cast<unsigned>(
+        (local_days(ld) - local_days(ld.year() / jan / 1) + days{1}).count());
+  };
+}
+
+inline std::function<unsigned(const int64_t)> get_day_of_year_caster(
+    const std::shared_ptr<DataType> type) {
+  const auto ts_type = std::static_pointer_cast<const TimestampType>(type);
+  const TimeUnit::type unit = ts_type->unit();
+  const std::string timezone = ts_type->timezone();
+
+  if (timezone.empty()) {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return day_of_year_caster_template<std::chrono::seconds>;
+      case TimeUnit::MILLI:
+        return day_of_year_caster_template<std::chrono::milliseconds>;
+      case TimeUnit::MICRO:
+        return day_of_year_caster_template<std::chrono::microseconds>;
+      case TimeUnit::NANO:
+        return day_of_year_caster_template<std::chrono::nanoseconds>;
+    }
+  } else {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return day_of_year_zoned_caster_template<std::chrono::seconds>(timezone);
+      case TimeUnit::MILLI:
+        return day_of_year_zoned_caster_template<std::chrono::milliseconds>(timezone);
+      case TimeUnit::MICRO:
+        return day_of_year_zoned_caster_template<std::chrono::microseconds>(timezone);
+      case TimeUnit::NANO:
+        return day_of_year_zoned_caster_template<std::chrono::nanoseconds>(timezone);
+    }
+  }
+  return day_of_year_caster_template<std::chrono::seconds>;
+}
+
+template <typename Duration>
+inline unsigned week_caster_template(const int64_t data) {
+  // Based on
+  // https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+  const auto dp = sys_days{floor<days>(Duration{data})};
+  auto y = year_month_day{dp + days{3}}.year();
+  auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+  if (dp < start) {
+    --y;
+    start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+  }
+  return static_cast<unsigned>(trunc<weeks>(dp - start).count() + 1);
+}
+
+template <typename Duration>
+inline std::function<unsigned(const int64_t)> week_zoned_caster_template(
+    const std::string timezone) {
+  static const arrow_vendored::date::time_zone* tz = locate_zone(timezone);
+  return [](const int64_t data) {
+    const auto ld = floor<days>(tz->to_local(sys_time<Duration>(Duration{data})));
+    auto y = year_month_day{ld + days{3}}.year();
+    auto start = local_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (ld < start) {
+      --y;
+      start = local_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<unsigned>(trunc<weeks>(local_days(ld) - start).count() + 1);
+  };
+}
+
+inline std::function<unsigned(const int64_t)> make_week_caster(
+    const std::shared_ptr<DataType> type) {
+  const auto ts_type = std::static_pointer_cast<const TimestampType>(type);
+  const TimeUnit::type unit = ts_type->unit();
+  const std::string timezone = ts_type->timezone();
+
+  if (timezone.empty()) {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return week_caster_template<std::chrono::seconds>;
+      case TimeUnit::MILLI:
+        return week_caster_template<std::chrono::milliseconds>;
+      case TimeUnit::MICRO:
+        return week_caster_template<std::chrono::microseconds>;
+      case TimeUnit::NANO:
+        return week_caster_template<std::chrono::nanoseconds>;
+    }
+  } else {
+    switch (unit) {
+      case TimeUnit::SECOND:
+        return week_zoned_caster_template<std::chrono::seconds>(timezone);
+      case TimeUnit::MILLI:
+        return week_zoned_caster_template<std::chrono::milliseconds>(timezone);
+      case TimeUnit::MICRO:
+        return week_zoned_caster_template<std::chrono::microseconds>(timezone);
+      case TimeUnit::NANO:
+        return week_zoned_caster_template<std::chrono::nanoseconds>(timezone);
+    }
+  }
+  return day_of_year_caster_template<std::chrono::seconds>;
+}
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename out_type>
+struct Year {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const auto& in_data = internal::UnboxScalar<const TimestampType>::Unbox(in);
+    auto ymd_caster = make_ymd_caster(in.type);
+    checked_cast<Int64Scalar*>(out)->value = static_cast<int>(ymd_caster(in_data).year());
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    auto in_data = in.GetValues<uint64_t>(1);
+    auto out_data = out->GetMutableValues<out_type>(1);
+    auto ymd_caster = make_ymd_caster(in.type);
+    for (int64_t i = 0; i < in.length; i++) {
+      out_data[i] = static_cast<int>(ymd_caster(in_data[i]).year());
+    }
+    return Status::OK();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename out_type>
+struct Month {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const auto& in_data = internal::UnboxScalar<const TimestampType>::Unbox(in);
+    auto ymd_caster = make_ymd_caster(in.type);
+    checked_cast<Int64Scalar*>(out)->value =
+        static_cast<unsigned>(ymd_caster(in_data).month());
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    auto in_data = in.GetValues<uint64_t>(1);
+    auto out_data = out->GetMutableValues<out_type>(1);
+    auto ymd_caster = make_ymd_caster(in.type);
+    for (int64_t i = 0; i < in.length; i++) {
+      out_data[i] = static_cast<unsigned>(ymd_caster(in_data[i]).month());
+    }
+    return Status::OK();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename out_type>
+struct Day {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const int64_t& in_data = internal::UnboxScalar<const TimestampType>::Unbox(in);
+    auto ymd_caster = make_ymd_caster(in.type);
+    checked_cast<Int64Scalar*>(out)->value =
+        static_cast<unsigned>(ymd_caster(in_data).day());
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    auto in_data = in.GetValues<uint64_t>(1);
+    auto out_data = out->GetMutableValues<out_type>(1);
+    auto ymd_caster = make_ymd_caster(in.type);
+    for (int64_t i = 0; i < in.length; i++) {
+      out_data[i] = static_cast<unsigned>(ymd_caster(in_data[i]).day());
+    }
+    return Status::OK();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract week from timestamp
+
+template <typename out_type>
+struct Week {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const auto& in_data = internal::UnboxScalar<const TimestampType>::Unbox(in);
+    auto week_caster = make_week_caster(in.type);
+    checked_cast<Int64Scalar*>(out)->value = week_caster(in_data);
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    auto in_data = in.GetValues<uint64_t>(1);
+    auto out_data = out->GetMutableValues<out_type>(1);
+    auto week_caster = make_week_caster(in.type);
+    for (int64_t i = 0; i < in.length; i++) {
+      out_data[i] = week_caster(in_data[i]);
+    }
+    return Status::OK();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract quarter from timestamp
+
+template <typename out_type>
+struct Quarter {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    const auto& in_data = internal::UnboxScalar<const TimestampType>::Unbox(in);
+    auto ymd_caster = make_ymd_caster(in.type);
+    checked_cast<Int64Scalar*>(out)->value =
+        (static_cast<unsigned>(ymd_caster(in_data).month()) - 1) / 3 + 1;
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    auto in_data = in.GetValues<uint64_t>(1);
+    auto out_data = out->GetMutableValues<out_type>(1);
+    auto ymd_caster = make_ymd_caster(in.type);
+    for (int64_t i = 0; i < in.length; i++) {
+      out_data[i] = (static_cast<unsigned>(ymd_caster(in_data[i]).month()) - 1) / 3 + 1;
+    }
+    return Status::OK();
+  }
+};

Review comment:
       Indeed. If I "dry" this up do you feel the approach is ok?




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

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



[GitHub] [arrow] rok commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   Thanks @jorisvandenbossche @pitrou for the help and feedback! I'm very glad this is merged! :)


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

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



[GitHub] [arrow] jorisvandenbossche commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   I did a quick naive check of the performance of this branch, and comparing the simple components, they are consistently 6 to 25x faster compared to the pandas implementation (6x for the year/month/day, 25-30x for hour/minute/second). That aspect seems to be good! 
   (I don't know how good the pandas implementation is, so it might not be the most relevant comparison (comparing with eg clickhouse might be more interesting), but it at least says something)


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

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



[GitHub] [arrow] rok commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   > CI issues don't seem related.
   > I've also created a [followup Jira](https://issues.apache.org/jira/browse/ARROW-12980) for the timezone aware component extraction.
   
   


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

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



[GitHub] [arrow] jorisvandenbossche commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   > Does this need to be timezone aware?
   
   Unfortunately yes .. But since that's a lot more complicated, and I don't think we already have tz-related functionality, for now those kernels could maybe just raise an error if the timestamp type has a `tz` and is thus not tz-naive?
   


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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -450,5 +450,145 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Week returns week of year value for each element of `values`
+///
+/// \param[in] values input to extract week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Week(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns quarter of year value for each element of `values`
+///
+/// \param[in] values input to extract quarter of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Quarter(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Hour returns hour value for each element of `values`
+///
+/// \param[in] values input to extract hour from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Hour(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Minute returns minutes value for each element of `values`
+///
+/// \param[in] values input to extract minutes from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Minute(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Second returns seconds value for each element of `values`
+///
+/// \param[in] values input to extract seconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Second(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Millisecond returns milliseconds value for each element of `values`
+///
+/// \param[in] values input to extract milliseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Millisecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Microsecond returns microseconds value for each element of `values`
+///
+/// \param[in] values input to extract microseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Microsecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Nanosecond returns nanoseconds value for each element of `values`
+///
+/// \param[in] values input to extract nanoseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Nanosecond(const Datum& values, ExecContext* ctx = NULLPTR);

Review comment:
       How about for `2012-03-26 01:02:03.123456789`:
   * subsecond -> 123456789
   * second[int] -> 3
   * second[float] -> 3.123456789
   * microsecond -> 123
   * millisecond -> 456
   * nanosecond -> 789
   
   This way we have minimal `postprocessing` in languages other than c++




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: docs/source/cpp/compute.rst
##########
@@ -665,6 +665,57 @@ String component extraction
   e.g. 'letter' and 'digit' for the regular expression
   ``(?P<letter>[ab])(?P<digit>\\d)``.
 
+Temporal component extraction

Review comment:
       Moved. Please check if the location is ok.




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return static_cast<T>((sd - sys_days(year_month_day(sd).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day{sys_days{floor<days>(Duration{arg})} + days{3}}.year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto dp = sys_days{floor<days>(Duration{arg})};
+    auto y = year_month_day{dp + days{3}}.year();
+    auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (dp < start) {
+      --y;
+      start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(dp - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        std::chrono::duration<double>(t - floor<std::chrono::minutes>(t)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::seconds>(t)) /
+                          std::chrono::nanoseconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+template <typename Duration, typename OutType>
+struct ISOCalendar {
+  using T = typename OutType::c_type;
+
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    using ScalarType = typename TypeTraits<OutType>::ScalarType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    if (!in.is_valid) {
+      out->is_valid = false;
+    } else {
+      const std::shared_ptr<DataType> iso_calendar_type =
+          struct_({field("iso_year", out_type), field("iso_week", out_type),
+                   field("weekday", out_type)});
+
+      const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+      const auto dp = sys_days{floor<days>(Duration{in_val})};
+      const auto ymd = year_month_day(dp);
+      auto y = year_month_day{dp + days{3}}.year();
+      auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      if (dp < start) {
+        --y;
+        start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      }
+
+      std::vector<std::shared_ptr<Scalar>> values = {
+          std::make_shared<ScalarType>(static_cast<T>(static_cast<int32_t>(ymd.year()))),
+          std::make_shared<ScalarType>(
+              static_cast<T>(trunc<weeks>(dp - start).count() + 1)),
+          std::make_shared<ScalarType>(static_cast<T>(weekday(ymd).iso_encoding()))};
+      *checked_cast<StructScalar*>(out) = StructScalar(values, iso_calendar_type);
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    using BuilderType = typename TypeTraits<OutType>::BuilderType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    const std::shared_ptr<DataType> iso_calendar_type =
+        struct_({field("iso_year", out_type), field("iso_week", out_type),
+                 field("weekday", out_type)});
+
+    std::unique_ptr<ArrayBuilder> array_builder;
+    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), iso_calendar_type, &array_builder));
+    StructBuilder* struct_builder = checked_cast<StructBuilder*>(array_builder.get());
+
+    std::vector<BuilderType*> field_builders;
+    field_builders.reserve(3);
+    for (int i = 0; i < 3; i++) {
+      field_builders.push_back(
+          checked_cast<BuilderType*>(struct_builder->field_builder(i)));
+    }
+    auto visit_null = [&]() { return struct_builder->AppendNull(); };
+    auto visit_value = [&](int64_t arg) {
+      const auto dp = sys_days{floor<days>(Duration{arg})};
+      const auto ymd = year_month_day(dp);
+      auto y = year_month_day{dp + days{3}}.year();
+      auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      if (dp < start) {
+        --y;
+        start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      }
+
+      RETURN_NOT_OK(
+          field_builders[0]->Append(static_cast<T>(static_cast<int32_t>(ymd.year()))));
+      RETURN_NOT_OK(field_builders[1]->Append(
+          static_cast<T>(trunc<weeks>(dp - start).count() + 1)));
+      RETURN_NOT_OK(
+          field_builders[2]->Append(static_cast<T>(weekday(ymd).iso_encoding())));
+
+      return struct_builder->Append();
+    };
+    RETURN_NOT_OK(VisitArrayDataInline<OutType>(in, visit_value, visit_null));
+
+    std::shared_ptr<Array> out_array;
+    RETURN_NOT_OK(struct_builder->Finish(&out_array));
+    *out = *std::move(out_array->data());
+
+    return Status::OK();
+  }
+};
+
+// Generate a kernel given an arithmetic functor
+template <template <typename... Args> class KernelGenerator,
+          template <typename... Args> class Op, typename Duration>
+ArrayKernelExec ExecFromOp(detail::GetTypeId get_id) {
+  switch (get_id.id) {
+    case Type::INT8:
+      return KernelGenerator<Int8Type, Op<Duration>>::Exec;
+    case Type::UINT8:
+      return KernelGenerator<UInt8Type, Op<Duration>>::Exec;
+    case Type::INT16:
+      return KernelGenerator<Int16Type, Op<Duration>>::Exec;
+    case Type::UINT16:
+      return KernelGenerator<UInt16Type, Op<Duration>>::Exec;
+    case Type::INT32:
+      return KernelGenerator<Int32Type, Op<Duration>>::Exec;
+    case Type::UINT32:
+      return KernelGenerator<UInt32Type, Op<Duration>>::Exec;
+    case Type::INT64:
+    case Type::TIMESTAMP:
+      return KernelGenerator<Int64Type, Op<Duration>>::Exec;
+    case Type::UINT64:
+      return KernelGenerator<UInt64Type, Op<Duration>>::Exec;
+    case Type::FLOAT:
+      return KernelGenerator<FloatType, Op<Duration>>::Exec;
+    case Type::DOUBLE:
+      return KernelGenerator<DoubleType, Op<Duration>>::Exec;
+    default:
+      DCHECK(false);
+      return ExecFail;
+  }
+}
+
+template <template <typename...> class Op>
+std::shared_ptr<ScalarFunction> MakeTemporalFunction(
+    std::string name, const FunctionDoc* doc,
+    std::vector<std::shared_ptr<DataType>> types) {
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+
+  for (auto ty : types) {
+    for (auto unit : AllTimeUnits()) {
+      ArrayKernelExec exec;
+      switch (unit) {
+        case TimeUnit::SECOND: {
+          exec = ExecFromOp<ScalarUnaryTemporal, Op, std::chrono::seconds>(ty);
+        }
+        case TimeUnit::MILLI: {
+          exec = ExecFromOp<ScalarUnaryTemporal, Op, std::chrono::milliseconds>(ty);
+        }
+        case TimeUnit::MICRO: {
+          exec = ExecFromOp<ScalarUnaryTemporal, Op, std::chrono::microseconds>(ty);
+        }
+        case TimeUnit::NANO: {
+          exec = ExecFromOp<ScalarUnaryTemporal, Op, std::chrono::nanoseconds>(ty);
+        }
+      }
+      ScalarKernel kernel =
+          ScalarKernel({match::TimestampTypeUnit(unit)}, OutputType(ty), exec);
+      DCHECK_OK(func->AddKernel(kernel));
+    }
+  }
+  return func;
+}
+
+// Generate a kernel given an arithmetic functor
+template <template <typename... Args> class Op, typename Duration>
+ArrayKernelExec SimpleUnaryFromOp(detail::GetTypeId get_id) {
+  switch (get_id.id) {
+    case Type::INT32:
+      return SimpleUnary<Op<Duration, Int32Type>>;
+    case Type::INT64:
+      return SimpleUnary<Op<Duration, Int64Type>>;
+    default:
+      DCHECK(false);
+      return ExecFail;
+  }
+}
+
+template <template <typename...> class Op>
+std::shared_ptr<ScalarFunction> MakeSimpleUnaryTemporalFunction(
+    std::string name, const FunctionDoc* doc,
+    std::vector<std::shared_ptr<DataType>> types) {
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+
+  for (auto ty : types) {
+    for (auto unit : AllTimeUnits()) {
+      ArrayKernelExec exec;
+      switch (unit) {
+        case TimeUnit::SECOND: {
+          exec = SimpleUnaryFromOp<Op, std::chrono::seconds>(ty);
+        }
+        case TimeUnit::MILLI: {
+          exec = SimpleUnaryFromOp<Op, std::chrono::milliseconds>(ty);
+        }
+        case TimeUnit::MICRO: {
+          exec = SimpleUnaryFromOp<Op, std::chrono::microseconds>(ty);
+        }
+        case TimeUnit::NANO: {
+          exec = SimpleUnaryFromOp<Op, std::chrono::nanoseconds>(ty);
+        }
+      }
+      auto output_type =
+          struct_({field("iso_year", ty), field("iso_week", ty), field("weekday", ty)});
+      ScalarKernel kernel =
+          ScalarKernel({match::TimestampTypeUnit(unit)}, OutputType(output_type), exec);
+      DCHECK_OK(func->AddKernel(kernel));
+    }
+  }
+  return func;
+}
+
+const FunctionDoc year_doc{"Extract year values", "", {"values"}};
+const FunctionDoc month_doc{"Extract month values", "", {"values"}};
+const FunctionDoc day_doc{"Extract day values", "", {"values"}};
+const FunctionDoc day_of_week_doc{"Extract day of week values", "", {"values"}};
+const FunctionDoc day_of_year_doc{"Extract day of year values", "", {"values"}};
+const FunctionDoc iso_year_doc{"Extract ISO year values", "", {"values"}};
+const FunctionDoc iso_week_doc{"Extract ISO week values", "", {"values"}};
+const FunctionDoc iso_calendar_doc{"Extract ISO calendar values", "", {"values"}};
+const FunctionDoc quarter_doc{"Extract quarter values", "", {"values"}};
+const FunctionDoc hour_doc{"Extract hour values", "", {"values"}};
+const FunctionDoc minute_doc{"Extract minute values", "", {"values"}};
+const FunctionDoc second_doc{"Extract second values", "", {"values"}};
+const FunctionDoc millisecond_doc{"Extract millisecond values", "", {"values"}};
+const FunctionDoc microsecond_doc{"Extract microsecond values", "", {"values"}};
+const FunctionDoc nanosecond_doc{"Extract nanosecond values", "", {"values"}};
+const FunctionDoc subsecond_doc{"Extract subsecond values", "", {"values"}};
+
+void RegisterScalarTemporal(FunctionRegistry* registry) {
+  static std::vector<std::shared_ptr<DataType>> kUnsignedFloatTypes8 = {
+      uint8(), int8(),   uint16(), int16(),   uint32(),
+      int32(), uint64(), int64(),  float32(), float64()};
+  static std::vector<std::shared_ptr<DataType>> kUnsignedIntegerTypes8 = {
+      uint8(), int8(), uint16(), int16(), uint32(), int32(), uint64(), int64()};
+  static std::vector<std::shared_ptr<DataType>> kUnsignedIntegerTypes16 = {
+      uint16(), int16(), uint32(), int32(), uint64(), int64()};
+  static std::vector<std::shared_ptr<DataType>> kUnsignedIntegerTypes32 = {
+      uint32(), int32(), uint64(), int64()};
+  static std::vector<std::shared_ptr<DataType>> kSignedIntegerTypes = {int32(), int64()};

Review comment:
       [See](https://github.com/apache/arrow/pull/10176#discussion_r643454925)




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(

Review comment:
       date.h won't allow casting to types other than int32 and throws at compile-time:
   ```
   ../src/arrow/compute/kernels/scalar_temporal.cc:144:78: error: invalid static_cast from type 'arrow_vendored::date::year' to type 'const int64_t' {aka 'const long int'}
     144 |         year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
   ```
   I'll take a look if there's a recommended way to deal with 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.

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



[GitHub] [arrow] rok commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   CI issues don't seem related.
   I've also created a [followup Jira](https://issues.apache.org/jira/browse/ARROW-12980) for the timezone aware component extraction.


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

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -450,5 +450,145 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Week returns week of year value for each element of `values`
+///
+/// \param[in] values input to extract week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Week(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns quarter of year value for each element of `values`
+///
+/// \param[in] values input to extract quarter of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Quarter(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Hour returns hour value for each element of `values`
+///
+/// \param[in] values input to extract hour from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Hour(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Minute returns minutes value for each element of `values`
+///
+/// \param[in] values input to extract minutes from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Minute(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Second returns seconds value for each element of `values`
+///
+/// \param[in] values input to extract seconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Second(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Millisecond returns milliseconds value for each element of `values`
+///
+/// \param[in] values input to extract milliseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Millisecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Microsecond returns microseconds value for each element of `values`
+///
+/// \param[in] values input to extract microseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Microsecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Nanosecond returns nanoseconds value for each element of `values`
+///
+/// \param[in] values input to extract nanoseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Nanosecond(const Datum& values, ExecContext* ctx = NULLPTR);

Review comment:
       Yeah, I was also looking at pandas to compare ;) And that was the reason I am asking.
   
   So the situation is a bit complex there:
   
   ```
   >>> ts = pd.Timestamp("2012-01-02 03:04:05.123456789")
   >>> ts.second
   5
   ```
   
   There is a `microsecond` attribute which gives the total microseconds, but this is inherited from `datetime.datetime` (as this actually stores the sub-second part as microseconds):
   
   ```
   >>> ts.microsecond
   123456
   ```
   
   and then there is also a `nanosecond` attribute which is only the part after the microseconds (so the "additive componenent", not total nanoseconds):
   ```
   >>> ts.nanosecond
   789
   ```
   
   
   (the example in the pandas docs is indeed not good! As you can't tell which of the two cases it is because of the used value)




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -450,5 +450,145 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief Year returns year value for each element of `values`
+///
+/// \param[in] values input to extract year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Year(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Month returns month value for each element of `values`
+///
+/// \param[in] values input to extract month from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Month(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Day returns day value for each element of `values`
+///
+/// \param[in] values input to extract day from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Day(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Week returns week of year value for each element of `values`
+///
+/// \param[in] values input to extract week of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Week(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Quarter returns quarter of year value for each element of `values`
+///
+/// \param[in] values input to extract quarter of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> Quarter(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfYear returns day of year value for each element of `values`
+///
+/// \param[in] values input to extract day of year from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> DayOfYear(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief DayOfWeek returns day of the week value for each element of `values`
+///
+/// \param[in] values input to extract dat of the week from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Hour returns hour value for each element of `values`
+///
+/// \param[in] values input to extract hour from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Hour(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Minute returns minutes value for each element of `values`
+///
+/// \param[in] values input to extract minutes from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Minute(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Second returns seconds value for each element of `values`
+///
+/// \param[in] values input to extract seconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Second(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Millisecond returns milliseconds value for each element of `values`
+///
+/// \param[in] values input to extract milliseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Millisecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Microsecond returns microseconds value for each element of `values`
+///
+/// \param[in] values input to extract microseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Microsecond(const Datum& values, ExecContext* ctx = NULLPTR);
+
+/// \brief Nanosecond returns nanoseconds value for each element of `values`
+///
+/// \param[in] values input to extract nanoseconds from
+/// \param[in] ctx the function execution context, optional
+/// \return the resulting datum
+///
+/// \since 4.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Nanosecond(const Datum& values, ExecContext* ctx = NULLPTR);

Review comment:
       Oh yeah indeed. This would behave in the latter way which would be surprising to users. Will fix.




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,107 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestSimpleTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO);
+  auto timestamps = ArrayFromJSON(unit, times);
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("weekday", int64())});
+
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar = ArrayFromJSON(iso_calendar_type,
+                                    R"([{"iso_year": 1970, "iso_week": 1, "weekday": 4},
+                        {"iso_year": 2000, "iso_week": 9, "weekday": 2},
+                        {"iso_year": 1899, "iso_week": 52, "weekday": 7},
+                        {"iso_year": 2033, "iso_week": 20, "weekday": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59.123456789, 23.999999999, 20.001001001, 20.0, null]";
+  auto millisecond = "[123, 999, 1, 0, null]";
+  auto microsecond = "[456, 999, 1, 0, null]";
+  auto nanosecond = "[789, 999, 1, 0, null]";
+  auto subsecond = "[123456789, 999999999, 1001001, 0, null]";
+
+  CheckScalarUnary("year", unit, times, int64(), year);
+  CheckScalarUnary("month", unit, times, int64(), month);
+  CheckScalarUnary("day", unit, times, int64(), day);
+  CheckScalarUnary("day_of_week", unit, times, int64(), day_of_week);
+  CheckScalarUnary("day_of_year", unit, times, int64(), day_of_year);
+  CheckScalarUnary("iso_year", unit, times, int64(), iso_year);
+  CheckScalarUnary("iso_week", unit, times, int64(), iso_week);
+  CheckScalarUnary("iso_calendar", timestamps, iso_calendar);
+  CheckScalarUnary("quarter", unit, times, int64(), quarter);
+  CheckScalarUnary("hour", unit, times, int64(), hour);
+  CheckScalarUnary("minute", unit, times, int64(), minute);
+  CheckScalarUnary("second", unit, times, float64(), second);
+  CheckScalarUnary("millisecond", unit, times, int64(), millisecond);
+  CheckScalarUnary("microsecond", unit, times, int64(), microsecond);
+  CheckScalarUnary("nanosecond", unit, times, int64(), nanosecond);
+  CheckScalarUnary("subsecond", unit, times, int64(), subsecond);
+}
+
+TEST(ScalarTemporalTest, TestZonedTemporalComponentExtraction) {
+  std::string timezone = "Etc/UTC-2";
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO, timezone);
+  auto timestamps = ArrayFromJSON(unit, times);
+
+  ASSERT_RAISES(Invalid, Year(timestamps));
+  ASSERT_RAISES(Invalid, Month(timestamps));
+  ASSERT_RAISES(Invalid, Day(timestamps));
+  ASSERT_RAISES(Invalid, DayOfWeek(timestamps));
+  ASSERT_RAISES(Invalid, DayOfYear(timestamps));
+  ASSERT_RAISES(Invalid, ISOYear(timestamps));
+  ASSERT_RAISES(Invalid, ISOWeek(timestamps));
+  ASSERT_RAISES(Invalid, ISOCalendar(timestamps));
+  ASSERT_RAISES(Invalid, Quarter(timestamps));
+  ASSERT_RAISES(Invalid, Hour(timestamps));
+  ASSERT_RAISES(Invalid, Minute(timestamps));
+  ASSERT_RAISES(Invalid, Second(timestamps));
+  ASSERT_RAISES(Invalid, Millisecond(timestamps));
+  ASSERT_RAISES(Invalid, Microsecond(timestamps));
+  ASSERT_RAISES(Invalid, Nanosecond(timestamps));
+  ASSERT_RAISES(Invalid, Subsecond(timestamps));

Review comment:
       ML discussion on this would be good indeed.
   About raising when timezone is present - as a user I would be slightly surprised but could live with it.
   @joris [previously suggested raising](https://github.com/apache/arrow/pull/10176#issuecomment-828254868).
   
   I'm indifferent on this but would like add timezone processing ([like so](https://github.com/rok/arrow/commit/36abbcb512b2b45c8b5615fe29074be2afbd01d8)).




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

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



[GitHub] [arrow] jorisvandenbossche commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   General comment: the PR is quite big, and the timezone-related logic further added complexity. So from a workflow perspective (to ensure reviewing / getting this merged is manageable), could it make sense to keep the timezone-related logic for a separate PR? (didn't check the code to see whether that's actually feasible)


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

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: docs/source/cpp/compute.rst
##########
@@ -860,6 +860,58 @@ null input value is converted into a null output value.
   available).
 
 
+Temporal component extraction
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+These functions extract datetime components (year, month, day, etc) from timestamp type.
+Note: this is currently not supported for timestamps with timezone information.
+
++--------------------+------------+-------------------+---------------+--------+
+| Function name      | Arity      | Input types       | Output type   | Notes  |
++====================+============+===================+===============+========+
+| year               | Unary      | Temporal          | Int64         |        |
++--------------------+------------+-------------------+---------------+--------+
+| month              | Unary      | Temporal          | Int64         |        |
++--------------------+------------+-------------------+---------------+--------+
+| day                | Unary      | Temporal          | Int64         |        |
++--------------------+------------+-------------------+---------------+--------+
+| day_of_week        | Unary      | Temporal          | Int64         | \(1)   |
++--------------------+------------+-------------------+---------------+--------+
+| day_of_year        | Unary      | Temporal          | Int64         |        |
++--------------------+------------+-------------------+---------------+--------+
+| iso_year           | Unary      | Temporal          | Int64         | \(2)   |
++--------------------+------------+-------------------+---------------+--------+
+| iso_week           | Unary      | Temporal          | Int64         | \(2)   |
++--------------------+------------+-------------------+---------------+--------+
+| iso_calendar       | Unary      | Temporal          | Struct        | \(3)   |
++--------------------+------------+-------------------+---------------+--------+
+| quarter            | Unary      | Temporal          | Int64         |        |
++--------------------+------------+-------------------+---------------+--------+
+| hour               | Unary      | Temporal          | Int64         |        |
++--------------------+------------+-------------------+---------------+--------+
+| minute             | Unary      | Temporal          | Int64         |        |
++--------------------+------------+-------------------+---------------+--------+
+| second             | Unary      | Temporal          | Int64         |        |
++--------------------+------------+-------------------+---------------+--------+
+| millisecond        | Unary      | Temporal          | Int64         |        |
++--------------------+------------+-------------------+---------------+--------+
+| microsecond        | Unary      | Temporal          | Int64         |        |
++--------------------+------------+-------------------+---------------+--------+
+| nanosecond         | Unary      | Temporal          | Int64         |        |
++--------------------+------------+-------------------+---------------+--------+
+| subsecond          | Unary      | Temporal          | Double        |        |
++--------------------+------------+-------------------+---------------+--------+
+
+* \(1) Outputs the number of the day of the week. Week begins on Monday and is denoted
+  by 0 and ends on Sunday denoted by 6.
+* \(2) First ISO week has the majority (4 or more) of it's days in January. ISO year
+  starts with the first ISO week.
+  See `ISO 8601 week date definition`_ for more details.
+* \(3) Output is a ``{"iso_year": output type, "iso_week": output type, "day_of_week":  output type}`` Struct.

Review comment:
       ```suggestion
   * \(3) Output is a ``{"iso_year": output type, "iso_week": output type, "iso_day_of_week":  output type}`` Struct.
   ```
   
   Just noted one last detail, but @rok let's leave this for your next PR to fix this, so we can let CI continue here and get this merged ;)




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: docs/source/cpp/compute.rst
##########
@@ -637,6 +637,54 @@ String extraction
   e.g. 'letter' and 'digit' for the regular expression
   ``(?P<letter>[ab])(?P<digit>\\d)``.
 
+Temporal component extraction
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+These functions extract datetime components (year, month, day, etc) from timestamp type.
+Note: timezone information is currently ignored if present.
+
++--------------------+------------+-------------------+-----------------+--------+
+| Function name      | Arity      | Input types       | Output type     | Notes  |
++====================+============+===================+=================+========+
+| year               | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| month              | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| day                | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| day_of_week        | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| day_of_year        | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| iso_year           | Unary      | Temporal          | Numeric         | \(1)   |
++--------------------+------------+-------------------+-----------------+--------+
+| iso_week           | Unary      | Temporal          | Numeric         | \(1)   |
++--------------------+------------+-------------------+-----------------+--------+
+| iso_calendar       | Unary      | Temporal          | Scalar Struct   | \(2)   |
++--------------------+------------+-------------------+-----------------+--------+
+| quarter            | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| hour               | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| minute             | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| second             | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| millisecond        | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| microsecond        | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| nanosecond         | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| subsecond          | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+
+* \(1) The ISO 8601 definition for week 01 is the week with the first Thursday
+  of the Gregorian year (i.e. of January) in it.
+  .. _Wikipedia ISO Week date: https://en.wikipedia.org/wiki/ISO_week_date#First_week

Review comment:
       Changed.




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

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



[GitHub] [arrow] rok commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   Thanks for the comments and suggestions @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.

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: docs/source/cpp/compute.rst
##########
@@ -637,6 +637,55 @@ String extraction
   e.g. 'letter' and 'digit' for the regular expression
   ``(?P<letter>[ab])(?P<digit>\\d)``.
 
+Temporal component extraction
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+These functions extract datetime components (year, month, day, etc) from timestamp type.
+Note: timezone information is currently ignored if present.
+
++--------------------+------------+-------------------+-----------------+--------+
+| Function name      | Arity      | Input types       | Output type     | Notes  |
++====================+============+===================+=================+========+
+| year               | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| month              | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| day                | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| day_of_week        | Unary      | Temporal          | Numeric         | \(1)   |
++--------------------+------------+-------------------+-----------------+--------+
+| day_of_year        | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| iso_year           | Unary      | Temporal          | Numeric         | \(2)   |
++--------------------+------------+-------------------+-----------------+--------+
+| iso_week           | Unary      | Temporal          | Numeric         | \(2)   |
++--------------------+------------+-------------------+-----------------+--------+
+| iso_calendar       | Unary      | Temporal          | Scalar Struct   | \(3)   |
++--------------------+------------+-------------------+-----------------+--------+
+| quarter            | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| hour               | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| minute             | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| second             | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| millisecond        | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| microsecond        | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| nanosecond         | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+| subsecond          | Unary      | Temporal          | Numeric         |        |
++--------------------+------------+-------------------+-----------------+--------+
+
+* \(1) Outputs the number of the day of the week. Week begins on Monday and is denoted
+  by 0 and ends on Sunday denoted by 6.
+* \(2) First ISO week has the majority (4 or more) of it's days in January. ISO year
+  starts with the first ISO week.
+  See `ISO 8601 week date definition`_ for more details.
+* \(3) Output is a ``{"iso_year": output type, "iso_week": output type, "day_of_week":  output type}`` Struct.
+.. _ISO 8601 week date definition: https://en.wikipedia.org/wiki/ISO_week_date#First_week

Review comment:
       Sounds like a good idea!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,348 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+        .iso_encoding();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return (sd - sys_days(year_month_day(sd).year() / jan / 0)).count();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct Week {

Review comment:
       We should probably be explicit here about it being the "ISO week" number (and not the "logical" week number), as it gives some surprising results around New Year (https://en.wikipedia.org/wiki/ISO_week_date#First_week)
   
   (and if adding a ISO week, we should maybe also add the corresponding ISO year)




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

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



[GitHub] [arrow] pitrou commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(

Review comment:
       Ah, ok. You can just add a comment 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.

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,509 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return static_cast<T>((sd - sys_days(year_month_day(sd).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day{sys_days{floor<days>(Duration{arg})} + days{3}}.year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    const auto dp = sys_days{floor<days>(Duration{arg})};
+    auto y = year_month_day{dp + days{3}}.year();
+    auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (dp < start) {
+      --y;
+      start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(dp - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+const std::shared_ptr<DataType> iso_calendar_type = struct_(
+    {field("iso_year", int64()), field("iso_week", int64()), field("weekday", int64())});
+
+template <typename Duration>
+struct ISOCalendar {
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    using ScalarType = typename TypeTraits<Int64Type>::ScalarType;
+
+    const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+    const auto dp = sys_days{floor<days>(Duration{in_val})};
+    const auto ymd = year_month_day(dp);
+    auto y = year_month_day{dp + days{3}}.year();
+    auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (dp < start) {
+      --y;
+      start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+
+    std::vector<std::shared_ptr<Scalar>> values = {
+        std::make_shared<ScalarType>(
+            static_cast<int64_t>(static_cast<int32_t>(ymd.year()))),
+        std::make_shared<ScalarType>(
+            static_cast<int64_t>(trunc<weeks>(dp - start).count() + 1)),
+        std::make_shared<ScalarType>(static_cast<int64_t>(weekday(ymd).iso_encoding()))};
+    *checked_cast<StructScalar*>(out) =
+        StructScalar(std::move(values), iso_calendar_type);
+    return Status::OK();
+  }

Review comment:
       Got it. Kernel signature was set to only expect arrays and not scalars.




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

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



[GitHub] [arrow] pitrou commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -0,0 +1,107 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/formatting.h"
+
+namespace arrow {
+
+using internal::StringFormatter;
+
+class ScalarTemporalTest : public ::testing::Test {};
+
+namespace compute {
+
+TEST(ScalarTemporalTest, TestSimpleTemporalComponentExtraction) {
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO);
+  auto timestamps = ArrayFromJSON(unit, times);
+  auto iso_calendar_type =
+      struct_({field("iso_year", int64()), field("iso_week", int64()),
+               field("weekday", int64())});
+
+  auto year = "[1970, 2000, 1899, 2033, null]";
+  auto month = "[1, 2, 1, 5, null]";
+  auto day = "[1, 29, 1, 18, null]";
+  auto day_of_week = "[4, 2, 7, 3, null]";
+  auto day_of_year = "[1, 60, 1, 138, null]";
+  auto iso_year = "[1970, 2000, 1899, 2033, null]";
+  auto iso_week = "[1, 9, 52, 20, null]";
+  auto iso_calendar = ArrayFromJSON(iso_calendar_type,
+                                    R"([{"iso_year": 1970, "iso_week": 1, "weekday": 4},
+                        {"iso_year": 2000, "iso_week": 9, "weekday": 2},
+                        {"iso_year": 1899, "iso_week": 52, "weekday": 7},
+                        {"iso_year": 2033, "iso_week": 20, "weekday": 3}, null])");
+  auto quarter = "[1, 1, 1, 2, null]";
+  auto hour = "[0, 23, 0, 3, null]";
+  auto minute = "[0, 23, 59, 33, null]";
+  auto second = "[59.123456789, 23.999999999, 20.001001001, 20.0, null]";
+  auto millisecond = "[123, 999, 1, 0, null]";
+  auto microsecond = "[456, 999, 1, 0, null]";
+  auto nanosecond = "[789, 999, 1, 0, null]";
+  auto subsecond = "[123456789, 999999999, 1001001, 0, null]";
+
+  CheckScalarUnary("year", unit, times, int64(), year);
+  CheckScalarUnary("month", unit, times, int64(), month);
+  CheckScalarUnary("day", unit, times, int64(), day);
+  CheckScalarUnary("day_of_week", unit, times, int64(), day_of_week);
+  CheckScalarUnary("day_of_year", unit, times, int64(), day_of_year);
+  CheckScalarUnary("iso_year", unit, times, int64(), iso_year);
+  CheckScalarUnary("iso_week", unit, times, int64(), iso_week);
+  CheckScalarUnary("iso_calendar", timestamps, iso_calendar);
+  CheckScalarUnary("quarter", unit, times, int64(), quarter);
+  CheckScalarUnary("hour", unit, times, int64(), hour);
+  CheckScalarUnary("minute", unit, times, int64(), minute);
+  CheckScalarUnary("second", unit, times, float64(), second);
+  CheckScalarUnary("millisecond", unit, times, int64(), millisecond);
+  CheckScalarUnary("microsecond", unit, times, int64(), microsecond);
+  CheckScalarUnary("nanosecond", unit, times, int64(), nanosecond);
+  CheckScalarUnary("subsecond", unit, times, int64(), subsecond);
+}
+
+TEST(ScalarTemporalTest, TestZonedTemporalComponentExtraction) {
+  std::string timezone = "Etc/UTC-2";
+  const char* times =
+      R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
+          "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000", null])";
+  auto unit = timestamp(TimeUnit::NANO, timezone);
+  auto timestamps = ArrayFromJSON(unit, times);
+
+  ASSERT_RAISES(Invalid, Year(timestamps));
+  ASSERT_RAISES(Invalid, Month(timestamps));
+  ASSERT_RAISES(Invalid, Day(timestamps));
+  ASSERT_RAISES(Invalid, DayOfWeek(timestamps));
+  ASSERT_RAISES(Invalid, DayOfYear(timestamps));
+  ASSERT_RAISES(Invalid, ISOYear(timestamps));
+  ASSERT_RAISES(Invalid, ISOWeek(timestamps));
+  ASSERT_RAISES(Invalid, ISOCalendar(timestamps));
+  ASSERT_RAISES(Invalid, Quarter(timestamps));
+  ASSERT_RAISES(Invalid, Hour(timestamps));
+  ASSERT_RAISES(Invalid, Minute(timestamps));
+  ASSERT_RAISES(Invalid, Second(timestamps));
+  ASSERT_RAISES(Invalid, Millisecond(timestamps));
+  ASSERT_RAISES(Invalid, Microsecond(timestamps));
+  ASSERT_RAISES(Invalid, Nanosecond(timestamps));
+  ASSERT_RAISES(Invalid, Subsecond(timestamps));

Review comment:
       What is the expected semantics? It seems to me that the results would simply returned in the local timezone, i.e. if I have a timestamp for '2000-01-01 00:00:00' in the UTC+1 timezone, then the year is 2000... in the UTC+1 timezone.
   
   If users want to convert to UTC first, perhaps we want a separate kernel for that, but I see no reason to do that conversion in the datetime component extraction kernels.
   
   For example in Python:
   ```python
   >>> t = dateutil.tz.gettz('Europe/Paris')
   >>> dt = datetime.datetime(2000,1,1,0,0,0, tzinfo=t)
   >>> dt
   datetime.datetime(2000, 1, 1, 0, 0, tzinfo=tzfile('/usr/share/zoneinfo/Europe/Paris'))
   >>> dt.year
   2000
   >>> t.utcoffset(dt)
   datetime.timedelta(seconds=3600)
   >>> dt.utctimetuple().tm_year
   1999
   ```
   




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,626 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+
+namespace internal {
+
+namespace {
+
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+using internal::applicator::ScalarUnaryNotNull;
+using internal::applicator::SimpleUnary;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename Op, typename OutType>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<Op, OutType>;
+  using OutValue = typename internal::GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          checked_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        internal::VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        checked_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = internal::UnboxScalar<Int64Type>::Unbox(arg0);
+        internal::BoxScalar<OutType>::Box(
+            this->op.template Call<OutValue>(ctx, arg0_val, &st), out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename Op, typename OutType>
+struct ScalarUnaryTemporal {
+  using OutValue = typename internal::GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<Op, OutType> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding() -
+        1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    return static_cast<T>(
+        (t - sys_time<days>(year_month_day(t).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+//
+// First week of an ISO year has the majority (4 or more) of it's days in January.
+// Last week of an ISO year has the year's last Thursday in it.
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    auto y = year_month_day{t + days{3}}.year();
+    auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (t < start) {
+      --y;
+    }
+    return static_cast<T>(static_cast<int32_t>(y));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+//
+// First week of an ISO year has the majority (4 or more) of it's days in January.
+// Last week of an ISO year has the year's last Thursday in it.
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+    auto y = year_month_day{t + days{3}}.year();
+    auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (t < start) {
+      --y;
+      start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(t - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::minutes>(t)) / std::chrono::seconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        (std::chrono::duration<double>(t - floor<std::chrono::seconds>(t)).count()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+template <typename Duration>
+inline std::vector<int64_t> get_iso_calendar(int64_t arg) {
+  const auto t = floor<days>(sys_time<Duration>(Duration{arg}));
+  const auto ymd = year_month_day(t);
+  auto y = year_month_day{t + days{3}}.year();
+  auto start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+  if (t < start) {
+    --y;
+    start = sys_time<days>((y - years{1}) / dec / thu[last]) + (mon - thu);
+  }
+  return {static_cast<int64_t>(static_cast<int32_t>(y)),
+          static_cast<int64_t>(trunc<weeks>(t - start).count() + 1),
+          static_cast<int64_t>(weekday(ymd).iso_encoding())};

Review comment:
       Added to docs and prepended.




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

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



[GitHub] [arrow] rok commented on a change in pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -0,0 +1,614 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/builder.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/time.h"
+#include "arrow/vendored/datetime.h"
+
+namespace arrow {
+
+namespace compute {
+namespace internal {
+
+using applicator::ScalarUnaryNotNull;
+using applicator::SimpleUnary;
+using arrow_vendored::date::days;
+using arrow_vendored::date::floor;
+using arrow_vendored::date::hh_mm_ss;
+using arrow_vendored::date::sys_days;
+using arrow_vendored::date::sys_time;
+using arrow_vendored::date::trunc;
+using arrow_vendored::date::weekday;
+using arrow_vendored::date::weeks;
+using arrow_vendored::date::year_month_day;
+using arrow_vendored::date::years;
+using arrow_vendored::date::literals::dec;
+using arrow_vendored::date::literals::jan;
+using arrow_vendored::date::literals::last;
+using arrow_vendored::date::literals::mon;
+using arrow_vendored::date::literals::thu;
+
+// Based on ScalarUnaryNotNullStateful. Adds timezone awareness.
+template <typename OutType, typename Op>
+struct ScalarUnaryStatefulTemporal {
+  using ThisType = ScalarUnaryStatefulTemporal<OutType, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  Op op;
+  explicit ScalarUnaryStatefulTemporal(Op op) : op(std::move(op)) {}
+
+  template <typename Type>
+  struct ArrayExec {
+    static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
+                       Datum* out) {
+      const std::string timezone =
+          std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+      Status st = Status::OK();
+      ArrayData* out_arr = out->mutable_array();
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
+
+      if (timezone.empty()) {
+        VisitArrayValuesInline<Int64Type>(
+            arg0,
+            [&](int64_t v) {
+              *out_data++ = functor.op.template Call<OutValue>(ctx, v, &st);
+            },
+            [&]() {
+              // null
+              ++out_data;
+            });
+      } else {
+        st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+      }
+      return st;
+    }
+  };
+
+  Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(arg0.type)->timezone();
+    Status st = Status::OK();
+    if (timezone.empty()) {
+      if (arg0.is_valid) {
+        int64_t arg0_val = UnboxScalar<Int64Type>::Unbox(arg0);
+        BoxScalar<OutType>::Box(this->op.template Call<OutValue>(ctx, arg0_val, &st),
+                                out->scalar().get());
+      }
+    } else {
+      st = Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                           timezone);
+    }
+    return st;
+  }
+
+  Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      return ArrayExec<OutType>::Exec(*this, ctx, *batch[0].array(), out);
+    } else {
+      return Scalar(ctx, *batch[0].scalar(), out);
+    }
+  }
+};
+
+template <typename OutType, typename Op>
+struct ScalarUnaryTemporal {
+  using OutValue = typename GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarUnaryStatefulTemporal<OutType, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract year from timestamp
+
+template <typename Duration>
+struct Year {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract month from timestamp
+
+template <typename Duration>
+struct Month {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).month()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day from timestamp
+
+template <typename Duration>
+struct Day {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const uint32_t>(
+        year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))).day()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of week from timestamp
+
+template <typename Duration>
+struct DayOfWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(
+        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
+            .iso_encoding());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of year from timestamp
+
+template <typename Duration>
+struct DayOfYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto sd = sys_days{floor<days>(Duration{arg})};
+    return static_cast<T>((sd - sys_days(year_month_day(sd).year() / jan / 0)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO Year values from timestamp
+
+template <typename Duration>
+struct ISOYear {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    return static_cast<T>(static_cast<const int32_t>(
+        year_month_day{sys_days{floor<days>(Duration{arg})} + days{3}}.year()));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO week from timestamp
+
+// Based on
+// https://github.com/HowardHinnant/date/blob/6e921e1b1d21e84a5c82416ba7ecd98e33a436d0/include/date/iso_week.h#L1503
+template <typename Duration>
+struct ISOWeek {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto dp = sys_days{floor<days>(Duration{arg})};
+    auto y = year_month_day{dp + days{3}}.year();
+    auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    if (dp < start) {
+      --y;
+      start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+    }
+    return static_cast<T>(trunc<weeks>(dp - start).count() + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract day of quarter from timestamp
+
+template <typename Duration>
+struct Quarter {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    const auto ymd = year_month_day(floor<days>(sys_time<Duration>(Duration{arg})));
+    return static_cast<T>((static_cast<const uint32_t>(ymd.month()) - 1) / 3 + 1);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract hour from timestamp
+
+template <typename Duration>
+struct Hour {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<days>(t)) / std::chrono::hours(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract minute from timestamp
+
+template <typename Duration>
+struct Minute {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::hours>(t)) / std::chrono::minutes(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract second from timestamp
+
+template <typename Duration>
+struct Second {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        std::chrono::duration<double>(t - floor<std::chrono::minutes>(t)).count());
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract subsecond from timestamp
+
+template <typename Duration>
+struct Subsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>((t - floor<std::chrono::seconds>(t)) /
+                          std::chrono::nanoseconds(1));
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract milliseconds from timestamp
+
+template <typename Duration>
+struct Millisecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::milliseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract microseconds from timestamp
+
+template <typename Duration>
+struct Microsecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::microseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract nanoseconds from timestamp
+
+template <typename Duration>
+struct Nanosecond {
+  template <typename T>
+  static T Call(KernelContext*, int64_t arg, Status*) {
+    Duration t = Duration{arg};
+    return static_cast<T>(
+        ((t - floor<std::chrono::seconds>(t)) / std::chrono::nanoseconds(1)) % 1000);
+  }
+};
+
+// ----------------------------------------------------------------------
+// Extract ISO calendar values from timestamp
+
+template <typename Duration, typename OutType>
+struct ISOCalendar {
+  using T = typename OutType::c_type;
+
+  static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) {
+    using ScalarType = typename TypeTraits<OutType>::ScalarType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    if (!in.is_valid) {
+      out->is_valid = false;
+    } else {
+      const std::shared_ptr<DataType> iso_calendar_type =
+          struct_({field("iso_year", out_type), field("iso_week", out_type),
+                   field("weekday", out_type)});
+
+      const auto& in_val = internal::UnboxScalar<const TimestampType>::Unbox(in);
+      const auto dp = sys_days{floor<days>(Duration{in_val})};
+      const auto ymd = year_month_day(dp);
+      auto y = year_month_day{dp + days{3}}.year();
+      auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      if (dp < start) {
+        --y;
+        start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      }
+
+      std::vector<std::shared_ptr<Scalar>> values = {
+          std::make_shared<ScalarType>(static_cast<T>(static_cast<int32_t>(ymd.year()))),
+          std::make_shared<ScalarType>(
+              static_cast<T>(trunc<weeks>(dp - start).count() + 1)),
+          std::make_shared<ScalarType>(static_cast<T>(weekday(ymd).iso_encoding()))};
+      *checked_cast<StructScalar*>(out) = StructScalar(values, iso_calendar_type);
+    }
+    return Status::OK();
+  }
+
+  static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) {
+    using BuilderType = typename TypeTraits<OutType>::BuilderType;
+    const auto& out_type = TypeTraits<OutType>::type_singleton();
+
+    const std::string timezone =
+        std::static_pointer_cast<const TimestampType>(in.type)->timezone();
+    if (!timezone.empty()) {
+      return Status::Invalid("Timezone aware timestamps not supported. Timezone found: ",
+                             timezone);
+    }
+
+    const std::shared_ptr<DataType> iso_calendar_type =
+        struct_({field("iso_year", out_type), field("iso_week", out_type),
+                 field("weekday", out_type)});
+
+    std::unique_ptr<ArrayBuilder> array_builder;
+    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), iso_calendar_type, &array_builder));
+    StructBuilder* struct_builder = checked_cast<StructBuilder*>(array_builder.get());
+
+    std::vector<BuilderType*> field_builders;
+    field_builders.reserve(3);
+    for (int i = 0; i < 3; i++) {
+      field_builders.push_back(
+          checked_cast<BuilderType*>(struct_builder->field_builder(i)));
+    }
+    auto visit_null = [&]() { return struct_builder->AppendNull(); };
+    auto visit_value = [&](int64_t arg) {
+      const auto dp = sys_days{floor<days>(Duration{arg})};
+      const auto ymd = year_month_day(dp);
+      auto y = year_month_day{dp + days{3}}.year();
+      auto start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      if (dp < start) {
+        --y;
+        start = sys_days((y - years{1}) / dec / thu[last]) + (mon - thu);
+      }
+
+      RETURN_NOT_OK(
+          field_builders[0]->Append(static_cast<T>(static_cast<int32_t>(ymd.year()))));
+      RETURN_NOT_OK(field_builders[1]->Append(
+          static_cast<T>(trunc<weeks>(dp - start).count() + 1)));
+      RETURN_NOT_OK(
+          field_builders[2]->Append(static_cast<T>(weekday(ymd).iso_encoding())));
+
+      return struct_builder->Append();
+    };
+    RETURN_NOT_OK(VisitArrayDataInline<OutType>(in, visit_value, visit_null));
+
+    std::shared_ptr<Array> out_array;
+    RETURN_NOT_OK(struct_builder->Finish(&out_array));
+    *out = *std::move(out_array->data());
+
+    return Status::OK();
+  }
+};
+
+// Generate a kernel given an arithmetic functor
+template <template <typename... Args> class KernelGenerator,
+          template <typename... Args> class Op, typename Duration>
+ArrayKernelExec ExecFromOp(detail::GetTypeId get_id) {
+  switch (get_id.id) {

Review comment:
       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.

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



[GitHub] [arrow] jorisvandenbossche commented on pull request #10176: ARROW-11759: [C++] Kernel to extract datetime components (year, month, day, etc) from timestamp type

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


   There seems to be something wrong with the ISO calendar (year) implementation. I was playing a bit with this branch, and seeing:
   
   ```python
   In [25]: arr = pa.array([pd.Timestamp("2012-01-01 01:02:03")], type=pa.timestamp("ns"))
   
   In [26]: pc.iso_calendar(arr)
   Out[26]: 
   <pyarrow.lib.StructArray object at 0x7f5bbf3639a0>
   -- is_valid: all not null
   -- child 0 type: int64
     [
       2012            #  <--------------  this should be 2011 (since the week is 52, it falls in the "previous" year)
     ]
   -- child 1 type: int64
     [
       52
     ]
   -- child 2 type: int64
     [
       7
     ]
   ```
   
   Also in general, for non-nanosecond resolutions, you get wrong results for all kernels. It seems that it is interpreting the non-nanosecond epoch as nanosecond epoch. That's related to your previous comment that tests for other units are still failing?
   


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

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