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/24 22:02:55 UTC

[GitHub] [arrow] rok opened a new pull request #10598: ARROW-13054: [C++] Add TemporalOptions

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


   This is to resolve [ARROW-13054](https://issues.apache.org/jira/browse/ARROW-13054).
   This will be needed for casting timezone-naive timestamps [ARROW-13033](https://issues.apache.org/jira/browse/ARROW-13033) and defining [starting day of the week](https://github.com/apache/arrow/pull/10507#pullrequestreview-681491088).


-- 
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 #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -136,19 +136,25 @@ struct Day {
 
 // ----------------------------------------------------------------------
 // Extract day of week from timestamp
+//
+// By default week starts on Monday represented by 0 and ends on Sunday represented
+// by 6. Start day of the week (Monday=1, Sunday=7) and numbering start (0 or 1) can be
+// set using DayOfWeekOptions
 
 template <typename Duration>
 struct DayOfWeek {
-  explicit DayOfWeek(TemporalComponentExtractionOptions options) : options(options) {}
+  explicit DayOfWeek(DayOfWeekOptions options) : options(options) {}
 
   template <typename T, typename Arg0>
   T Call(KernelContext*, Arg0 arg, Status*) const {
-    return static_cast<T>(
-        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
-            .iso_encoding() -
-        1 + options.week_start);
+    auto wd = arrow_vendored::date::year_month_weekday(
+                  floor<days>(sys_time<Duration>(Duration{arg})))
+                  .weekday()
+                  .iso_encoding();
+    return (wd + 7 - options.week_start) % 7 + options.one_based_numbering;

Review comment:
       Or, of course, a simple lookup table could be created in the kernel constructor.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -136,19 +136,25 @@ struct Day {
 
 // ----------------------------------------------------------------------
 // Extract day of week from timestamp
+//
+// By default week starts on Monday represented by 0 and ends on Sunday represented
+// by 6. Start day of the week (Monday=1, Sunday=7) and numbering start (0 or 1) can be
+// set using DayOfWeekOptions
 
 template <typename Duration>
 struct DayOfWeek {
-  explicit DayOfWeek(TemporalComponentExtractionOptions options) : options(options) {}
+  explicit DayOfWeek(DayOfWeekOptions options) : options(options) {}
 
   template <typename T, typename Arg0>
   T Call(KernelContext*, Arg0 arg, Status*) const {
-    return static_cast<T>(
-        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
-            .iso_encoding() -
-        1 + options.week_start);
+    auto wd = arrow_vendored::date::year_month_weekday(
+                  floor<days>(sys_time<Duration>(Duration{arg})))
+                  .weekday()
+                  .iso_encoding();
+    return (wd + 7 - options.week_start) % 7 + options.one_based_numbering;

Review comment:
       Done.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -80,6 +84,25 @@ struct TemporalComponentExtract {
   }
 };
 
+template <typename Op, typename OutType>
+struct DayOfWeekOptionsWithOptions {

Review comment:
       Well, it's still confusing. `DayOfWeekExec`?




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -118,16 +141,31 @@ struct Day {
 
 // ----------------------------------------------------------------------
 // Extract day of week from timestamp
+//
+// By default week starts on Monday represented by 0 and ends on Sunday represented
+// by 6. Start day of the week (Monday=1, Sunday=7) and numbering start (0 or 1) can be
+// set using DayOfWeekOptions
 
 template <typename Duration>
 struct DayOfWeek {
+  explicit DayOfWeek(DayOfWeekOptions options) : options(options) {

Review comment:
       Nit: can you usually pass options as `const&`.

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -80,6 +84,25 @@ struct TemporalComponentExtract {
   }
 };
 
+template <typename Op, typename OutType>
+struct DayOfWeekExec {
+  using OutValue = typename internal::GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    DayOfWeekOptions options = DayOfWeekState::Get(ctx);

Review comment:
       Nit: could be `const&`.

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -118,16 +141,31 @@ struct Day {
 
 // ----------------------------------------------------------------------
 // Extract day of week from timestamp
+//
+// By default week starts on Monday represented by 0 and ends on Sunday represented
+// by 6. Start day of the week (Monday=1, Sunday=7) and numbering start (0 or 1) can be
+// set using DayOfWeekOptions
 
 template <typename Duration>
 struct DayOfWeek {
+  explicit DayOfWeek(DayOfWeekOptions options) : options(options) {
+    for (int i = 0; i < 7; i++) {
+      lookup_table[i] = i + 8 - options.week_start;
+      lookup_table[i] = (lookup_table[i] > 6) ? lookup_table[i] - 7 : lookup_table[i];
+      lookup_table[i] += options.one_based_numbering;
+    }
+  }
+
   template <typename T, typename Arg0>
-  static T Call(KernelContext*, Arg0 arg, Status*) {
-    return static_cast<T>(
-        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
-            .iso_encoding() -
-        1);
+  T Call(KernelContext*, Arg0 arg, Status*) const {
+    const auto wd = arrow_vendored::date::year_month_weekday(
+                        floor<days>(sys_time<Duration>(Duration{arg})))
+                        .weekday()
+                        .iso_encoding();
+    return lookup_table[wd - 1];
   }
+  std::array<int64_t, 7> lookup_table;
+  DayOfWeekOptions options;

Review comment:
       This member doesn't seem used anymore?




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] thisisnic commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: r/src/compute.cpp
##########
@@ -264,6 +264,12 @@ std::shared_ptr<arrow::compute::FunctionOptions> make_compute_options(
                                      max_replacements);
   }
 
+  if (func_name == "day_of_week") {
+    using Options = arrow::compute::TemporalComponentExtractionOptions;
+    return std::make_shared<Options>(cpp11::as_cpp<bool>(options["one_based_numbering"]),

Review comment:
       Looks good, although 1 minor suggestion - please could we have default options set here so if someone calls `day_of_week` via `call_function("day_of_week", x)`, they don't have to supply either argument?




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] thisisnic commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: r/src/compute.cpp
##########
@@ -264,6 +264,12 @@ std::shared_ptr<arrow::compute::FunctionOptions> make_compute_options(
                                      max_replacements);
   }
 
+  if (func_name == "day_of_week") {
+    using Options = arrow::compute::TemporalComponentExtractionOptions;
+    return std::make_shared<Options>(cpp11::as_cpp<bool>(options["one_based_numbering"]),

Review comment:
       @rok Not quite - `one_based_numbering` has a default value of `false` in C++, whereas in R it should be `true`.  I _think_ the way to achieve it is to not use the C++ defaults and do something like this: https://github.com/apache/arrow/blob/e9fa30406215b76ed6c885302fbfe6075c47badf/r/src/compute.cpp#L244-L254




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -216,16 +216,17 @@ struct ARROW_EXPORT ProjectOptions : public FunctionOptions {
   std::vector<std::shared_ptr<const KeyValueMetadata>> field_metadata;
 };
 
-struct ARROW_EXPORT TemporalComponentExtractionOptions : public FunctionOptions {
-  explicit TemporalComponentExtractionOptions(int64_t week_start = 0)
-      : week_start(std::move(week_start)) {}
+struct ARROW_EXPORT DayOfWeekOptions : public FunctionOptions {
+  explicit DayOfWeekOptions(bool one_based_numbering, uint32_t week_start)
+      : one_based_numbering(one_based_numbering), week_start(week_start) {}
+  explicit DayOfWeekOptions() : one_based_numbering(false), week_start(1) {}
 
-  static TemporalComponentExtractionOptions Defaults() {
-    return TemporalComponentExtractionOptions{};
-  }
+  static DayOfWeekOptions Defaults() { return DayOfWeekOptions{}; }
 
-  /// Index of the first day of the week.
-  int64_t week_start;
+  /// Number days from 1 if true and form 0 if false

Review comment:
       ```suggestion
     /// Number days from 1 if true and from 0 if false
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -136,19 +136,25 @@ struct Day {
 
 // ----------------------------------------------------------------------
 // Extract day of week from timestamp
+//
+// By default week starts on Monday represented by 0 and ends on Sunday represented
+// by 6. Start day of the week (Monday=1, Sunday=7) and numbering start (0 or 1) can be
+// set using DayOfWeekOptions
 
 template <typename Duration>
 struct DayOfWeek {
-  explicit DayOfWeek(TemporalComponentExtractionOptions options) : options(options) {}
+  explicit DayOfWeek(DayOfWeekOptions options) : options(options) {}
 
   template <typename T, typename Arg0>
   T Call(KernelContext*, Arg0 arg, Status*) const {
-    return static_cast<T>(
-        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
-            .iso_encoding() -
-        1 + options.week_start);
+    auto wd = arrow_vendored::date::year_month_weekday(
+                  floor<days>(sys_time<Duration>(Duration{arg})))
+                  .weekday()
+                  .iso_encoding();
+    return (wd + 7 - options.week_start) % 7 + options.one_based_numbering;

Review comment:
       Would it be worth to have a separate if branch for the case that `options.week_start` is 1 (the default), in which case this is this is much simpler (eg you don't need the `%`)? 
   (I don't know how expensive those calculations are compared to the actual `year_month_weekday(..).weekday()` call, though, so maybe it doesn't matter)

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -216,6 +216,19 @@ struct ARROW_EXPORT ProjectOptions : public FunctionOptions {
   std::vector<std::shared_ptr<const KeyValueMetadata>> field_metadata;
 };
 
+struct ARROW_EXPORT DayOfWeekOptions : public FunctionOptions {
+  explicit DayOfWeekOptions(bool one_based_numbering, uint32_t week_start)
+      : one_based_numbering(one_based_numbering), week_start(week_start) {}

Review comment:
       Can you validate the `week_start` here? (to check that it is in the correct range, and eg a user is not passing 0. Since the numbering here is different as the default output of DayOfWeek, that seems like an easy mistake)

##########
File path: docs/source/cpp/compute.rst
##########
@@ -954,44 +954,46 @@ 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.
++--------------------+------------+-------------------+---------------+--------+----------------------------+
+| Function name      | Arity      | Input types       | Output type   | Notes  | Options class              |
++====================+============+===================+===============+========+============================+
+| year               | Unary      | Temporal          | Int64         |        |                            |
++--------------------+------------+-------------------+---------------+--------+----------------------------+
+| month              | Unary      | Temporal          | Int64         |        |                            |
++--------------------+------------+-------------------+---------------+--------+----------------------------+
+| day                | Unary      | Temporal          | Int64         |        |                            |
++--------------------+------------+-------------------+---------------+--------+----------------------------+
+| day_of_week        | Unary      | Temporal          | Int64         | \(1)   | :struct:`DayOfWeekOptions` |
++--------------------+------------+-------------------+---------------+--------+----------------------------+
+| 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. By default week begins on Monday
+  represented by 0 and ends on Sunday represented by 6. :member:`DayOfWeekOptions::week_start` can be used to set
+  the starting day of the week using ISO convention (Monday=1, Sunday=7). Day numbering can start with 0 or 1
+  using :member:`DayOfWeekOptions::week_start` paramter.

Review comment:
       ```suggestion
     using :member:`DayOfWeekOptions::week_start` parameter.
   ```

##########
File path: python/pyarrow/_compute.pyx
##########
@@ -1043,6 +1043,23 @@ class StrptimeOptions(_StrptimeOptions):
         self._set_options(format, unit)
 
 
+cdef class _DayOfWeekOptions(FunctionOptions):
+    cdef:
+        unique_ptr[CDayOfWeekOptions] day_of_week_options
+
+    cdef const CFunctionOptions* get_options(self) except NULL:
+        return self.day_of_week_options.get()
+
+    def _set_options(self, one_based_numbering, week_start):
+        self.day_of_week_options.reset(
+            new CDayOfWeekOptions(one_based_numbering, week_start))
+
+
+class DayOfWeekOptions(_DayOfWeekOptions):
+    def __init__(self, one_based_numbering=True, week_start=1):

Review comment:
       ```suggestion
       def __init__(self, one_based_numbering=False, week_start=1):
   ```
   
   The default is False?

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -451,9 +517,14 @@ const FunctionDoc day_doc{
 
 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"}};
+    ("By default Week starts on Monday represented by 0 and ends on Sunday represented "
+     "by 6.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null.\n"
+     "DayOfWeekOptions.week_start can be used to set another starting day using ISO "

Review comment:
       Can you move this up to put it directly after the "By default, the week starts with ..", so before the "Returns an error if ...". Since this explanation of the options directly adds to the default monday-sunday as 0-6, putting the other sentence in between breaks a bit the logical flow I think

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -512,12 +517,14 @@ const FunctionDoc day_doc{
 
 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"
+    ("By default Week starts on Monday represented by 0 and ends on Sunday represented "

Review comment:
       ```suggestion
       ("By default, the week starts on Monday represented by 0 and ends on Sunday represented "
   ```




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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


   @pitrou ping :)


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -136,19 +136,25 @@ struct Day {
 
 // ----------------------------------------------------------------------
 // Extract day of week from timestamp
+//
+// By default week starts on Monday represented by 0 and ends on Sunday represented
+// by 6. Start day of the week (Monday=1, Sunday=7) and numbering start (0 or 1) can be
+// set using DayOfWeekOptions
 
 template <typename Duration>
 struct DayOfWeek {
-  explicit DayOfWeek(TemporalComponentExtractionOptions options) : options(options) {}
+  explicit DayOfWeek(DayOfWeekOptions options) : options(options) {}
 
   template <typename T, typename Arg0>
   T Call(KernelContext*, Arg0 arg, Status*) const {
-    return static_cast<T>(
-        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
-            .iso_encoding() -
-        1 + options.week_start);
+    auto wd = arrow_vendored::date::year_month_weekday(
+                  floor<days>(sys_time<Duration>(Duration{arg})))
+                  .weekday()
+                  .iso_encoding();
+    return (wd + 7 - options.week_start) % 7 + options.one_based_numbering;

Review comment:
       If we are worried about the modulo cost, we could instead use a simple condition:
   ```c++
   const auto shifted_wd = wd + 7 - options.week_start;
   return options.one_based_numbering + ((shifted_wd > 7) ? shifted_wd - 7 : shifted_wd);
   ```




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: r/src/compute.cpp
##########
@@ -264,6 +264,12 @@ std::shared_ptr<arrow::compute::FunctionOptions> make_compute_options(
                                      max_replacements);
   }
 
+  if (func_name == "day_of_week") {
+    using Options = arrow::compute::TemporalComponentExtractionOptions;
+    return std::make_shared<Options>(cpp11::as_cpp<bool>(options["one_based_numbering"]),

Review comment:
       Would users ever use it? If yes then I'd add it.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] thisisnic commented on pull request #10598: ARROW-13054: [C++] Add TemporalOptions

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


   > > About the actual options in this PR: that seems like a good start, but I don't think all those options would be needed in a single Options struct? For example, the ambiguous/non-existent handling, those can be put in a specific options struct for the kernel that needs it?
   > 
   > I'm ok with single Options struct or multiple.
   > If we go for multiple we'd probably split this into `TemporalStrftimeOptions` ([ARROW-13174](https://issues.apache.org/jira/browse/ARROW-13174)), `TemporalComponentExtractionOptions`, `TemporalLocalizationOptions` ([ARROW-13033](https://issues.apache.org/jira/browse/ARROW-13033)). Am I missing something?
   
   Those make sense to me.  I think there may be other Options structs we might need; for example, ones for kernels that do maths with dates. However, we don't know to know all of these in advance, right?
   
   


-- 
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 #10598: ARROW-13054: [C++] Add TemporalComponentExtractionOptions

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


   @rok you mention this PR now only has `index_of_monday`, but in the diff I currently see `start_index`, which is something else? 
   Also the explanation of it, "Index of the first day of the week", is a bit ambiguous (what is the first day of the 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok edited a comment on pull request #10598: ARROW-13054: [C++] Add TemporalOptions

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


   > However, we don't need to know all of these in advance, right?
   
   Indeed. Even the ones I listed above should be implemented just-in-time with their respective kernels IMO. So I'll only implement `TemporalComponentExtractionOptions` here.
   Meaning we only get `index_of_monday` parameter from this PR.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -102,6 +102,44 @@ TEST(ScalarTemporalTest, TestTemporalComponentExtraction) {
   CheckScalarUnary("microsecond", unit, times, int64(), microsecond);
   CheckScalarUnary("nanosecond", unit, times, int64(), nanosecond);
   CheckScalarUnary("subsecond", unit, times, float64(), subsecond);
+
+  auto timestamps = ArrayFromJSON(unit, times);

Review comment:
       Done :).




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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


   Thanks for the review @pitrou! I've implemented the suggestions.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] thisisnic commented on pull request #10598: ARROW-13054: [C++] Add TemporalOptions

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


   > > * `fiscal_start` - this is an argument to `quarter` which indicates the starting month of a fiscal year - I think this would be useful to implement
   > 
   > Is this just an arbitrary month user sets or is there a standard?
   
   The default value in `lubridate` is 1 (the calendar year quarter), and I think it would make sense for that to be the default unless otherwise specified.
   
   > I was wondering about locale as well. Would it be a good idea to be able to set an arbitrary locale in some cases? [Pandas appears not to offer this option](https://pandas.pydata.org/docs/reference/api/pandas.Series.dt.dayofweek.html). It would be useful for strftime but could cause misinterpretations for say day_of_week.
   
   Again, I'd like input from others on this (I am not experienced in this and may be missing something!), but my initial thoughts are that if we do choose to extract the day of week and month as integers only (and not implement the `label` argument), then all the locale stuff can be handled by `strftime` in terms of converting it from an integer to a text representation, so the locale is not relevant to TemporalOptions specifically within the context of extracting components from datetime objects.
   
   > Added [strftime jira](https://issues.apache.org/jira/browse/ARROW-13174).
   
   Thanks!
   
   


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

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



[GitHub] [arrow] pitrou commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: r/R/dplyr-functions.R
##########
@@ -526,14 +526,6 @@ nse_funcs$second <- function(x) {
   Expression$create("add", Expression$create("second", x), Expression$create("subsecond", x))
 }
 
-# After ARROW-13054 is completed, we can refactor this for simplicity

Review comment:
       That assumes you're using the same options class for all these kernels, which seems a bit weird to me (why would the `year` kernel take unrelated options pertaining to day-of-week numbering?).
   
   cc @bkietz for a second opinion on this.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: r/src/compute.cpp
##########
@@ -264,6 +264,12 @@ std::shared_ptr<arrow::compute::FunctionOptions> make_compute_options(
                                      max_replacements);
   }
 
+  if (func_name == "day_of_week") {
+    using Options = arrow::compute::TemporalComponentExtractionOptions;
+    return std::make_shared<Options>(cpp11::as_cpp<bool>(options["one_based_numbering"]),

Review comment:
       https://github.com/apache/arrow/pull/10598/commits/bf4ab293a4353e7e357030dc5675baedb042e5ec - does this look right?




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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


   > @rok You'll need to rebase and adapt the code to the recent changes in `FunctionOptions`.
   
   Ah, finally I am getting the same errors locally as in 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: docs/source/cpp/compute.rst
##########
@@ -954,44 +954,46 @@ 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.
++--------------------+------------+-------------------+---------------+--------+----------------------------+
+| Function name      | Arity      | Input types       | Output type   | Notes  | Options class              |
++====================+============+===================+===============+========+============================+
+| year               | Unary      | Temporal          | Int64         |        |                            |
++--------------------+------------+-------------------+---------------+--------+----------------------------+
+| month              | Unary      | Temporal          | Int64         |        |                            |
++--------------------+------------+-------------------+---------------+--------+----------------------------+
+| day                | Unary      | Temporal          | Int64         |        |                            |
++--------------------+------------+-------------------+---------------+--------+----------------------------+
+| day_of_week        | Unary      | Temporal          | Int64         | \(1)   | :struct:`DayOfWeekOptions` |
++--------------------+------------+-------------------+---------------+--------+----------------------------+
+| 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. By default week begins on Monday
+  represented by 0 and ends on Sunday represented by 6. :member:`DayOfWeekOptions::week_start` can be used to set
+  the starting day of the week using ISO convention (Monday=1, Sunday=7). Day numbering can start with 0 or 1
+  using :member:`DayOfWeekOptions::week_start` parameter.

Review comment:
       Probably not `week_start` here.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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


   @rok Don't hesitate to ping when this is ready for review again.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] jorisvandenbossche commented on pull request #10598: ARROW-13054: [C++] Add TemporalOptions

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


   > I was wondering about locale as well. Would it be a good idea to be able to set an arbitrary locale in some cases? [Pandas appears not to offer this option](https://pandas.pydata.org/docs/reference/api/pandas.Series.dt.dayofweek.html). It would be useful for strftime but could cause misinterpretations for say day_of_week.
   
   Pandas actually has a different method (the one you link is an attribute that has been there for a long time, so but pandas added an additional method so it could have arguments) that takes a locale arguments: https://pandas.pydata.org/docs/reference/api/pandas.Series.dt.day_name.html
   
   Personally, I would (at least for now) leave all locale-specific handling to the bindings / downstream applications (except for `strftime`). As Nic says, if we only return integers (and not `labels`) for now, we shouldn't need locale handling (I wouldn't let the number depend on the locale, that seems very easy to miss).
   
   ----
   
   About the actual options in this PR: that seems like a good start, but I don't think all those options would be needed in a single Options struct? For example, the ambiguous/non-existent handling, those can be put in a specific options struct for the kernel that needs it?
   
   


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

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



[GitHub] [arrow] pitrou commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -624,15 +636,20 @@ 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.
+/// `values`. By default week starts on Monday denoted by 0 and ends on Sunday denoted

Review comment:
       Nit, but the `\brief` line should be a one-line summary. Leave an empty line before the more detailed description.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] thisisnic commented on pull request #10598: ARROW-13054: [C++] Add TemporalComponentExtractionOptions

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


   @rok Sounds good to me.  Let me know if you want me to take a look at the code that will need to be added to fix the R builds which are now failing because you've implemented those options.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -624,15 +636,20 @@ 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.
+/// `values`. By default week starts on Monday denoted by 0 and ends on Sunday denoted

Review comment:
       Done.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: docs/source/cpp/compute.rst
##########
@@ -988,44 +988,46 @@ 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.
++--------------------+------------+-------------------+---------------+--------+----------------------------+
+| Function name      | Arity      | Input types       | Output type   | Notes  | Options class              |
++====================+============+===================+===============+========+============================+
+| year               | Unary      | Temporal          | Int64         |        |                            |
++--------------------+------------+-------------------+---------------+--------+----------------------------+
+| month              | Unary      | Temporal          | Int64         |        |                            |
++--------------------+------------+-------------------+---------------+--------+----------------------------+
+| day                | Unary      | Temporal          | Int64         |        |                            |
++--------------------+------------+-------------------+---------------+--------+----------------------------+
+| day_of_week        | Unary      | Temporal          | Int64         | \(1)   | :struct:`DayOfWeekOptions` |

Review comment:
       Indeed. Fixed.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -216,6 +216,19 @@ struct ARROW_EXPORT ProjectOptions : public FunctionOptions {
   std::vector<std::shared_ptr<const KeyValueMetadata>> field_metadata;
 };
 
+struct ARROW_EXPORT DayOfWeekOptions : public FunctionOptions {
+  explicit DayOfWeekOptions(bool one_based_numbering, uint32_t week_start)
+      : one_based_numbering(one_based_numbering), week_start(week_start) {}

Review comment:
       I've added this:
   ```C++
   if (week_start < 1 || 7 < week_start) {
     week_start = 1;
   }
   ```
   
   However raising would be better I suppose?
   ```C++
   throw std::invalid_argument("week_start must follow ISO convention (Monday=1, Sunday=7). Got week_start=" + std::to_string(week_start));
   ```




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -102,6 +102,44 @@ TEST(ScalarTemporalTest, TestTemporalComponentExtraction) {
   CheckScalarUnary("microsecond", unit, times, int64(), microsecond);
   CheckScalarUnary("nanosecond", unit, times, int64(), nanosecond);
   CheckScalarUnary("subsecond", unit, times, float64(), subsecond);
+
+  auto timestamps = ArrayFromJSON(unit, times);

Review comment:
       Instead of putting all checks in a single test function, can you define a test class that initializes the test vectors and then define separate test "methods" (using `TEST_F`)?




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on pull request #10598: ARROW-13054: [C++] Add TemporalComponentExtractionOptions

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


   Thanks for the input @thisisnic & @jorisvandenbossche!
   
   I've refactored this to use `week_start` and `one_based_numbering` parameters to control the starting day and `Monday=0` vs `Monday=1`. I've also updated documentation.
   Please check if this now 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: r/R/dplyr-functions.R
##########
@@ -526,14 +526,6 @@ nse_funcs$second <- function(x) {
   Expression$create("add", Expression$create("second", x), Expression$create("subsecond", x))
 }
 
-# After ARROW-13054 is completed, we can refactor this for simplicity

Review comment:
       I don't know if you want to do it, but the Python side probably needs to be updated in order to expose the options class.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -80,6 +84,25 @@ struct TemporalComponentExtract {
   }
 };
 
+template <typename Op, typename OutType>
+struct DayOfWeekOptionsWithOptions {

Review comment:
       Indeed. `DayOfWeekWithOptions`?




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -118,16 +141,31 @@ struct Day {
 
 // ----------------------------------------------------------------------
 // Extract day of week from timestamp
+//
+// By default week starts on Monday represented by 0 and ends on Sunday represented
+// by 6. Start day of the week (Monday=1, Sunday=7) and numbering start (0 or 1) can be
+// set using DayOfWeekOptions
 
 template <typename Duration>
 struct DayOfWeek {
+  explicit DayOfWeek(DayOfWeekOptions options) : options(options) {

Review comment:
       Done.

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -118,16 +141,31 @@ struct Day {
 
 // ----------------------------------------------------------------------
 // Extract day of week from timestamp
+//
+// By default week starts on Monday represented by 0 and ends on Sunday represented
+// by 6. Start day of the week (Monday=1, Sunday=7) and numbering start (0 or 1) can be
+// set using DayOfWeekOptions
 
 template <typename Duration>
 struct DayOfWeek {
+  explicit DayOfWeek(DayOfWeekOptions options) : options(options) {
+    for (int i = 0; i < 7; i++) {
+      lookup_table[i] = i + 8 - options.week_start;
+      lookup_table[i] = (lookup_table[i] > 6) ? lookup_table[i] - 7 : lookup_table[i];
+      lookup_table[i] += options.one_based_numbering;
+    }
+  }
+
   template <typename T, typename Arg0>
-  static T Call(KernelContext*, Arg0 arg, Status*) {
-    return static_cast<T>(
-        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
-            .iso_encoding() -
-        1);
+  T Call(KernelContext*, Arg0 arg, Status*) const {
+    const auto wd = arrow_vendored::date::year_month_weekday(
+                        floor<days>(sys_time<Duration>(Duration{arg})))
+                        .weekday()
+                        .iso_encoding();
+    return lookup_table[wd - 1];
   }
+  std::array<int64_t, 7> lookup_table;
+  DayOfWeekOptions options;

Review comment:
       Indeed! Removed.

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -80,6 +84,25 @@ struct TemporalComponentExtract {
   }
 };
 
+template <typename Op, typename OutType>
+struct DayOfWeekExec {
+  using OutValue = typename internal::GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    DayOfWeekOptions options = DayOfWeekState::Get(ctx);

Review comment:
       Done.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: r/src/compute.cpp
##########
@@ -264,6 +264,12 @@ std::shared_ptr<arrow::compute::FunctionOptions> make_compute_options(
                                      max_replacements);
   }
 
+  if (func_name == "day_of_week") {
+    using Options = arrow::compute::TemporalComponentExtractionOptions;
+    return std::make_shared<Options>(cpp11::as_cpp<bool>(options["one_based_numbering"]),

Review comment:
       Sure!




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -136,19 +136,25 @@ struct Day {
 
 // ----------------------------------------------------------------------
 // Extract day of week from timestamp
+//
+// By default week starts on Monday represented by 0 and ends on Sunday represented
+// by 6. Start day of the week (Monday=1, Sunday=7) and numbering start (0 or 1) can be
+// set using DayOfWeekOptions
 
 template <typename Duration>
 struct DayOfWeek {
-  explicit DayOfWeek(TemporalComponentExtractionOptions options) : options(options) {}
+  explicit DayOfWeek(DayOfWeekOptions options) : options(options) {}
 
   template <typename T, typename Arg0>
   T Call(KernelContext*, Arg0 arg, Status*) const {
-    return static_cast<T>(
-        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
-            .iso_encoding() -
-        1 + options.week_start);
+    auto wd = arrow_vendored::date::year_month_weekday(
+                  floor<days>(sys_time<Duration>(Duration{arg})))
+                  .weekday()
+                  .iso_encoding();
+    return (wd + 7 - options.week_start) % 7 + options.one_based_numbering;

Review comment:
       This is on my TODO, will get back to it asap.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok edited a comment on pull request #10598: ARROW-13054: [C++] Add TemporalOptions

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


   Hey @thisisnic! Thanks for the input :).
   
   > * `fiscal_start` - this is an argument to `quarter` which indicates the starting month of a fiscal year - I think this would be useful to implement
   
   Is this just an arbitrary month user sets or is there a standard?
   
   > * `with_year` - an argument to `quarter` which defines whether the returned value is just the quarter or the quarter _and_ the year - I don't think we need this as it makes the outputs ambiguous and can be achieved at the R layer
   > * `label` - an argument to `month` and `wday` (aka `day_of_week` in Arrow) which defines whether the returned value is an integer or character representation of the month - as discussed on https://issues.apache.org/jira/browse/ARROW-13133, this can be achieved via a separate `strftime` kernel
   > * `abbr` - works with `label` and determines whether the label is abbreviated - I think this belongs to the `strftime` kernel
   > * `locale` - locale to use for month/week day names if `label` is set to `TRUE`- I think this belongs to the `strftime` kernel
   
   I was wondering about locale as well. Would it be a good idea to be able to set an arbitrary locale in some cases? [Pandas appears not to offer this option](https://pandas.pydata.org/docs/reference/api/pandas.Series.dt.dayofweek.html). It would be useful for strftime but could cause misinterpretations for say day_of_week.
   
   Added [strftime jira](https://issues.apache.org/jira/browse/ARROW-13174).
    
   
   


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

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



[GitHub] [arrow] rok commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -136,19 +136,25 @@ struct Day {
 
 // ----------------------------------------------------------------------
 // Extract day of week from timestamp
+//
+// By default week starts on Monday represented by 0 and ends on Sunday represented
+// by 6. Start day of the week (Monday=1, Sunday=7) and numbering start (0 or 1) can be
+// set using DayOfWeekOptions
 
 template <typename Duration>
 struct DayOfWeek {
-  explicit DayOfWeek(TemporalComponentExtractionOptions options) : options(options) {}
+  explicit DayOfWeek(DayOfWeekOptions options) : options(options) {}
 
   template <typename T, typename Arg0>
   T Call(KernelContext*, Arg0 arg, Status*) const {
-    return static_cast<T>(
-        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
-            .iso_encoding() -
-        1 + options.week_start);
+    auto wd = arrow_vendored::date::year_month_weekday(
+                  floor<days>(sys_time<Duration>(Duration{arg})))
+                  .weekday()
+                  .iso_encoding();
+    return (wd + 7 - options.week_start) % 7 + options.one_based_numbering;

Review comment:
       I'm not sure about this  either. Added a `options.week_start == 1` branch.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -102,6 +102,44 @@ TEST(ScalarTemporalTest, TestTemporalComponentExtraction) {
   CheckScalarUnary("microsecond", unit, times, int64(), microsecond);
   CheckScalarUnary("nanosecond", unit, times, int64(), nanosecond);
   CheckScalarUnary("subsecond", unit, times, float64(), subsecond);
+
+  auto timestamps = ArrayFromJSON(unit, times);

Review comment:
       Can now add a `TEST_F(ScalarTemporalTest, DayOfWeek)` :-)




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] thisisnic edited a comment on pull request #10598: ARROW-13054: [C++] Add TemporalOptions

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


   > Is this missing an option we will need in the near future? Should we remove something.
   > @jorisvandenbossche @thisisnic
   
   Thanks for this! I've taken a look at the `lubridate` package in R, and in terms of options that are implemented in `lubridate` (specifically for functions which are used for extracting components of date-time objects as I am assuming that is the scope of this - correct me if I'm wrong though!) but not here, there is:
   
   * `fiscal_start` - this is an argument to `quarter` which indicates the starting month of a fiscal year - I think this would be useful to implement
   
   * `with_year` - an argument to `quarter` which defines whether the returned value is just the quarter or the quarter *and* the year - I'm not sure about this one - it feels like it's doing something that can be achieved by combining the year/quarter returned values but if it's  widely used in other implementations, perhaps we should consider it?
   
   * `label` - an argument to `month` and `wday` (aka `day_of_week` in Arrow) which defines whether the returned value is an integer or character representation of the month - as discussed on https://issues.apache.org/jira/browse/ARROW-13133, this can be achieved via a separate `strftime` kernel
   
   * `abbr` - works with `label` and determines whether the label is abbreviated - I think this belongs to the `strftime` kernel
   
   * `locale` - locale to use for month/week day names if `label` is set to `TRUE`- I think this belongs to the `strftime` kernel
   
   So, yeah, I'm thinking `fiscal_start` is the only other accessor option that would definitely be good to add, but it'd be good to get input from @nealrichardson on 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] thisisnic commented on pull request #10598: ARROW-13054: [C++] Add TemporalComponentExtractionOptions

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






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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on pull request #10598: ARROW-13054: [C++] Add TemporalOptions

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


   Is this missing an option we will need in the near future? Should we remove something.
   @jorisvandenbossche @thisisnic


-- 
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] thisisnic edited a comment on pull request #10598: ARROW-13054: [C++] Add TemporalComponentExtractionOptions

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


   > > @rok you mention this PR now only has `index_of_monday`, but in the diff I currently see `start_index`, which is something else?
   > > Also the explanation of it, "Index of the first day of the week", is a bit ambiguous (what is the first day of the week?)
   > 
   > @jorisvandenbossche well the beginning of the week could be e.g.: `Monday=0`, `Monday=1`, `Sunday=0` or `Sunday=1`. Meaning we really need two parameters:
   > 
   >     * which day (Sunday/Monday) starts the week - this is normally (eventually) given by locale but in the scope of this PR it's Monday.
   > 
   >     * what index does the week start at - this would here be set by `start_index`
   > 
   > 
   > Alternatively we can have `index_of_monday` and calculate kinda like `(day_of_week(timestamp) + index_of_monday) % 7`.
   > 
   > How about `"Index of the first day of the week"` to `"Offset of day of week"`?
   
   I'm not sure if this might be a little confusing combining them.  Could we perhaps define ourselves which integer maps to which day, and then just have a single parameter, `start_index`, similarly to how `week_start` works in R's lubridate package (https://lubridate.tidyverse.org/reference/day.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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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


   Thanks for the review @pitrou! I've implemented the suggestions.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: r/src/compute.cpp
##########
@@ -264,6 +264,16 @@ std::shared_ptr<arrow::compute::FunctionOptions> make_compute_options(
                                      max_replacements);
   }
 
+  if (func_name == "day_of_week") {
+    using Options = arrow::compute::TemporalComponentExtractionOptions;

Review comment:
       Ah missed that one! :)




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] jorisvandenbossche commented on pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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


   Thanks!


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: r/src/compute.cpp
##########
@@ -264,6 +264,12 @@ std::shared_ptr<arrow::compute::FunctionOptions> make_compute_options(
                                      max_replacements);
   }
 
+  if (func_name == "day_of_week") {
+    using Options = arrow::compute::TemporalComponentExtractionOptions;
+    return std::make_shared<Options>(cpp11::as_cpp<bool>(options["one_based_numbering"]),

Review comment:
       I added your suggestion just in case :).




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on pull request #10598: ARROW-13054: [C++] Add TemporalComponentExtractionOptions

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






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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] github-actions[bot] commented on pull request #10598: ARROW-13054: [C++] Add TemporalOptions

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


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


-- 
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 #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: docs/source/cpp/compute.rst
##########
@@ -954,44 +954,46 @@ 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.
++--------------------+------------+-------------------+---------------+--------+----------------------------+
+| Function name      | Arity      | Input types       | Output type   | Notes  | Options class              |
++====================+============+===================+===============+========+============================+
+| year               | Unary      | Temporal          | Int64         |        |                            |
++--------------------+------------+-------------------+---------------+--------+----------------------------+
+| month              | Unary      | Temporal          | Int64         |        |                            |
++--------------------+------------+-------------------+---------------+--------+----------------------------+
+| day                | Unary      | Temporal          | Int64         |        |                            |
++--------------------+------------+-------------------+---------------+--------+----------------------------+
+| day_of_week        | Unary      | Temporal          | Int64         | \(1)   | :struct:`DayOfWeekOptions` |
++--------------------+------------+-------------------+---------------+--------+----------------------------+
+| 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. By default week begins on Monday
+  represented by 0 and ends on Sunday represented by 6. :member:`DayOfWeekOptions::week_start` can be used to set
+  the starting day of the week using ISO convention (Monday=1, Sunday=7). Day numbering can start with 0 or 1
+  using :member:`DayOfWeekOptions::week_start` parameter.

Review comment:
       Indeed. Fixed.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
##########
@@ -102,6 +102,44 @@ TEST(ScalarTemporalTest, TestTemporalComponentExtraction) {
   CheckScalarUnary("microsecond", unit, times, int64(), microsecond);
   CheckScalarUnary("nanosecond", unit, times, int64(), nanosecond);
   CheckScalarUnary("subsecond", unit, times, float64(), subsecond);
+
+  auto timestamps = ArrayFromJSON(unit, times);

Review comment:
       Done.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] jorisvandenbossche closed pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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


   


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] bkietz commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: r/src/compute.cpp
##########
@@ -264,6 +264,16 @@ std::shared_ptr<arrow::compute::FunctionOptions> make_compute_options(
                                      max_replacements);
   }
 
+  if (func_name == "day_of_week") {
+    using Options = arrow::compute::TemporalComponentExtractionOptions;

Review comment:
       ```suggestion
       using Options = arrow::compute::DayOfWeekOptions;
   ```




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -451,9 +517,14 @@ const FunctionDoc day_doc{
 
 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"}};
+    ("By default Week starts on Monday represented by 0 and ends on Sunday represented "
+     "by 6.\n"
+     "Returns an error if timestamp has a defined timezone. Null values return null.\n"
+     "DayOfWeekOptions.week_start can be used to set another starting day using ISO "

Review comment:
       Done. Please check if it looks right https://github.com/apache/arrow/pull/10598/commits/db63b17be020bd686f2c4d4494fe04b7e3175d99#diff-53850537a2f3f7977473b30241fdd3ddb35aa97269284c06471d9b0c599e2c59.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] thisisnic edited a comment on pull request #10598: ARROW-13054: [C++] Add TemporalComponentExtractionOptions

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


   > > @rok you mention this PR now only has `index_of_monday`, but in the diff I currently see `start_index`, which is something else?
   > > Also the explanation of it, "Index of the first day of the week", is a bit ambiguous (what is the first day of the week?)
   > 
   > @jorisvandenbossche well the beginning of the week could be e.g.: `Monday=0`, `Monday=1`, `Sunday=0` or `Sunday=1`. Meaning we really need two parameters:
   > 
   >     * which day (Sunday/Monday) starts the week - this is normally (eventually) given by locale but in the scope of this PR it's Monday.
   > 
   >     * what index does the week start at - this would here be set by `start_index`
   > 
   > 
   > Alternatively we can have `index_of_monday` and calculate kinda like `(day_of_week(timestamp) + index_of_monday) % 7`.
   > 
   > How about `"Index of the first day of the week"` to `"Offset of day of week"`?
   
   I'm not sure if this might be a little confusing combining them.  Could we perhaps define ourselves which integer maps to which day, and then just have a single parameter, `start_index`, similarly to how `week_start` works in R's lubridate package (https://lubridate.tidyverse.org/reference/day.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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] jorisvandenbossche commented on pull request #10598: ARROW-13054: [C++] Add TemporalComponentExtractionOptions

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


   @rok it's still unclear to me what the `week_start` option in the PR now means / does (eg how should the "Index of the first day" be expressed. A number relative to what?). Please try to make the docstring more explicit.
   
   There are two aspects that could be controlled:
   - On which day do we start counting (eg typically Sunday vs Monday) -> "start day"
   - At which number do we start counting (eg typically 0 vs 1) -> "start index"
   
   ISO starts counting at 1 on Monday, C++ starts counting at 0 on Sunday (but `date.h` has a `c_encoding()` and `iso_encoding()` to get the other), lubridate starts counting at 1 on Sunday, Python starts counting at 0 on Monday,  (to cover all the variations .. ;-))
   
   lubridate's `week_start` actually covers the start day (first point, on which day do we start counting, for the rest it always uses numbers 1-7)). While I *think* what you implemented here covers the start index (the second point, at which number to start counting)? 
   
   Taking a step back: for the "start index", would users ever want something else as 0 or 1 ? (I don't think that counting from 2 to 8 would ever make sense). In addition, going from 0-indexed to 1-indexed is a simple operation (addition +1). On the other hand, the "start day" is more difficult to change afterwards (since you need to wrap around, not a simple +1,  but eg 7 needs to become 0). So if we want to add some option, "start day" seems the more useful one to add?


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: r/R/dplyr-functions.R
##########
@@ -526,14 +526,6 @@ nse_funcs$second <- function(x) {
   Expression$create("add", Expression$create("second", x), Expression$create("subsecond", x))
 }
 
-# After ARROW-13054 is completed, we can refactor this for simplicity

Review comment:
       Done.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -216,6 +216,22 @@ struct ARROW_EXPORT ProjectOptions : public FunctionOptions {
   std::vector<std::shared_ptr<const KeyValueMetadata>> field_metadata;
 };
 
+struct ARROW_EXPORT TemporalComponentExtractionOptions : public FunctionOptions {

Review comment:
       Done.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: r/R/dplyr-functions.R
##########
@@ -526,14 +526,6 @@ nse_funcs$second <- function(x) {
   Expression$create("add", Expression$create("second", x), Expression$create("subsecond", x))
 }
 
-# After ARROW-13054 is completed, we can refactor this for simplicity

Review comment:
       It's ready in [ARROW-12980](https://github.com/apache/arrow/pull/10457/files#diff-2010b86c11cb90f0ce6ef00648df2e388d11cda5d3fc74e4293dd862e228daec). Would it make sense to copy-paste it here too?




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on pull request #10598: ARROW-13054: [C++] Add TemporalComponentExtractionOptions

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


   > @rok you mention this PR now only has `index_of_monday`, but in the diff I currently see `start_index`, which is something else?
   > Also the explanation of it, "Index of the first day of the week", is a bit ambiguous (what is the first day of the week?)
   
   @jorisvandenbossche well the beginning of the week could be e.g.: `Monday=0`, `Monday=1`, `Sunday=0` or `Sunday=1`. Meaning we really need two parameters:
   * which day (Sunday/Monday) starts the week - this is normally (eventually) given by locale but in the scope of this PR it's Monday.
   * what index does the week start at - this would here be set by `start_index`
   
   Alternatively we can have `index_of_monday` and calculate kinda like `(day_of_week(timestamp) + index_of_monday) % 7`.
   
   How about `"Index of the first day of the week"` to `"Offset of 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on pull request #10598: ARROW-13054: [C++] Add TemporalComponentExtractionOptions

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


   @thisisnic I like `week_start`. So then week starts on Monday and we have only have `week_start` as  a parameter.
   The Sunday vs Monday week start can be revisited later.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on pull request #10598: ARROW-13054: [C++] Add TemporalOptions

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


   > Again, I'd like input from others on this (I am not experienced in this and may be missing something!), but my initial thoughts are that if we do choose to extract the day of week and month as integers only (and not implement the `label` argument), then all the locale stuff can be handled by `strftime` in terms of converting it from an integer to a text representation, so the locale is not relevant to TemporalOptions specifically within the context of extracting components from datetime objects.
   
   Well my thinking is to enable this `day_of_week(timestamp_monday, "en_US") == 2` and `day_of_week(timestamp_monday, "en_GB") == 1`. I don't know how useful this is but it looks like [matlab is doing it in this way](https://www.mathworks.com/help/matlab/ref/weekday.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 #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: r/src/compute.cpp
##########
@@ -264,6 +264,12 @@ std::shared_ptr<arrow::compute::FunctionOptions> make_compute_options(
                                      max_replacements);
   }
 
+  if (func_name == "day_of_week") {
+    using Options = arrow::compute::TemporalComponentExtractionOptions;
+    return std::make_shared<Options>(cpp11::as_cpp<bool>(options["one_based_numbering"]),

Review comment:
       https://github.com/apache/arrow/pull/10598/commits/3539634894c962f463b3947c03b9e0be135061ee




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok removed a comment on pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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


   > @rok You'll need to rebase and adapt the code to the recent changes in `FunctionOptions`.
   
   Ah, finally I am getting the same errors locally as in 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -216,6 +216,19 @@ struct ARROW_EXPORT ProjectOptions : public FunctionOptions {
   std::vector<std::shared_ptr<const KeyValueMetadata>> field_metadata;
 };
 
+struct ARROW_EXPORT DayOfWeekOptions : public FunctionOptions {
+  explicit DayOfWeekOptions(bool one_based_numbering, uint32_t week_start)
+      : one_based_numbering(one_based_numbering), week_start(week_start) {}

Review comment:
       Moved the check to `Exec` and added some tests.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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


   > @rok You'll need to rebase and adapt the code to the recent changes in `FunctionOptions`.
   
   Ah, finally I am getting the same errors locally as in 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -451,9 +524,15 @@ const FunctionDoc day_doc{
 
 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"
+    ("By default, the week starts on Monday represented by 0 and ends on Sunday "
+     "represented "
+     "by 6.\n"

Review comment:
       ```suggestion
        "represented by 6.\n"
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -35,6 +36,7 @@ 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::time_zone;

Review comment:
       This doesn't need to be added here?

##########
File path: docs/source/cpp/compute.rst
##########
@@ -988,44 +988,46 @@ 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.
++--------------------+------------+-------------------+---------------+--------+----------------------------+
+| Function name      | Arity      | Input types       | Output type   | Notes  | Options class              |
++====================+============+===================+===============+========+============================+
+| year               | Unary      | Temporal          | Int64         |        |                            |
++--------------------+------------+-------------------+---------------+--------+----------------------------+
+| month              | Unary      | Temporal          | Int64         |        |                            |
++--------------------+------------+-------------------+---------------+--------+----------------------------+
+| day                | Unary      | Temporal          | Int64         |        |                            |
++--------------------+------------+-------------------+---------------+--------+----------------------------+
+| day_of_week        | Unary      | Temporal          | Int64         | \(1)   | :struct:`DayOfWeekOptions` |

Review comment:
       Tiny remark, but can you switch the order of the Notes and Option class columns (so the Notes is the last column). We just had a PR that made this consistent throughout the full document (#10630), so good to keep it consistent now.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -118,16 +141,31 @@ struct Day {
 
 // ----------------------------------------------------------------------
 // Extract day of week from timestamp
+//
+// By default week starts on Monday represented by 0 and ends on Sunday represented
+// by 6. Start day of the week (Monday=1, Sunday=7) and numbering start (0 or 1) can be
+// set using DayOfWeekOptions
 
 template <typename Duration>
 struct DayOfWeek {
+  explicit DayOfWeek(DayOfWeekOptions options) : options(options) {

Review comment:
       Nit: can you usually pass options as `const&`.

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -80,6 +84,25 @@ struct TemporalComponentExtract {
   }
 };
 
+template <typename Op, typename OutType>
+struct DayOfWeekExec {
+  using OutValue = typename internal::GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    DayOfWeekOptions options = DayOfWeekState::Get(ctx);

Review comment:
       Nit: could be `const&`.

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -118,16 +141,31 @@ struct Day {
 
 // ----------------------------------------------------------------------
 // Extract day of week from timestamp
+//
+// By default week starts on Monday represented by 0 and ends on Sunday represented
+// by 6. Start day of the week (Monday=1, Sunday=7) and numbering start (0 or 1) can be
+// set using DayOfWeekOptions
 
 template <typename Duration>
 struct DayOfWeek {
+  explicit DayOfWeek(DayOfWeekOptions options) : options(options) {
+    for (int i = 0; i < 7; i++) {
+      lookup_table[i] = i + 8 - options.week_start;
+      lookup_table[i] = (lookup_table[i] > 6) ? lookup_table[i] - 7 : lookup_table[i];
+      lookup_table[i] += options.one_based_numbering;
+    }
+  }
+
   template <typename T, typename Arg0>
-  static T Call(KernelContext*, Arg0 arg, Status*) {
-    return static_cast<T>(
-        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
-            .iso_encoding() -
-        1);
+  T Call(KernelContext*, Arg0 arg, Status*) const {
+    const auto wd = arrow_vendored::date::year_month_weekday(
+                        floor<days>(sys_time<Duration>(Duration{arg})))
+                        .weekday()
+                        .iso_encoding();
+    return lookup_table[wd - 1];
   }
+  std::array<int64_t, 7> lookup_table;
+  DayOfWeekOptions options;

Review comment:
       This member doesn't seem used anymore?




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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


   @jorisvandenbossche Thanks for the review! I've pushed the suggested changes.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -35,6 +36,7 @@ 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::time_zone;

Review comment:
       Oh yeah. Not yet :)




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -118,16 +141,31 @@ struct Day {
 
 // ----------------------------------------------------------------------
 // Extract day of week from timestamp
+//
+// By default week starts on Monday represented by 0 and ends on Sunday represented
+// by 6. Start day of the week (Monday=1, Sunday=7) and numbering start (0 or 1) can be
+// set using DayOfWeekOptions
 
 template <typename Duration>
 struct DayOfWeek {
+  explicit DayOfWeek(DayOfWeekOptions options) : options(options) {

Review comment:
       Done.

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -118,16 +141,31 @@ struct Day {
 
 // ----------------------------------------------------------------------
 // Extract day of week from timestamp
+//
+// By default week starts on Monday represented by 0 and ends on Sunday represented
+// by 6. Start day of the week (Monday=1, Sunday=7) and numbering start (0 or 1) can be
+// set using DayOfWeekOptions
 
 template <typename Duration>
 struct DayOfWeek {
+  explicit DayOfWeek(DayOfWeekOptions options) : options(options) {
+    for (int i = 0; i < 7; i++) {
+      lookup_table[i] = i + 8 - options.week_start;
+      lookup_table[i] = (lookup_table[i] > 6) ? lookup_table[i] - 7 : lookup_table[i];
+      lookup_table[i] += options.one_based_numbering;
+    }
+  }
+
   template <typename T, typename Arg0>
-  static T Call(KernelContext*, Arg0 arg, Status*) {
-    return static_cast<T>(
-        weekday(year_month_day(floor<days>(sys_time<Duration>(Duration{arg}))))
-            .iso_encoding() -
-        1);
+  T Call(KernelContext*, Arg0 arg, Status*) const {
+    const auto wd = arrow_vendored::date::year_month_weekday(
+                        floor<days>(sys_time<Duration>(Duration{arg})))
+                        .weekday()
+                        .iso_encoding();
+    return lookup_table[wd - 1];
   }
+  std::array<int64_t, 7> lookup_table;
+  DayOfWeekOptions options;

Review comment:
       Indeed! Removed.

##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -80,6 +84,25 @@ struct TemporalComponentExtract {
   }
 };
 
+template <typename Op, typename OutType>
+struct DayOfWeekExec {
+  using OutValue = typename internal::GetOutputType<OutType>::T;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    DayOfWeekOptions options = DayOfWeekState::Get(ctx);

Review comment:
       Done.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: r/src/compute.cpp
##########
@@ -264,6 +264,12 @@ std::shared_ptr<arrow::compute::FunctionOptions> make_compute_options(
                                      max_replacements);
   }
 
+  if (func_name == "day_of_week") {
+    using Options = arrow::compute::TemporalComponentExtractionOptions;
+    return std::make_shared<Options>(cpp11::as_cpp<bool>(options["one_based_numbering"]),

Review comment:
       Got it, you mean like this?
   ```C++
   if (func_name == "day_of_week") {
     using Options = arrow::compute::TemporalComponentExtractionOptions;
     bool one_based_numbering = true;
     return std::make_shared<Options>(one_based_numbering,
                                      cpp11::as_cpp<uint32_t>(options["week_start"]));
   }
   ```




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -35,6 +36,7 @@ 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::time_zone;

Review comment:
       Oh yeah. Not yet :)
   Removed.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] thisisnic edited a comment on pull request #10598: ARROW-13054: [C++] Add TemporalOptions

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


   > > About the actual options in this PR: that seems like a good start, but I don't think all those options would be needed in a single Options struct? For example, the ambiguous/non-existent handling, those can be put in a specific options struct for the kernel that needs it?
   > 
   > I'm ok with single Options struct or multiple.
   > If we go for multiple we'd probably split this into `TemporalStrftimeOptions` ([ARROW-13174](https://issues.apache.org/jira/browse/ARROW-13174)), `TemporalComponentExtractionOptions`, `TemporalLocalizationOptions` ([ARROW-13033](https://issues.apache.org/jira/browse/ARROW-13033)). Am I missing something?
   
   Those make sense to me.  I think there may be other Options structs we might need; for example, ones for kernels that do maths with dates. However, we don't need to know all of these in advance, right?
   
   


-- 
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 #10598: ARROW-13054: [C++] Add TemporalOptions

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


   > However, we don't need to know all of these in advance, right?
   
   Indeed. Even the ones I listed above should be implemented just-in-time with their respective kernels IMO. So I'll only implement `TemporalComponentExtractionOptions` here.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] thisisnic commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: r/src/compute.cpp
##########
@@ -264,6 +264,12 @@ std::shared_ptr<arrow::compute::FunctionOptions> make_compute_options(
                                      max_replacements);
   }
 
+  if (func_name == "day_of_week") {
+    using Options = arrow::compute::TemporalComponentExtractionOptions;
+    return std::make_shared<Options>(cpp11::as_cpp<bool>(options["one_based_numbering"]),

Review comment:
       @rok Not quite - `one_based_numbering` has a default value of `false` in C++, whereas in R it should be `true`.  I _think_ the way to achieve it is to not use the C++ defaults and do something similar to this example from another compute function: https://github.com/apache/arrow/blob/e9fa30406215b76ed6c885302fbfe6075c47badf/r/src/compute.cpp#L244-L254




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on pull request #10598: ARROW-13054: [C++] Add TemporalOptions

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


   > Pandas actually has a different method (the one you link is an attribute that has been there for a long time, so but pandas added an additional method so it could have arguments) that takes a locale arguments: https://pandas.pydata.org/docs/reference/api/pandas.Series.dt.day_name.html
   
   Oh, nice to know!
    
   > Personally, I would (at least for now) leave all locale-specific handling to the bindings / downstream applications (except for `strftime`). As Nic says, if we only return integers (and not `labels`) for now, we shouldn't need locale handling (I wouldn't let the number depend on the locale, that seems very easy to miss).
   
   Yeah, leaving locale out of this does seem like the sane option for now.
    
   > About the actual options in this PR: that seems like a good start, but I don't think all those options would be needed in a single Options struct? For example, the ambiguous/non-existent handling, those can be put in a specific options struct for the kernel that needs it?
   
   I'm ok with single Options struct or multiple.
   If we go for multiple we'd probably split this into `TemporalStrftimeOptions` ([ARROW-13174](https://issues.apache.org/jira/browse/ARROW-13174)), `TemporalComponentExtractionOptions`, `TemporalLocalizationOptions` ([ARROW-13033](https://issues.apache.org/jira/browse/ARROW-13033)). Am I missing 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] thisisnic commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: r/src/compute.cpp
##########
@@ -264,6 +264,12 @@ std::shared_ptr<arrow::compute::FunctionOptions> make_compute_options(
                                      max_replacements);
   }
 
+  if (func_name == "day_of_week") {
+    using Options = arrow::compute::TemporalComponentExtractionOptions;
+    return std::make_shared<Options>(cpp11::as_cpp<bool>(options["one_based_numbering"]),

Review comment:
       I think that's OK - I had in mind also adding in 
   ```r
   if (!Rf_isNull(options["one_based_numbering"])) { 
        one_based_numbering = cpp11::as_cpp<bool>(options["one_based_numbering"]); 
    } 
   ```
   but now I think it might be redundant, so your solution is fine!




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] thisisnic commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: r/src/compute.cpp
##########
@@ -264,6 +264,12 @@ std::shared_ptr<arrow::compute::FunctionOptions> make_compute_options(
                                      max_replacements);
   }
 
+  if (func_name == "day_of_week") {
+    using Options = arrow::compute::TemporalComponentExtractionOptions;
+    return std::make_shared<Options>(cpp11::as_cpp<bool>(options["one_based_numbering"]),

Review comment:
       I think it'd be very uncommon or never used at all, and even if someone wanted to use it, it'd be easy for them just to deduct 1 from whatever value they get, so I don't think it matters.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: r/R/dplyr-functions.R
##########
@@ -526,14 +526,6 @@ nse_funcs$second <- function(x) {
   Expression$create("add", Expression$create("second", x), Expression$create("subsecond", x))
 }
 
-# After ARROW-13054 is completed, we can refactor this for simplicity

Review comment:
       Ah, I misunderstood the original question. I'll add the options to Python. 




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] jorisvandenbossche commented on pull request #10598: ARROW-13054: [C++] Add TemporalComponentExtractionOptions

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






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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on pull request #10598: ARROW-13054: [C++] Add TemporalOptions

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


   Hey @thisisnic! Thanks for the input :).
   
   > * `fiscal_start` - this is an argument to `quarter` which indicates the starting month of a fiscal year - I think this would be useful to implement
   
   Is this just an arbitrary month user sets or is there a standard?
   
   > * `with_year` - an argument to `quarter` which defines whether the returned value is just the quarter or the quarter _and_ the year - I don't think we need this as it makes the outputs ambiguous and can be achieved at the R layer
   > * `label` - an argument to `month` and `wday` (aka `day_of_week` in Arrow) which defines whether the returned value is an integer or character representation of the month - as discussed on https://issues.apache.org/jira/browse/ARROW-13133, this can be achieved via a separate `strftime` kernel
   > * `abbr` - works with `label` and determines whether the label is abbreviated - I think this belongs to the `strftime` kernel
   > * `locale` - locale to use for month/week day names if `label` is set to `TRUE`- I think this belongs to the `strftime` kernel
   
   I was wondering about locale as well. Would it be a good idea to be able to set an arbitrary locale in some cases as well? [Pandas appears not to offer this option](https://pandas.pydata.org/docs/reference/api/pandas.Series.dt.dayofweek.html).
   
   Added [strftime jira](https://issues.apache.org/jira/browse/ARROW-13174).
    
   
   


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

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



[GitHub] [arrow] rok commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -80,6 +84,25 @@ struct TemporalComponentExtract {
   }
 };
 
+template <typename Op, typename OutType>
+struct DayOfWeekOptionsWithOptions {

Review comment:
       Sure.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] rok commented on pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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


   @pitrou Ping :)


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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


   @rok You'll need to rebase and adapt the code to the recent changes in `FunctionOptions`.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] thisisnic commented on pull request #10598: ARROW-13054: [C++] Add TemporalComponentExtractionOptions

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


   > > @rok you mention this PR now only has `index_of_monday`, but in the diff I currently see `start_index`, which is something else?
   > > Also the explanation of it, "Index of the first day of the week", is a bit ambiguous (what is the first day of the week?)
   > 
   > @jorisvandenbossche well the beginning of the week could be e.g.: `Monday=0`, `Monday=1`, `Sunday=0` or `Sunday=1`. Meaning we really need two parameters:
   > 
   >     * which day (Sunday/Monday) starts the week - this is normally (eventually) given by locale but in the scope of this PR it's Monday.
   > 
   >     * what index does the week start at - this would here be set by `start_index`
   > 
   > 
   > Alternatively we can have `index_of_monday` and calculate kinda like `(day_of_week(timestamp) + index_of_monday) % 7`.
   > 
   > How about `"Index of the first day of the week"` to `"Offset of day of week"`?
   
   I'm not sure if this might be a little confusing combining them.  Could we perhaps define ourselves which integer maps to which day, and then just have the 1 parameter `start_index`, similarly to how `week_start` works in R's lubridate package (https://lubridate.tidyverse.org/reference/day.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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] thisisnic commented on pull request #10598: ARROW-13054: [C++] Add TemporalOptions

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


   > Is this missing an option we will need in the near future? Should we remove something.
   > @jorisvandenbossche @thisisnic
   
   Thanks for this! I've taken a look at the `lubridate` package in R, and in terms of options that are implemented in `lubridate` (specifically for functions which are used for extracting components of date-time objects as I am assuming that is the scope of this - correct me if I'm wrong though!) but not here, there is:
   
   * `fiscal_start` - this is an argument to `quarter` which indicates the starting month of a fiscal year - I think this would be useful to implement
   
   * `with_year` - an argument to `quarter` which defines whether the returned value is just the quarter or the quarter *and* the year - I don't think we need this as it makes the outputs ambiguous and can be achieved at the R layer
   
   * `label` - an argument to `month` and `wday` (aka `day_of_week` in Arrow) which defines whether the returned value is an integer or character representation of the month - as discussed on https://issues.apache.org/jira/browse/ARROW-13133, this can be achieved via a separate `strftime` kernel
   
   * `abbr` - works with `label` and determines whether the label is abbreviated - I think this belongs to the `strftime` kernel
   
   * `locale` - locale to use for month/week day names if `label` is set to `TRUE`- I think this belongs to the `strftime` kernel
   
   So, yeah, I'm thinking `fiscal_start` is the only other accessor option that could be good to add, but it'd be good to get input from @nealrichardson on 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] pitrou commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_temporal.cc
##########
@@ -80,6 +84,25 @@ struct TemporalComponentExtract {
   }
 };
 
+template <typename Op, typename OutType>
+struct DayOfWeekOptionsWithOptions {

Review comment:
       This name is... weird. Can you find something better?




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -216,6 +216,22 @@ struct ARROW_EXPORT ProjectOptions : public FunctionOptions {
   std::vector<std::shared_ptr<const KeyValueMetadata>> field_metadata;
 };
 
+struct ARROW_EXPORT TemporalComponentExtractionOptions : public FunctionOptions {

Review comment:
       Call this `DayOfWeekOptions`?




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -216,6 +216,19 @@ struct ARROW_EXPORT ProjectOptions : public FunctionOptions {
   std::vector<std::shared_ptr<const KeyValueMetadata>> field_metadata;
 };
 
+struct ARROW_EXPORT DayOfWeekOptions : public FunctionOptions {
+  explicit DayOfWeekOptions(bool one_based_numbering, uint32_t week_start)
+      : one_based_numbering(one_based_numbering), week_start(week_start) {}

Review comment:
       I suppose the Option construction cannot raise an error, since it doesn't return a Result or Status. But we should certainly not silently truncate the value passed by the user. 
   
   Is there another place where this can be validated in the C++ kernel machinery? For example in the kernel's `Exec` ? 




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on a change in pull request #10598: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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



##########
File path: r/R/dplyr-functions.R
##########
@@ -526,14 +526,6 @@ nse_funcs$second <- function(x) {
   Expression$create("add", Expression$create("second", x), Expression$create("subsecond", x))
 }
 
-# After ARROW-13054 is completed, we can refactor this for simplicity

Review comment:
       That assumes you're using the same options class for all these kernels, which seems a bit weird to me (why would the `year` kernel take unrelated options?).




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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