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 2022/06/27 08:31:40 UTC

[GitHub] [arrow] rok commented on a diff in pull request #12154: ARROW-14821: [R] Implement bindings for lubridate's floor_date, ceiling_date, and round_date

rok commented on code in PR #12154:
URL: https://github.com/apache/arrow/pull/12154#discussion_r907089623


##########
r/R/dplyr-funcs-datetime.R:
##########
@@ -686,4 +686,106 @@ register_bindings_datetime_parsers <- function() {
 
     build_expr("assume_timezone", coalesce_output, options = list(timezone = tz))
   })
+
+  register_binding("round_date", function(x, unit = "second",
+                                          week_start = getOption("lubridate.week.start", 7)) {
+    opts <- parse_period_unit(unit)
+
+    if (opts$unit == 7L) {
+
+      if (week_start == 7) { # Sunday
+        opts$week_starts_monday <- 0L
+        return(Expression$create("round_temporal", x, options = opts))
+
+      } else if (week_start == 1) { # Monday
+        opts$week_starts_monday <- 1L
+        return(Expression$create("round_temporal", x, options = opts))
+
+      } else { # for other values of week_start, compute using an offset
+        week_start_offset <- build_expr(
+          "cast",
+          Scalar$create((as.integer(week_start) - 1L) * 86400L, int64()),
+          options = cast_options(to_type = duration(unit = "s"))
+        )
+        interim <- build_expr("round_temporal", x - week_start_offset, options = opts)
+        return(interim + week_start_offset)
+      }
+    }
+    Expression$create("round_temporal", x, options = opts)
+  })
+
+  register_binding("floor_date", function(x, unit = "second",
+                                          week_start = getOption("lubridate.week.start", 7)) {
+    opts <- parse_period_unit(unit)
+    if (opts$unit == 7L) {
+      if (week_start == 7) { # Sunday
+        opts$week_starts_monday <- 0L
+        return(Expression$create("floor_temporal", x, options = opts))
+
+      } else if (week_start == 1) { # Monday
+        opts$week_starts_monday <- 1L
+        return(Expression$create("floor_temporal", x, options = opts))
+
+      } else { # for other values of week_start, compute using an offset
+        week_start_offset <- build_expr(
+          "cast",
+          Scalar$create((as.integer(week_start) - 1L) * 86400L, int64()),
+          options = cast_options(to_type = duration(unit = "s"))
+        )
+        interim <- build_expr("floor_temporal", x - week_start_offset, options = opts)
+        return(interim + week_start_offset)
+      }
+    }
+    Expression$create("floor_temporal", x, options = opts)
+  })
+
+  register_binding("ceiling_date", function(x, unit = "second",
+                                            change_on_boundary = NULL,
+                                            week_start = getOption("lubridate.week.start", 7)) {
+
+    opts <- parse_period_unit(unit)
+
+    if (is.null(change_on_boundary)) {
+      if (call_binding("is.Date", x)) {
+        change_on_boundary <- TRUE
+      } else {
+        change_on_boundary <- FALSE
+      }
+    }
+
+    if (change_on_boundary == FALSE) {
+      opts$ceil_is_strictly_greater <- 0L
+    }
+
+    if (change_on_boundary == TRUE) {
+      opts$ceil_is_strictly_greater <- 1L
+    }
+
+    if (opts$unit == 7L) {
+      if (week_start == 7) { # Sunday
+        opts$week_starts_monday <- 0L
+        return(Expression$create("ceil_temporal", x, options = opts))
+
+      } else if (week_start == 1) { # Monday
+        opts$week_starts_monday <- 1L
+        return(Expression$create("ceil_temporal", x, options = opts))
+
+      } else { # for other values of week_start, compute using an offset
+        week_start_offset <- build_expr(
+          "cast",
+          Scalar$create((as.integer(week_start) - 1L) * 86400L, int64()),
+          options = cast_options(to_type = duration(unit = "s"))
+        )
+        interim <- build_expr("ceil_temporal", x - week_start_offset, options = opts)
+        return(interim + week_start_offset)
+      }
+    }
+    return(Expression$create("ceil_temporal", x, options = opts))
+  })
+
+  register_binding("leap_year", function(date) {
+    year <- Expression$create("year", date)
+    (year %% 4 == 0) & ((year %% 100 != 0) | (year %% 400 == 0))
+  })

Review Comment:
   I think this is solved with a c++ kernel now and can be removed? I might have reintroduced it with my rebase.



##########
r/tests/testthat/test-dplyr-funcs-datetime.R:
##########
@@ -1965,3 +1973,421 @@ test_that("lubridate's fast_strptime", {
       collect()
   )
 })
+
+test_that("round/floor/ceiling on datetime (to nearest second)", {
+
+  skip_on_os("windows") # https://issues.apache.org/jira/browse/ARROW-13168
+
+  compare_dplyr_binding(
+    .input %>%
+      mutate(
+        out_1 = round_date(datetime),
+        out_2 = floor_date(datetime),
+        out_3 = ceiling_date(datetime, change_on_boundary = FALSE),
+      ) %>%
+      collect(),
+    test_df
+  )
+})
+
+test_that("period unit abbreviation", {
+
+  skip_on_os("windows") # https://issues.apache.org/jira/browse/ARROW-13168
+
+  compare_dplyr_binding(
+    .input %>%
+      mutate(
+        out_1 = round_date(datetime, "minute"),
+        out_2 = round_date(datetime, "minutes"),
+        out_3 = round_date(datetime, "mins"),
+      ) %>%
+      collect(),
+    test_df
+  )
+})
+
+test_that("period unit extracts integer multiples", {
+
+  skip_on_os("windows") # https://issues.apache.org/jira/browse/ARROW-13168
+
+  compare_dplyr_binding(
+    .input %>%
+      mutate(
+        out_1 = round_date(datetime, "1 minute"),
+        out_2 = round_date(datetime, "2 minutes"),
+        out_3 = round_date(datetime, "10 minutes")
+      ) %>%
+      collect(),
+    test_df
+  )
+})
+
+# lubridate errors when 60 sec/60 min/24 hour thresholds exceeded.
+# this test checks that arrow does too.
+test_that("period unit maxima are enforced", {
+
+  skip_on_os("windows") # https://issues.apache.org/jira/browse/ARROW-13168
+
+  expect_error(
+    call_binding("round_date", Expression$scalar(Sys.time()), "61 seconds"),
+    "Rounding with second > 60 is not supported"
+  )
+
+  expect_error(
+    call_binding("round_date", Expression$scalar(Sys.time()), "61 minutes"),
+    "Rounding with minute > 60 is not supported"
+  )
+
+  expect_error(
+    call_binding("round_date", Expression$scalar(Sys.time()), "25 hours"),
+    "Rounding with hour > 24 is not supported"
+  )
+
+})
+
+test_that("datetime rounding between 1sec and 1day", {
+
+  skip_on_os("windows") # https://issues.apache.org/jira/browse/ARROW-13168
+
+  compare_dplyr_binding(
+    .input %>%
+      mutate(
+        out_1 = round_date(datetime, "second"),
+        out_2 = round_date(datetime, "minute"),
+        out_3 = round_date(datetime, "hour"),
+        out_4 = round_date(datetime, "day")
+      ) %>%
+      collect(),
+    test_df
+  )
+})
+
+# lubridate doesn't accept millisecond, microsecond or nanosecond descriptors:
+# instead it supports corresponding fractions of 1 second. these tests added to
+# that arrow verify that fractional second inputs to arrow mirror lubridate
+
+test_that("datetime rounding below 1sec", {
+
+  skip_on_os("windows") # https://issues.apache.org/jira/browse/ARROW-13168
+
+  expect_equal(
+    test_df %>%
+      arrow_table() %>%
+      mutate(out = round_date(datetime, ".001 second")) %>%
+      collect(),
+
+    test_df %>%
+      arrow_table() %>%
+      mutate(out = round_date(datetime, "1 millisecond")) %>%
+      collect()
+  )
+
+  expect_equal(
+    test_df %>%
+      arrow_table() %>%
+      mutate(out = round_date(datetime, ".000001 second")) %>%
+      collect(),
+
+    test_df %>%
+      arrow_table() %>%
+      mutate(out = round_date(datetime, "1 microsecond")) %>%
+      collect()
+  )
+
+  expect_equal(
+    test_df %>%
+      arrow_table() %>%
+      mutate(out = round_date(datetime, ".000000001 second")) %>%
+      collect(),
+
+    test_df %>%
+      arrow_table() %>%
+      mutate(out = round_date(datetime, "1 nanosecond")) %>%
+      collect()
+  )
+
+  compare_dplyr_binding(
+    .input %>%
+      mutate(
+        out_1 = round_date(datetime, ".01 second"),
+        out_2 = round_date(datetime, ".001 second"),
+        out_3 = round_date(datetime, ".00001 second")
+      ) %>%
+      collect(),
+    test_df
+  )
+})
+
+test_that("datetime round/floor/ceil to month/quarter/year", {
+
+  skip_on_os("windows") # https://issues.apache.org/jira/browse/ARROW-13168
+
+  compare_dplyr_binding(
+    .input %>%
+      mutate(
+        out_1 = round_date(datetime, "month"),
+        out_2 = round_date(datetime, "quarter"),
+        out_3 = round_date(datetime, "year"),
+      ) %>%
+      collect(),
+    test_df_v2
+  )
+
+  compare_dplyr_binding(
+    .input %>%
+      mutate(
+        out_1 = floor_date(datetime, "month"),
+        out_2 = floor_date(datetime, "quarter"),
+        out_3 = floor_date(datetime, "year"),
+      ) %>%
+      collect(),
+    test_df_v2
+  )
+
+  compare_dplyr_binding(
+    .input %>%
+      mutate(
+        out_1 = ceiling_date(datetime, "month", change_on_boundary = FALSE),
+        out_2 = ceiling_date(datetime, "quarter", change_on_boundary = FALSE),
+        out_3 = ceiling_date(datetime, "year", change_on_boundary = FALSE),
+      ) %>%
+      collect(),
+    test_df_v2
+  )
+})
+
+
+
+boundary_times <- tibble::tibble(
+  datetime = as.POSIXct(strptime(c(
+    "2022-05-10 00:00:00", # boundary for week (Sunday / week_start = 7)
+    "2022-03-10 00:00:00", # boundary for: day, hour, minute, second, millisecond
+    "2022-03-10 00:00:01", # boundary for: second, millisecond
+    "2022-03-10 00:01:00", # boundary for: second, millisecond, minute
+    "2022-03-10 01:00:00"  # boundary for: second, millisecond, minute, hour
+  ), tz="UTC", format = "%F %T"))
+)
+
+check_boundary_with_unit <- function(unit, ...) {
+  compare_dplyr_binding(
+    .input %>%
+      mutate(
+        cob_null = ceiling_date(datetime, unit, change_on_boundary = NULL),
+        cob_true = ceiling_date(datetime, unit, change_on_boundary = TRUE),
+        cob_false = ceiling_date(datetime, unit, change_on_boundary = FALSE)
+      ) %>%
+      collect(),
+    boundary_times,
+    ...
+  )
+}
+
+test_that("ceiling_time works with change_on_boundary: unit = day", {
+  check_boundary_with_unit("day")
+})
+test_that("ceiling_time works with change_on_boundary: unit = hour", {
+  check_boundary_with_unit("hour")
+})
+test_that("ceiling_time works with change_on_boundary: unit = minute", {
+  check_boundary_with_unit("minute", tolerance = .001) # some weirdness with "2022-03-10 00:00:01", possibly floating point?
+})
+test_that("ceiling_time works with change_on_boundary: unit = second", {
+  check_boundary_with_unit("second")
+})
+test_that("ceiling_time works with change_on_boundary: unit = millisecond", {
+  check_boundary_with_unit(".001 second")
+})
+
+
+# NOTE: until 16142 is resolved, this test has to be written to avoid the
+# 32-bit temporal array misinterpreted as 64-bit bug. The easiest solution
+# is to never use an arrow array of length greater than 1:
+# https://issues.apache.org/jira/browse/ARROW-16142

Review Comment:
   Sorry I didn't get around to ARROW-16142 yet. Nice note! It will be useful when we get to it.



##########
r/tests/testthat/test-dplyr-funcs-datetime.R:
##########
@@ -1965,3 +1973,421 @@ test_that("lubridate's fast_strptime", {
       collect()
   )
 })
+
+test_that("round/floor/ceiling on datetime (to nearest second)", {
+
+  skip_on_os("windows") # https://issues.apache.org/jira/browse/ARROW-13168

Review Comment:
   ARROW-13168 was resolved for R (but not Python) on Windows. So this and the other below could probably be 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