You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by jo...@apache.org on 2021/07/07 07:28:46 UTC

[arrow] branch master updated: ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel

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

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


The following commit(s) were added to refs/heads/master by this push:
     new fdd7d32  ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel
fdd7d32 is described below

commit fdd7d32bcbc4086242e6a3517ef49e4f4468bd56
Author: Rok <ro...@mihevc.org>
AuthorDate: Wed Jul 7 09:27:07 2021 +0200

    ARROW-13054: [C++] Add option to specify the first day of the week for the "day_of_week" temporal kernel
    
    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).
    
    Closes #10598 from rok/ARROW-13054
    
    Lead-authored-by: Rok <ro...@mihevc.org>
    Co-authored-by: Rok Mihevc <ro...@mihevc.org>
    Signed-off-by: Joris Van den Bossche <jo...@gmail.com>
---
 cpp/src/arrow/compute/api_scalar.cc                |  15 +-
 cpp/src/arrow/compute/api_scalar.h                 |  23 ++-
 cpp/src/arrow/compute/function_test.cc             |   1 +
 cpp/src/arrow/compute/kernels/scalar_temporal.cc   |  97 +++++++++-
 .../arrow/compute/kernels/scalar_temporal_test.cc  | 215 ++++++++++++---------
 docs/source/cpp/compute.rst                        |  78 ++++----
 python/pyarrow/_compute.pyx                        |  12 ++
 python/pyarrow/compute.py                          |   1 +
 python/pyarrow/includes/libarrow.pxd               |   6 +
 python/pyarrow/tests/test_compute.py               |  75 +++++++
 r/R/dplyr-functions.R                              |  12 +-
 r/src/compute.cpp                                  |  10 +
 12 files changed, 396 insertions(+), 149 deletions(-)

diff --git a/cpp/src/arrow/compute/api_scalar.cc b/cpp/src/arrow/compute/api_scalar.cc
index 719fbed..be6498a 100644
--- a/cpp/src/arrow/compute/api_scalar.cc
+++ b/cpp/src/arrow/compute/api_scalar.cc
@@ -158,6 +158,9 @@ static auto kProjectOptionsType = GetFunctionOptionsType<ProjectOptions>(
     DataMember("field_names", &ProjectOptions::field_names),
     DataMember("field_nullability", &ProjectOptions::field_nullability),
     DataMember("field_metadata", &ProjectOptions::field_metadata));
+static auto kDayOfWeekOptionsType = GetFunctionOptionsType<DayOfWeekOptions>(
+    DataMember("one_based_numbering", &DayOfWeekOptions::one_based_numbering),
+    DataMember("week_start", &DayOfWeekOptions::week_start));
 }  // namespace
 }  // namespace internal
 
@@ -278,6 +281,12 @@ ProjectOptions::ProjectOptions(std::vector<std::string> n)
 ProjectOptions::ProjectOptions() : ProjectOptions(std::vector<std::string>()) {}
 constexpr char ProjectOptions::kTypeName[];
 
+DayOfWeekOptions::DayOfWeekOptions(bool one_based_numbering, uint32_t week_start)
+    : FunctionOptions(internal::kDayOfWeekOptionsType),
+      one_based_numbering(one_based_numbering),
+      week_start(week_start) {}
+constexpr char DayOfWeekOptions::kTypeName[];
+
 namespace internal {
 void RegisterScalarOptions(FunctionRegistry* registry) {
   DCHECK_OK(registry->AddFunctionOptionsType(kArithmeticOptionsType));
@@ -296,6 +305,7 @@ void RegisterScalarOptions(FunctionRegistry* registry) {
   DCHECK_OK(registry->AddFunctionOptionsType(kSliceOptionsType));
   DCHECK_OK(registry->AddFunctionOptionsType(kCompareOptionsType));
   DCHECK_OK(registry->AddFunctionOptionsType(kProjectOptionsType));
+  DCHECK_OK(registry->AddFunctionOptionsType(kDayOfWeekOptionsType));
 }
 }  // namespace internal
 
@@ -462,7 +472,6 @@ Result<Datum> IfElse(const Datum& cond, const Datum& if_true, const Datum& if_fa
 SCALAR_EAGER_UNARY(Year, "year")
 SCALAR_EAGER_UNARY(Month, "month")
 SCALAR_EAGER_UNARY(Day, "day")
-SCALAR_EAGER_UNARY(DayOfWeek, "day_of_week")
 SCALAR_EAGER_UNARY(DayOfYear, "day_of_year")
 SCALAR_EAGER_UNARY(ISOYear, "iso_year")
 SCALAR_EAGER_UNARY(ISOWeek, "iso_week")
@@ -476,5 +485,9 @@ SCALAR_EAGER_UNARY(Microsecond, "microsecond")
 SCALAR_EAGER_UNARY(Nanosecond, "nanosecond")
 SCALAR_EAGER_UNARY(Subsecond, "subsecond")
 
+Result<Datum> DayOfWeek(const Datum& arg, DayOfWeekOptions options, ExecContext* ctx) {
+  return CallFunction("day_of_week", {arg}, &options, ctx);
+}
+
 }  // namespace compute
 }  // namespace arrow
diff --git a/cpp/src/arrow/compute/api_scalar.h b/cpp/src/arrow/compute/api_scalar.h
index 8417d77..f0aebc8 100644
--- a/cpp/src/arrow/compute/api_scalar.h
+++ b/cpp/src/arrow/compute/api_scalar.h
@@ -244,6 +244,18 @@ class ARROW_EXPORT ProjectOptions : public FunctionOptions {
   std::vector<std::shared_ptr<const KeyValueMetadata>> field_metadata;
 };
 
+struct ARROW_EXPORT DayOfWeekOptions : public FunctionOptions {
+ public:
+  explicit DayOfWeekOptions(bool one_based_numbering = false, uint32_t week_start = 1);
+  constexpr static char const kTypeName[] = "DayOfWeekOptions";
+  static DayOfWeekOptions Defaults() { return DayOfWeekOptions{}; }
+
+  /// Number days from 1 if true and from 0 if false
+  bool one_based_numbering;
+  /// What day does the week start with (Monday=1, Sunday=7)
+  uint32_t week_start;
+};
+
 /// @}
 
 /// \brief Get the absolute value of a value.
@@ -764,15 +776,22 @@ 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
+/// by 6. Start day of the week (Monday=1, Sunday=7) and numbering base (0 or 1) can be
+/// set using DayOfWeekOptions
 ///
 /// \param[in] values input to extract number of the day of the week from
+/// \param[in] options for setting start of the week and day numbering
 /// \param[in] ctx the function execution context, optional
 /// \return the resulting datum
 ///
 /// \since 5.0.0
 /// \note API not yet finalized
-ARROW_EXPORT Result<Datum> DayOfWeek(const Datum& values, ExecContext* ctx = NULLPTR);
+ARROW_EXPORT Result<Datum> DayOfWeek(const Datum& values,
+                                     DayOfWeekOptions options = DayOfWeekOptions(),
+                                     ExecContext* ctx = NULLPTR);
 
 /// \brief DayOfYear returns number of day of the year for each element of `values`.
 /// January 1st maps to day number 1, February 1st to 32, etc.
diff --git a/cpp/src/arrow/compute/function_test.cc b/cpp/src/arrow/compute/function_test.cc
index bbe514a..752ade2 100644
--- a/cpp/src/arrow/compute/function_test.cc
+++ b/cpp/src/arrow/compute/function_test.cc
@@ -90,6 +90,7 @@ TEST(FunctionOptions, Equality) {
   options.emplace_back(new ProjectOptions({"col1"}, {false}, {}));
   options.emplace_back(
       new ProjectOptions({"col1"}, {false}, {key_value_metadata({{"key", "val"}})}));
+  options.emplace_back(new DayOfWeekOptions(false, 1));
   options.emplace_back(new CastOptions(CastOptions::Safe(boolean())));
   options.emplace_back(new CastOptions(CastOptions::Unsafe(int64())));
   options.emplace_back(new FilterOptions());
diff --git a/cpp/src/arrow/compute/kernels/scalar_temporal.cc b/cpp/src/arrow/compute/kernels/scalar_temporal.cc
index 1694d22..f025777 100644
--- a/cpp/src/arrow/compute/kernels/scalar_temporal.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_temporal.cc
@@ -16,6 +16,7 @@
 // under the License.
 
 #include "arrow/builder.h"
+#include "arrow/compute/api_scalar.h"
 #include "arrow/compute/kernels/common.h"
 #include "arrow/util/checked_cast.h"
 #include "arrow/util/time.h"
@@ -48,6 +49,8 @@ using arrow_vendored::date::literals::thu;
 using internal::applicator::ScalarUnaryNotNull;
 using internal::applicator::SimpleUnary;
 
+using DayOfWeekState = OptionsWrapper<DayOfWeekOptions>;
+
 const std::string& GetInputTimezone(const Datum& datum) {
   return checked_cast<const TimestampType&>(*datum.type()).timezone();
 }
@@ -80,6 +83,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) {
+    const DayOfWeekOptions& options = DayOfWeekState::Get(ctx);
+    if (options.week_start < 1 || 7 < options.week_start) {
+      return Status::Invalid(
+          "week_start must follow ISO convention (Monday=1, Sunday=7). Got week_start=",
+          options.week_start);
+    }
+
+    RETURN_NOT_OK(TemporalComponentExtractCheckTimezone(batch.values[0]));
+    applicator::ScalarUnaryNotNullStateful<OutType, TimestampType, Op> kernel{
+        Op(options)};
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
 // ----------------------------------------------------------------------
 // Extract year from timestamp
 
@@ -118,16 +140,30 @@ 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(const DayOfWeekOptions& 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;
 };
 
 // ----------------------------------------------------------------------
@@ -398,6 +434,42 @@ std::shared_ptr<ScalarFunction> MakeTemporal(std::string name, const FunctionDoc
   return func;
 }
 
+template <template <typename...> class Op, typename OutType>
+std::shared_ptr<ScalarFunction> MakeTemporalWithOptions(
+    std::string name, const FunctionDoc* doc, const DayOfWeekOptions& default_options,
+    KernelInit init) {
+  const auto& out_type = TypeTraits<OutType>::type_singleton();
+  auto func =
+      std::make_shared<ScalarFunction>(name, Arity::Unary(), doc, &default_options);
+
+  for (auto unit : internal::AllTimeUnits()) {
+    InputType in_type{match::TimestampTypeUnit(unit)};
+    switch (unit) {
+      case TimeUnit::SECOND: {
+        auto exec = DayOfWeekExec<Op<std::chrono::seconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec), init));
+        break;
+      }
+      case TimeUnit::MILLI: {
+        auto exec = DayOfWeekExec<Op<std::chrono::milliseconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec), init));
+        break;
+      }
+      case TimeUnit::MICRO: {
+        auto exec = DayOfWeekExec<Op<std::chrono::microseconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec), init));
+        break;
+      }
+      case TimeUnit::NANO: {
+        auto exec = DayOfWeekExec<Op<std::chrono::nanoseconds>, OutType>::Exec;
+        DCHECK_OK(func->AddKernel({in_type}, out_type, std::move(exec), init));
+        break;
+      }
+    }
+  }
+  return func;
+}
+
 template <template <typename...> class Op>
 std::shared_ptr<ScalarFunction> MakeStructTemporal(std::string name,
                                                    const FunctionDoc* doc) {
@@ -451,9 +523,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, the week starts on Monday represented by 0 and ends on Sunday "
+     "represented by 6.\n"
+     "DayOfWeekOptions.week_start can be used to set another starting day using ISO "
+     "convention (Monday=1, Sunday=7). Day numbering can start with 0 or 1 using "
+     "DayOfWeekOptions.one_based_numbering parameter.\n"
      "Returns an error if timestamp has a defined timezone. Null values return null."),
-    {"values"}};
+    {"values"},
+    "DayOfWeekOptions"};
 
 const FunctionDoc day_of_year_doc{
     "Extract number of day of year",
@@ -537,7 +614,9 @@ void RegisterScalarTemporal(FunctionRegistry* registry) {
   auto day = MakeTemporal<Day, Int64Type>("day", &year_doc);
   DCHECK_OK(registry->AddFunction(std::move(day)));
 
-  auto day_of_week = MakeTemporal<DayOfWeek, Int64Type>("day_of_week", &day_of_week_doc);
+  static auto default_day_of_week_options = DayOfWeekOptions::Defaults();
+  auto day_of_week = MakeTemporalWithOptions<DayOfWeek, Int64Type>(
+      "day_of_week", &day_of_week_doc, default_day_of_week_options, DayOfWeekState::Init);
   DCHECK_OK(registry->AddFunction(std::move(day_of_week)));
 
   auto day_of_year = MakeTemporal<DayOfYear, Int64Type>("day_of_year", &day_of_year_doc);
@@ -561,7 +640,7 @@ void RegisterScalarTemporal(FunctionRegistry* registry) {
   auto minute = MakeTemporal<Minute, Int64Type>("minute", &minute_doc);
   DCHECK_OK(registry->AddFunction(std::move(minute)));
 
-  auto second = MakeTemporal<Second, DoubleType>("second", &second_doc);
+  auto second = MakeTemporal<Second, Int64Type>("second", &second_doc);
   DCHECK_OK(registry->AddFunction(std::move(second)));
 
   auto millisecond =
diff --git a/cpp/src/arrow/compute/kernels/scalar_temporal_test.cc b/cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
index cc01d25..f2e9c12 100644
--- a/cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_temporal_test.cc
@@ -26,11 +26,8 @@ namespace arrow {
 
 using internal::StringFormatter;
 
-class ScalarTemporalTest : public ::testing::Test {};
-
-namespace compute {
-
-TEST(ScalarTemporalTest, TestTemporalComponentExtraction) {
+class ScalarTemporalTest : public ::testing::Test {
+ public:
   const char* times =
       R"(["1970-01-01T00:00:59.123456789","2000-02-29T23:23:23.999999999",
           "1899-01-01T00:59:20.001001001","2033-05-18T03:33:20.000000000",
@@ -39,52 +36,70 @@ TEST(ScalarTemporalTest, TestTemporalComponentExtraction) {
           "2010-01-01T05:25:25.005321", "2010-01-03T06:30:30.006163",
           "2010-01-04T07:35:35", "2006-01-01T08:40:40", "2005-12-31T09:45:45",
           "2008-12-28", "2008-12-29", "2012-01-01 01:02:03"])";
-  auto unit = timestamp(TimeUnit::NANO);
-  auto iso_calendar_type =
+  const char* times_seconds_precision =
+      R"(["1970-01-01T00:00:59","2000-02-29T23:23:23",
+          "1899-01-01T00:59:20","2033-05-18T03:33:20",
+          null, "2020-01-01T01:05:05", "2019-12-31T02:10:10",
+          "2019-12-30T03:15:15", "2009-12-31T04:20:20",
+          "2010-01-01T05:25:25", "2010-01-03T06:30:30",
+          "2010-01-04T07:35:35", "2006-01-01T08:40:40", "2005-12-31T09:45:45",
+          "2008-12-28", "2008-12-29", "2012-01-01 01:02:03"])";
+  std::shared_ptr<arrow::DataType> iso_calendar_type =
       struct_({field("iso_year", int64()), field("iso_week", int64()),
                field("iso_day_of_week", int64())});
-
-  auto year =
+  std::shared_ptr<arrow::Array> iso_calendar =
+      ArrayFromJSON(iso_calendar_type,
+                    R"([{"iso_year": 1970, "iso_week": 1, "iso_day_of_week": 4},
+                          {"iso_year": 2000, "iso_week": 9, "iso_day_of_week": 2},
+                          {"iso_year": 1898, "iso_week": 52, "iso_day_of_week": 7},
+                          {"iso_year": 2033, "iso_week": 20, "iso_day_of_week": 3},
+                          null,
+                          {"iso_year": 2020, "iso_week": 1, "iso_day_of_week": 3},
+                          {"iso_year": 2020, "iso_week": 1, "iso_day_of_week": 2},
+                          {"iso_year": 2020, "iso_week": 1, "iso_day_of_week": 1},
+                          {"iso_year": 2009, "iso_week": 53, "iso_day_of_week": 4},
+                          {"iso_year": 2009, "iso_week": 53, "iso_day_of_week": 5},
+                          {"iso_year": 2009, "iso_week": 53, "iso_day_of_week": 7},
+                          {"iso_year": 2010, "iso_week": 1, "iso_day_of_week": 1},
+                          {"iso_year": 2005, "iso_week": 52, "iso_day_of_week": 7},
+                          {"iso_year": 2005, "iso_week": 52, "iso_day_of_week": 6},
+                          {"iso_year": 2008, "iso_week": 52, "iso_day_of_week": 7},
+                          {"iso_year": 2009, "iso_week": 1, "iso_day_of_week": 1},
+                          {"iso_year": 2011, "iso_week": 52, "iso_day_of_week": 7}])");
+  std::string year =
       "[1970, 2000, 1899, 2033, null, 2020, 2019, 2019, 2009, 2010, 2010, 2010, 2006, "
       "2005, 2008, 2008, 2012]";
-  auto month = "[1, 2, 1, 5, null, 1, 12, 12, 12, 1, 1, 1, 1, 12, 12, 12, 1]";
-  auto day = "[1, 29, 1, 18, null, 1, 31, 30, 31, 1, 3, 4, 1, 31, 28, 29, 1]";
-  auto day_of_week = "[3, 1, 6, 2, null, 2, 1, 0, 3, 4, 6, 0, 6, 5, 6, 0, 6]";
-  auto day_of_year =
+  std::string month = "[1, 2, 1, 5, null, 1, 12, 12, 12, 1, 1, 1, 1, 12, 12, 12, 1]";
+  std::string day = "[1, 29, 1, 18, null, 1, 31, 30, 31, 1, 3, 4, 1, 31, 28, 29, 1]";
+  std::string day_of_week = "[3, 1, 6, 2, null, 2, 1, 0, 3, 4, 6, 0, 6, 5, 6, 0, 6]";
+  std::string day_of_year =
       "[1, 60, 1, 138, null, 1, 365, 364, 365, 1, 3, 4, 1, 365, 363, 364, 1]";
-  auto iso_year =
+  std::string iso_year =
       "[1970, 2000, 1898, 2033, null, 2020, 2020, 2020, 2009, 2009, 2009, 2010, 2005, "
       "2005, 2008, 2009, 2011]";
-  auto iso_week = "[1, 9, 52, 20, null, 1, 1, 1, 53, 53, 53, 1, 52, 52, 52, 1, 52]";
-  auto iso_calendar =
-      ArrayFromJSON(iso_calendar_type,
-                    R"([{"iso_year": 1970, "iso_week": 1, "iso_day_of_week": 4},
-                        {"iso_year": 2000, "iso_week": 9, "iso_day_of_week": 2},
-                        {"iso_year": 1898, "iso_week": 52, "iso_day_of_week": 7},
-                        {"iso_year": 2033, "iso_week": 20, "iso_day_of_week": 3},
-                        null,
-                        {"iso_year": 2020, "iso_week": 1, "iso_day_of_week": 3},
-                        {"iso_year": 2020, "iso_week": 1, "iso_day_of_week": 2},
-                        {"iso_year": 2020, "iso_week": 1, "iso_day_of_week": 1},
-                        {"iso_year": 2009, "iso_week": 53, "iso_day_of_week": 4},
-                        {"iso_year": 2009, "iso_week": 53, "iso_day_of_week": 5},
-                        {"iso_year": 2009, "iso_week": 53, "iso_day_of_week": 7},
-                        {"iso_year": 2010, "iso_week": 1, "iso_day_of_week": 1},
-                        {"iso_year": 2005, "iso_week": 52, "iso_day_of_week": 7},
-                        {"iso_year": 2005, "iso_week": 52, "iso_day_of_week": 6},
-                        {"iso_year": 2008, "iso_week": 52, "iso_day_of_week": 7},
-                        {"iso_year": 2009, "iso_week": 1, "iso_day_of_week": 1},
-                        {"iso_year": 2011, "iso_week": 52, "iso_day_of_week": 7}])");
-  auto quarter = "[1, 1, 1, 2, null, 1, 4, 4, 4, 1, 1, 1, 1, 4, 4, 4, 1]";
-  auto hour = "[0, 23, 0, 3, null, 1, 2, 3, 4, 5, 6, 7, 8, 9, 0, 0, 1]";
-  auto minute = "[0, 23, 59, 33, null, 5, 10, 15, 20, 25, 30, 35, 40, 45, 0, 0, 2]";
-  auto second = "[59, 23, 20, 20, null, 5, 10, 15, 20, 25, 30, 35, 40, 45, 0, 0, 3]";
-  auto millisecond = "[123, 999, 1, 0, null, 1, 2, 3, 4, 5, 6, 0, 0, 0, 0, 0, 0]";
-  auto microsecond = "[456, 999, 1, 0, null, 0, 0, 0, 132, 321, 163, 0, 0, 0, 0, 0, 0]";
-  auto nanosecond = "[789, 999, 1, 0, null, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0]";
-  auto subsecond =
+  std::string iso_week =
+      "[1, 9, 52, 20, null, 1, 1, 1, 53, 53, 53, 1, 52, 52, 52, 1, 52]";
+
+  std::string quarter = "[1, 1, 1, 2, null, 1, 4, 4, 4, 1, 1, 1, 1, 4, 4, 4, 1]";
+  std::string hour = "[0, 23, 0, 3, null, 1, 2, 3, 4, 5, 6, 7, 8, 9, 0, 0, 1]";
+  std::string minute =
+      "[0, 23, 59, 33, null, 5, 10, 15, 20, 25, 30, 35, 40, 45, 0, 0, 2]";
+  std::string second =
+      "[59, 23, 20, 20, null, 5, 10, 15, 20, 25, 30, 35, 40, 45, 0, 0, 3]";
+  std::string millisecond = "[123, 999, 1, 0, null, 1, 2, 3, 4, 5, 6, 0, 0, 0, 0, 0, 0]";
+  std::string microsecond =
+      "[456, 999, 1, 0, null, 0, 0, 0, 132, 321, 163, 0, 0, 0, 0, 0, 0]";
+  std::string nanosecond = "[789, 999, 1, 0, null, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0]";
+  std::string subsecond =
       "[0.123456789, 0.999999999, 0.001001001, 0, null, 0.001, 0.002, 0.003, 0.004132, "
       "0.005321, 0.006163, 0, 0, 0, 0, 0, 0]";
+  std::string zeros = "[0, 0, 0, 0, null, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0]";
+};
+
+namespace compute {
+
+TEST_F(ScalarTemporalTest, TestTemporalComponentExtraction) {
+  auto unit = timestamp(TimeUnit::NANO);
 
   CheckScalarUnary("year", unit, times, int64(), year);
   CheckScalarUnary("month", unit, times, int64(), month);
@@ -97,67 +112,42 @@ TEST(ScalarTemporalTest, TestTemporalComponentExtraction) {
   CheckScalarUnary("quarter", unit, times, int64(), quarter);
   CheckScalarUnary("hour", unit, times, int64(), hour);
   CheckScalarUnary("minute", unit, times, int64(), minute);
-  CheckScalarUnary("second", unit, times, float64(), second);
+  CheckScalarUnary("second", unit, times, int64(), second);
   CheckScalarUnary("millisecond", unit, times, int64(), millisecond);
   CheckScalarUnary("microsecond", unit, times, int64(), microsecond);
   CheckScalarUnary("nanosecond", unit, times, int64(), nanosecond);
   CheckScalarUnary("subsecond", unit, times, float64(), subsecond);
 }
 
-TEST(ScalarTemporalTest, TestTemporalComponentExtractionWithDifferentUnits) {
-  auto iso_calendar_type =
-      struct_({field("iso_year", int64()), field("iso_week", int64()),
-               field("iso_day_of_week", int64())});
-  const char* times =
-      R"(["1970-01-01T00:00:59","2000-02-29T23:23:23",
-            "1899-01-01T00:59:20","2033-05-18T03:33:20", null])";
-  auto year = "[1970, 2000, 1899, 2033, null]";
-  auto month = "[1, 2, 1, 5, null]";
-  auto day = "[1, 29, 1, 18, null]";
-  auto day_of_week = "[3, 1, 6, 2, null]";
-  auto day_of_year = "[1, 60, 1, 138, null]";
-  auto iso_year = "[1970, 2000, 1898, 2033, null]";
-  auto iso_week = "[1, 9, 52, 20, null]";
-  auto iso_calendar =
-      ArrayFromJSON(iso_calendar_type,
-                    R"([{"iso_year": 1970, "iso_week": 1, "iso_day_of_week": 4},
-                          {"iso_year": 2000, "iso_week": 9, "iso_day_of_week": 2},
-                          {"iso_year": 1898, "iso_week": 52, "iso_day_of_week": 7},
-                          {"iso_year": 2033, "iso_week": 20, "iso_day_of_week": 3}, null])");
-  auto quarter = "[1, 1, 1, 2, null]";
-  auto hour = "[0, 23, 0, 3, null]";
-  auto minute = "[0, 23, 59, 33, null]";
-  auto second = "[59, 23, 20, 20, null]";
-  auto zeros = "[0, 0, 0, 0, null]";
-
+TEST_F(ScalarTemporalTest, TestTemporalComponentExtractionWithDifferentUnits) {
   for (auto u : internal::AllTimeUnits()) {
     auto unit = timestamp(u);
-    CheckScalarUnary("year", unit, times, int64(), year);
-    CheckScalarUnary("month", unit, times, int64(), month);
-    CheckScalarUnary("day", unit, times, int64(), day);
-    CheckScalarUnary("day_of_week", unit, times, int64(), day_of_week);
-    CheckScalarUnary("day_of_year", unit, times, int64(), day_of_year);
-    CheckScalarUnary("iso_year", unit, times, int64(), iso_year);
-    CheckScalarUnary("iso_week", unit, times, int64(), iso_week);
-    CheckScalarUnary("iso_calendar", ArrayFromJSON(unit, times), iso_calendar);
-    CheckScalarUnary("quarter", unit, times, int64(), quarter);
-    CheckScalarUnary("hour", unit, times, int64(), hour);
-    CheckScalarUnary("minute", unit, times, int64(), minute);
-    CheckScalarUnary("second", unit, times, float64(), second);
-    CheckScalarUnary("millisecond", unit, times, int64(), zeros);
-    CheckScalarUnary("microsecond", unit, times, int64(), zeros);
-    CheckScalarUnary("nanosecond", unit, times, int64(), zeros);
-    CheckScalarUnary("subsecond", unit, times, float64(), zeros);
+    CheckScalarUnary("year", unit, times_seconds_precision, int64(), year);
+    CheckScalarUnary("month", unit, times_seconds_precision, int64(), month);
+    CheckScalarUnary("day", unit, times_seconds_precision, int64(), day);
+    CheckScalarUnary("day_of_week", unit, times_seconds_precision, int64(), day_of_week);
+    CheckScalarUnary("day_of_year", unit, times_seconds_precision, int64(), day_of_year);
+    CheckScalarUnary("iso_year", unit, times_seconds_precision, int64(), iso_year);
+    CheckScalarUnary("iso_week", unit, times_seconds_precision, int64(), iso_week);
+    CheckScalarUnary("iso_calendar", ArrayFromJSON(unit, times_seconds_precision),
+                     iso_calendar);
+    CheckScalarUnary("quarter", unit, times_seconds_precision, int64(), quarter);
+    CheckScalarUnary("hour", unit, times_seconds_precision, int64(), hour);
+    CheckScalarUnary("minute", unit, times_seconds_precision, int64(), minute);
+    CheckScalarUnary("second", unit, times_seconds_precision, int64(), second);
+    CheckScalarUnary("millisecond", unit, times_seconds_precision, int64(), zeros);
+    CheckScalarUnary("microsecond", unit, times_seconds_precision, int64(), zeros);
+    CheckScalarUnary("nanosecond", unit, times_seconds_precision, int64(), zeros);
+    CheckScalarUnary("subsecond", unit, times_seconds_precision, float64(), zeros);
   }
 }
 
-TEST(ScalarTemporalTest, TestZonedTemporalComponentExtraction) {
+TEST_F(ScalarTemporalTest, TestZonedTemporalComponentExtraction) {
   std::string timezone = "Asia/Kolkata";
-  const char* times = R"(["1970-01-01T00:00:59", null])";
 
   for (auto u : internal::AllTimeUnits()) {
     auto unit = timestamp(u, timezone);
-    auto timestamps = ArrayFromJSON(unit, times);
+    auto timestamps = ArrayFromJSON(unit, times_seconds_precision);
 
     ASSERT_RAISES(NotImplemented, Year(timestamps));
     ASSERT_RAISES(NotImplemented, Month(timestamps));
@@ -177,5 +167,54 @@ TEST(ScalarTemporalTest, TestZonedTemporalComponentExtraction) {
     ASSERT_RAISES(NotImplemented, Subsecond(timestamps));
   }
 }
+
+TEST_F(ScalarTemporalTest, DayOfWeek) {
+  auto unit = timestamp(TimeUnit::NANO);
+
+  auto timestamps = ArrayFromJSON(unit, times);
+  auto day_of_week_week_start_7_zero_based =
+      "[4, 2, 0, 3, null, 3, 2, 1, 4, 5, 0, 1, 0, 6, 0, 1, 0]";
+  auto day_of_week_week_start_2_zero_based =
+      "[2, 0, 5, 1, null, 1, 0, 6, 2, 3, 5, 6, 5, 4, 5, 6, 5]";
+  auto day_of_week_week_start_7_one_based =
+      "[5, 3, 1, 4, null, 4, 3, 2, 5, 6, 1, 2, 1, 7, 1, 2, 1]";
+  auto day_of_week_week_start_2_one_based =
+      "[3, 1, 6, 2, null, 2, 1, 7, 3, 4, 6, 7, 6, 5, 6, 7, 6]";
+
+  auto expected_70 = ArrayFromJSON(int64(), day_of_week_week_start_7_zero_based);
+  ASSERT_OK_AND_ASSIGN(
+      Datum result_70,
+      DayOfWeek(timestamps, DayOfWeekOptions(
+                                /*one_based_numbering=*/false, /*week_start=*/7)));
+  ASSERT_TRUE(result_70.Equals(expected_70));
+
+  auto expected_20 = ArrayFromJSON(int64(), day_of_week_week_start_2_zero_based);
+  ASSERT_OK_AND_ASSIGN(
+      Datum result_20,
+      DayOfWeek(timestamps, DayOfWeekOptions(
+                                /*one_based_numbering=*/false, /*week_start=*/2)));
+  ASSERT_TRUE(result_20.Equals(expected_20));
+
+  auto expected_71 = ArrayFromJSON(int64(), day_of_week_week_start_7_one_based);
+  ASSERT_OK_AND_ASSIGN(
+      Datum result_71,
+      DayOfWeek(timestamps, DayOfWeekOptions(
+                                /*one_based_numbering=*/true, /*week_start=*/7)));
+  ASSERT_TRUE(result_71.Equals(expected_71));
+
+  auto expected_21 = ArrayFromJSON(int64(), day_of_week_week_start_2_one_based);
+  ASSERT_OK_AND_ASSIGN(
+      Datum result_21,
+      DayOfWeek(timestamps, DayOfWeekOptions(
+                                /*one_based_numbering=*/true, /*week_start=*/2)));
+  ASSERT_TRUE(result_21.Equals(expected_21));
+
+  ASSERT_RAISES(Invalid,
+                DayOfWeek(timestamps, DayOfWeekOptions(/*one_based_numbering=*/true,
+                                                       /*week_start=*/0)));
+  ASSERT_RAISES(Invalid,
+                DayOfWeek(timestamps, DayOfWeekOptions(/*one_based_numbering=*/false,
+                                                       /*week_start=*/8)));
+}
 }  // namespace compute
 }  // namespace arrow
diff --git a/docs/source/cpp/compute.rst b/docs/source/cpp/compute.rst
index b2bbe68..fc6c8b7 100644
--- a/docs/source/cpp/compute.rst
+++ b/docs/source/cpp/compute.rst
@@ -1014,44 +1014,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   | Options class              | Notes |
++====================+============+===================+===============+============================+=======+
+| year               | Unary      | Temporal          | Int64         |                            |       |
++--------------------+------------+-------------------+---------------+----------------------------+-------+
+| month              | Unary      | Temporal          | Int64         |                            |       |
++--------------------+------------+-------------------+---------------+----------------------------+-------+
+| day                | Unary      | Temporal          | Int64         |                            |       |
++--------------------+------------+-------------------+---------------+----------------------------+-------+
+| day_of_week        | Unary      | Temporal          | Int64         | :struct:`DayOfWeekOptions` | \(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. 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::one_based_numbering` parameter.
 * \(2) First ISO week has the majority (4 or more) of it's days in January. ISO year
   starts with the first ISO week.
   See `ISO 8601 week date definition`_ for more details.
diff --git a/python/pyarrow/_compute.pyx b/python/pyarrow/_compute.pyx
index 02855ee..1b66f74 100644
--- a/python/pyarrow/_compute.pyx
+++ b/python/pyarrow/_compute.pyx
@@ -961,6 +961,18 @@ class StrptimeOptions(_StrptimeOptions):
         self._set_options(format, unit)
 
 
+cdef class _DayOfWeekOptions(FunctionOptions):
+    def _set_options(self, one_based_numbering, week_start):
+        self.wrapped.reset(
+            new CDayOfWeekOptions(one_based_numbering, week_start)
+        )
+
+
+class DayOfWeekOptions(_DayOfWeekOptions):
+    def __init__(self, one_based_numbering=False, week_start=1):
+        self._set_options(one_based_numbering, week_start)
+
+
 cdef class _VarianceOptions(FunctionOptions):
     def _set_options(self, ddof):
         self.wrapped.reset(new CVarianceOptions(ddof))
diff --git a/python/pyarrow/compute.py b/python/pyarrow/compute.py
index fbe9e2c..15d1adc 100644
--- a/python/pyarrow/compute.py
+++ b/python/pyarrow/compute.py
@@ -52,6 +52,7 @@ from pyarrow._compute import (  # noqa
     SplitOptions,
     SplitPatternOptions,
     StrptimeOptions,
+    DayOfWeekOptions,
     TakeOptions,
     TDigestOptions,
     TrimOptions,
diff --git a/python/pyarrow/includes/libarrow.pxd b/python/pyarrow/includes/libarrow.pxd
index 67a960f..6977c26 100644
--- a/python/pyarrow/includes/libarrow.pxd
+++ b/python/pyarrow/includes/libarrow.pxd
@@ -1940,6 +1940,12 @@ cdef extern from "arrow/compute/api.h" namespace "arrow::compute" nogil:
             "arrow::compute::StrptimeOptions"(CFunctionOptions):
         CStrptimeOptions(c_string format, TimeUnit unit)
 
+    cdef cppclass CDayOfWeekOptions \
+            "arrow::compute::DayOfWeekOptions"(CFunctionOptions):
+        CDayOfWeekOptions(c_bool one_based_numbering, uint32_t week_start)
+        c_bool one_based_numbering
+        uint32_t week_start
+
     cdef cppclass CVarianceOptions \
             "arrow::compute::VarianceOptions"(CFunctionOptions):
         CVarianceOptions(int ddof)
diff --git a/python/pyarrow/tests/test_compute.py b/python/pyarrow/tests/test_compute.py
index 35b37d8..37040ec 100644
--- a/python/pyarrow/tests/test_compute.py
+++ b/python/pyarrow/tests/test_compute.py
@@ -120,6 +120,7 @@ def test_option_class_equality():
         pc.PadOptions(5, " "),
         pc.PartitionNthOptions(1),
         pc.ProjectOptions([b"field", b"names"]),
+        pc.DayOfWeekOptions(False, 0),
         pc.ReplaceSliceOptions(start=0, stop=1, replacement="a"),
         pc.ReplaceSubstringOptions("a", "b"),
         pc.SetLookupOptions(value_set=pa.array([1])),
@@ -1346,6 +1347,80 @@ def test_strptime():
     assert got == expected
 
 
+def _check_datetime_components(timestamps, timezone=None):
+    from pyarrow.vendored.version import Version
+
+    ts = pd.to_datetime(timestamps).to_series()
+    tsa = pa.array(ts)
+
+    subseconds = ((ts.dt.microsecond * 10**3 +
+                   ts.dt.nanosecond) * 10**-9).round(9)
+    iso_calendar_fields = [
+        pa.field('iso_year', pa.int64()),
+        pa.field('iso_week', pa.int64()),
+        pa.field('iso_day_of_week', pa.int64())
+    ]
+
+    if Version(pd.__version__) < Version("1.1.0"):
+        # https://github.com/pandas-dev/pandas/issues/33206
+        iso_year = ts.map(lambda x: x.isocalendar()[0]).astype("Int64")
+        iso_week = ts.map(lambda x: x.isocalendar()[1]).astype("Int64")
+        iso_day = ts.map(lambda x: x.isocalendar()[2]).astype("Int64")
+    else:
+        # Casting is required because pandas isocalendar returns int32
+        # while arrow isocalendar returns int64.
+        iso_year = ts.dt.isocalendar()["year"].astype("Int64")
+        iso_week = ts.dt.isocalendar()["week"].astype("Int64")
+        iso_day = ts.dt.isocalendar()["day"].astype("Int64")
+
+    iso_calendar = pa.StructArray.from_arrays(
+        [iso_year, iso_week, iso_day],
+        fields=iso_calendar_fields)
+
+    assert pc.year(tsa).equals(pa.array(ts.dt.year))
+    assert pc.month(tsa).equals(pa.array(ts.dt.month))
+    assert pc.day(tsa).equals(pa.array(ts.dt.day))
+    assert pc.day_of_week(tsa).equals(pa.array(ts.dt.dayofweek))
+    assert pc.day_of_year(tsa).equals(pa.array(ts.dt.dayofyear))
+    assert pc.iso_year(tsa).equals(pa.array(iso_year))
+    assert pc.iso_week(tsa).equals(pa.array(iso_week))
+    assert pc.iso_calendar(tsa).equals(iso_calendar)
+    assert pc.quarter(tsa).equals(pa.array(ts.dt.quarter))
+    assert pc.hour(tsa).equals(pa.array(ts.dt.hour))
+    assert pc.minute(tsa).equals(pa.array(ts.dt.minute))
+    assert pc.second(tsa).equals(pa.array(ts.dt.second.values))
+    assert pc.millisecond(tsa).equals(pa.array(ts.dt.microsecond // 10**3))
+    assert pc.microsecond(tsa).equals(pa.array(ts.dt.microsecond % 10**3))
+    assert pc.nanosecond(tsa).equals(pa.array(ts.dt.nanosecond))
+    assert pc.subsecond(tsa).equals(pa.array(subseconds))
+
+    day_of_week_options = pc.DayOfWeekOptions(
+        one_based_numbering=True, week_start=1)
+    assert pc.day_of_week(tsa, options=day_of_week_options).equals(
+        pa.array(ts.dt.dayofweek+1))
+
+
+@pytest.mark.pandas
+def test_extract_datetime_components():
+    timestamps = ["1970-01-01T00:00:59.123456789",
+                  "2000-02-29T23:23:23.999999999",
+                  "2033-05-18T03:33:20.000000000",
+                  "2020-01-01T01:05:05.001",
+                  "2019-12-31T02:10:10.002",
+                  "2019-12-30T03:15:15.003",
+                  "2009-12-31T04:20:20.004132",
+                  "2010-01-01T05:25:25.005321",
+                  "2010-01-03T06:30:30.006163",
+                  "2010-01-04T07:35:35",
+                  "2006-01-01T08:40:40",
+                  "2005-12-31T09:45:45",
+                  "2008-12-28",
+                  "2008-12-29",
+                  "2012-01-01 01:02:03"]
+
+    _check_datetime_components(timestamps)
+
+
 def test_count():
     arr = pa.array([1, 2, 3, None, None])
     assert pc.count(arr).as_py() == 3
diff --git a/r/R/dplyr-functions.R b/r/R/dplyr-functions.R
index 27d6e88..055cff5 100644
--- a/r/R/dplyr-functions.R
+++ b/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
-#
-# Arrow's `day_of_week` kernel counts from 0 (Monday) to 6 (Sunday), whereas
-# `lubridate::wday` counts from 1 to 7, and allows users to specify which day
-# of the week is first (Sunday by default).  This Expression converts the returned
-# day of the week back to the value that would be returned by lubridate by
-# providing offset values based on the specified week_start day, and adding 1
-# so the returned value is 1-indexed instead of 0-indexed.
 nse_funcs$wday <- function(x, label = FALSE, abbr = TRUE, week_start = getOption("lubridate.week.start", 7)) {
 
   # The "day_of_week" compute function returns numeric days of week and not locale-aware strftime
@@ -543,8 +535,6 @@ nse_funcs$wday <- function(x, label = FALSE, abbr = TRUE, week_start = getOption
     arrow_not_supported("Label argument")
   }
 
-  # overall formula to convert from arrow::wday to lubridate::wday is:
-  #  ((wday(day) - start + 8) %% 7) + 1
-  ((Expression$create("day_of_week", x) - Expression$scalar(week_start) + 8) %% 7) + 1
+  Expression$create("day_of_week", x, options = list(one_based_numbering = TRUE, week_start = week_start))
 
 }
diff --git a/r/src/compute.cpp b/r/src/compute.cpp
index 458e0e3..9be1cc3 100644
--- a/r/src/compute.cpp
+++ b/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::DayOfWeekOptions;
+    bool one_based_numbering = true;
+    if (!Rf_isNull(options["one_based_numbering"])) {
+      one_based_numbering = cpp11::as_cpp<bool>(options["one_based_numbering"]);
+    }
+    return std::make_shared<Options>(one_based_numbering,
+                                     cpp11::as_cpp<uint32_t>(options["week_start"]));
+  }
+
   if (func_name == "strptime") {
     using Options = arrow::compute::StrptimeOptions;
     return std::make_shared<Options>(