Merge pull request #60598 from jrdi/week-default-mode

Add setting `first_day_of_week` for function `toStartOfInterval`
This commit is contained in:
Robert Schulze 2024-04-17 08:31:46 +00:00 committed by GitHub
commit 61df8aa466
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
12 changed files with 137 additions and 64 deletions

View File

@ -4384,6 +4384,17 @@ Possible values:
Default value: `ignore`.
## first_day_of_week
The first day of the week assumed by [`toStartOfInterval`](../../sql-reference/functions/date-time-functions.md#toStartOfInterval) function when using weeks as unit.
Possible values:
- Monday - Week starts on Monday
- Sunday - Week starts on Sunday
Default value: 'Monday'.
## optimize_move_to_prewhere {#optimize_move_to_prewhere}
Enables or disables automatic [PREWHERE](../../sql-reference/statements/select/prewhere.md) optimization in [SELECT](../../sql-reference/statements/select/index.md) queries.

View File

@ -1052,7 +1052,7 @@ toStartOfWeek(t[, mode[, timezone]])
**Arguments**
- `t` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
- `mode` - determines the first day of the week as described in the [toWeek()](date-time-functions#toweek) function
- `mode` - determines the first day of the week as described in the [toWeek()](date-time-functions#toweek) function. Default: 0
- `timezone` - Optional parameter, it behaves like any other conversion function
**Returned value**
@ -1413,7 +1413,7 @@ toStartOfFifteenMinutes(toDateTime('2023-04-21 10:20:00')): 2023-04-21 10:15:00
toStartOfFifteenMinutes(toDateTime('2023-04-21 10:23:00')): 2023-04-21 10:15:00
```
## toStartOfInterval(date_or_date_with_time, INTERVAL x unit \[, time_zone\])
## toStartOfInterval
This function generalizes other `toStartOf*()` functions. For example,
- `toStartOfInterval(t, INTERVAL 1 year)` returns the same as `toStartOfYear(t)`,
@ -1440,6 +1440,8 @@ The calculation is performed relative to specific points in time:
(*) hour intervals are special: the calculation is always performed relative to 00:00:00 (midnight) of the current day. As a result, only
hour values between 1 and 23 are useful.
If unit `week` was specified, `toStartOfInterval` assumes by default that weeks start on Monday. You can change this behavior with setting [`first_day_of_week`](../../operations/settings/settings.md/#first-day-of-week)
**See Also**
- [date_trunc](#date_trunc)
@ -1673,7 +1675,7 @@ Like [fromDaysSinceYearZero](#fromDaysSinceYearZero) but returns a [Date32](../.
Returns the `unit` component of the difference between `startdate` and `enddate`. The difference is calculated using a precision of 1 nanosecond.
E.g. the difference between `2021-12-29` and `2022-01-01` is 3 days for `day` unit, 0 months for `month` unit, 0 years for `year` unit.
For an alternative to `age`, see function `date\_diff`.
For an alternative to `age`, see function `date_diff`.
**Syntax**
@ -1747,9 +1749,9 @@ Result:
Returns the count of the specified `unit` boundaries crossed between the `startdate` and the `enddate`.
The difference is calculated using relative units, e.g. the difference between `2021-12-29` and `2022-01-01` is 3 days for unit `day` (see [toRelativeDayNum](#torelativedaynum)), 1 month for unit `month` (see [toRelativeMonthNum](#torelativemonthnum)) and 1 year for unit `year` (see [toRelativeYearNum](#torelativeyearnum)).
If unit `week` was specified, `date\_diff` assumes that weeks start on Monday. Note that this behavior is different from that of function `toWeek()` in which weeks start by default on Sunday.
If unit `week` was specified, `date_diff` assumes that weeks start on Monday. Note that this behavior is different from that of function `toWeek()` in which weeks start by default on Sunday.
For an alternative to `date\_diff`, see function `age`.
For an alternative to `date_diff`, see function `age`.
**Syntax**
@ -2843,7 +2845,7 @@ Result:
## fromUnixTimestamp
This function converts a Unix timestamp to a calendar date and a time of a day.
This function converts a Unix timestamp to a calendar date and a time of a day.
It can be called in two ways:

View File

@ -1048,16 +1048,20 @@ public:
template <typename Date>
requires std::is_same_v<Date, DayNum> || std::is_same_v<Date, ExtendedDayNum>
auto toStartOfWeekInterval(Date d, UInt64 weeks) const
auto toStartOfWeekInterval(Date d, UInt64 weeks, UInt8 week_mode) const
{
if (weeks == 1)
return toFirstDayNumOfWeek(d);
return toFirstDayNumOfWeek(d, week_mode);
bool monday_first_mode = week_mode & static_cast<UInt8>(WeekModeFlag::MONDAY_FIRST);
// January 1st 1970 was Thursday so we need this 4-days offset to make weeks start on Monday, or
// 3 days to start on Sunday.
auto offset = monday_first_mode ? 4 : 3;
UInt64 days = weeks * 7;
// January 1st 1970 was Thursday so we need this 4-days offset to make weeks start on Monday.
if constexpr (std::is_same_v<Date, DayNum>)
return DayNum(4 + (d - 4) / days * days);
return DayNum(offset + (d - offset) / days * days);
else
return ExtendedDayNum(static_cast<Int32>(4 + (d - 4) / days * days));
return ExtendedDayNum(static_cast<Int32>(offset + (d - offset) / days * days));
}
template <typename Date>

View File

@ -549,6 +549,7 @@ class IColumn;
M(Bool, formatdatetime_parsedatetime_m_is_month_name, true, "Formatter '%M' in functions 'formatDateTime()' and 'parseDateTime()' print/parse the month name instead of minutes.", 0) \
M(Bool, parsedatetime_parse_without_leading_zeros, true, "Formatters '%c', '%l' and '%k' in function 'parseDateTime()' parse months and hours without leading zeros.", 0) \
M(Bool, formatdatetime_format_without_leading_zeros, false, "Formatters '%c', '%l' and '%k' in function 'formatDateTime()' print months and hours without leading zeros.", 0) \
M(FirstDayOfWeek, first_day_of_week, FirstDayOfWeek::Monday, "The first day of the week (Monday or Sunday) used by date/time functions (default: Monday).", 0) \
\
M(UInt64, max_partitions_per_insert_block, 100, "Limit maximum number of partitions in single INSERTed block. Zero means unlimited. Throw exception if the block contains too many partitions. This setting is a safety threshold, because using large number of partitions is a common misconception.", 0) \
M(Bool, throw_on_max_partitions_per_insert_block, true, "Used with max_partitions_per_insert_block. If true (default), an exception will be thrown when max_partitions_per_insert_block is reached. If false, details of the insert query reaching this limit with the number of partitions will be logged. This can be useful if you're trying to understand the impact on users when changing max_partitions_per_insert_block.", 0) \

View File

@ -90,6 +90,7 @@ static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> sett
{"lightweight_deletes_sync", 2, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes"},
{"query_cache_system_table_handling", "save", "throw", "The query cache no longer caches results of queries against system tables"},
{"input_format_hive_text_allow_variable_number_of_columns", false, true, "Ignore extra columns in Hive Text input (if file has more columns than expected) and treat missing fields in Hive Text input as default values."},
{"first_day_of_week", "Monday", "Monday", "Added a setting for the first day of the week for date/time functions"},
}},
{"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"},
{"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"},

View File

@ -229,4 +229,8 @@ IMPLEMENT_SETTING_ENUM(SQLSecurityType, ErrorCodes::BAD_ARGUMENTS,
{{"DEFINER", SQLSecurityType::DEFINER},
{"INVOKER", SQLSecurityType::INVOKER},
{"NONE", SQLSecurityType::NONE}})
IMPLEMENT_SETTING_ENUM(FirstDayOfWeek, ErrorCodes::BAD_ARGUMENTS,
{{"Monday", FirstDayOfWeek::Monday},
{"Sunday", FirstDayOfWeek::Sunday}})
}

View File

@ -370,4 +370,12 @@ DECLARE_SETTING_ENUM(SchemaInferenceMode)
DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeOverflowBehavior, FormatSettings::DateTimeOverflowBehavior)
DECLARE_SETTING_ENUM(SQLSecurityType)
enum class FirstDayOfWeek
{
Monday,
Sunday
};
DECLARE_SETTING_ENUM(FirstDayOfWeek)
}

View File

@ -478,19 +478,19 @@ static constexpr auto TO_START_OF_INTERVAL_NAME = "toStartOfInterval";
template <>
struct ToStartOfInterval<IntervalKind::Kind::Nanosecond>
{
static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64)
static UInt32 execute(UInt16, Int64, Int64, UInt8, const DateLUTImpl &)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64)
static UInt32 execute(Int32, Int64, Int64, UInt8, const DateLUTImpl &)
{
throwDate32IsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32, Int64, const DateLUTImpl &, Int64)
static UInt32 execute(UInt32, Int64, Int64, UInt8, const DateLUTImpl &)
{
throwDateTimeIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static Int64 execute(Int64 t, Int64 nanoseconds, const DateLUTImpl &, Int64 scale_multiplier)
static Int64 execute(Int64 t, Int64 nanoseconds, Int64 scale_multiplier, UInt8, const DateLUTImpl &)
{
if (scale_multiplier < 1000000000)
{
@ -513,19 +513,19 @@ struct ToStartOfInterval<IntervalKind::Kind::Nanosecond>
template <>
struct ToStartOfInterval<IntervalKind::Kind::Microsecond>
{
static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64)
static UInt32 execute(UInt16, Int64, Int64, UInt8, const DateLUTImpl &)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64)
static UInt32 execute(Int32, Int64, Int64, UInt8, const DateLUTImpl &)
{
throwDate32IsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32, Int64, const DateLUTImpl &, Int64)
static UInt32 execute(UInt32, Int64, Int64, UInt8, const DateLUTImpl &)
{
throwDateTimeIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static Int64 execute(Int64 t, Int64 microseconds, const DateLUTImpl &, Int64 scale_multiplier)
static Int64 execute(Int64 t, Int64 microseconds, Int64 scale_multiplier, UInt8, const DateLUTImpl &)
{
if (scale_multiplier < 1000000)
{
@ -556,19 +556,19 @@ struct ToStartOfInterval<IntervalKind::Kind::Microsecond>
template <>
struct ToStartOfInterval<IntervalKind::Kind::Millisecond>
{
static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64)
static UInt32 execute(UInt16, Int64, Int64, UInt8, const DateLUTImpl &)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64)
static UInt32 execute(Int32, Int64, Int64, UInt8, const DateLUTImpl &)
{
throwDate32IsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32, Int64, const DateLUTImpl &, Int64)
static UInt32 execute(UInt32, Int64, Int64, UInt8, const DateLUTImpl &)
{
throwDateTimeIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static Int64 execute(Int64 t, Int64 milliseconds, const DateLUTImpl &, Int64 scale_multiplier)
static Int64 execute(Int64 t, Int64 milliseconds, Int64 scale_multiplier, UInt8, const DateLUTImpl &)
{
if (scale_multiplier < 1000)
{
@ -599,19 +599,19 @@ struct ToStartOfInterval<IntervalKind::Kind::Millisecond>
template <>
struct ToStartOfInterval<IntervalKind::Kind::Second>
{
static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64)
static UInt32 execute(UInt16, Int64, Int64, UInt8, const DateLUTImpl &)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64)
static UInt32 execute(Int32, Int64, Int64, UInt8, const DateLUTImpl &)
{
throwDate32IsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32 t, Int64 seconds, const DateLUTImpl & time_zone, Int64)
static UInt32 execute(UInt32 t, Int64 seconds, Int64, UInt8, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfSecondInterval(t, seconds);
}
static Int64 execute(Int64 t, Int64 seconds, const DateLUTImpl & time_zone, Int64 scale_multiplier)
static Int64 execute(Int64 t, Int64 seconds, Int64 scale_multiplier, UInt8, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfSecondInterval(t / scale_multiplier, seconds);
}
@ -620,19 +620,19 @@ struct ToStartOfInterval<IntervalKind::Kind::Second>
template <>
struct ToStartOfInterval<IntervalKind::Kind::Minute>
{
static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64)
static UInt32 execute(UInt16, Int64, Int64, UInt8, const DateLUTImpl &)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64)
static UInt32 execute(Int32, Int64, Int64, UInt8, const DateLUTImpl &)
{
throwDate32IsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32 t, Int64 minutes, const DateLUTImpl & time_zone, Int64)
static UInt32 execute(UInt32 t, Int64 minutes, Int64, UInt8, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfMinuteInterval(t, minutes);
}
static Int64 execute(Int64 t, Int64 minutes, const DateLUTImpl & time_zone, Int64 scale_multiplier)
static Int64 execute(Int64 t, Int64 minutes, Int64 scale_multiplier, UInt8, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfMinuteInterval(t / scale_multiplier, minutes);
}
@ -641,19 +641,19 @@ struct ToStartOfInterval<IntervalKind::Kind::Minute>
template <>
struct ToStartOfInterval<IntervalKind::Kind::Hour>
{
static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64)
static UInt32 execute(UInt16, Int64, Int64, UInt8, const DateLUTImpl &)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64)
static UInt32 execute(Int32, Int64, Int64, UInt8, const DateLUTImpl &)
{
throwDate32IsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32 t, Int64 hours, const DateLUTImpl & time_zone, Int64)
static UInt32 execute(UInt32 t, Int64 hours, Int64, UInt8, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfHourInterval(t, hours);
}
static Int64 execute(Int64 t, Int64 hours, const DateLUTImpl & time_zone, Int64 scale_multiplier)
static Int64 execute(Int64 t, Int64 hours, Int64 scale_multiplier, UInt8, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfHourInterval(t / scale_multiplier, hours);
}
@ -662,19 +662,19 @@ struct ToStartOfInterval<IntervalKind::Kind::Hour>
template <>
struct ToStartOfInterval<IntervalKind::Kind::Day>
{
static UInt32 execute(UInt16 d, Int64 days, const DateLUTImpl & time_zone, Int64)
static UInt32 execute(UInt16 d, Int64 days, Int64, UInt8, const DateLUTImpl & time_zone)
{
return static_cast<UInt32>(time_zone.toStartOfDayInterval(ExtendedDayNum(d), days));
}
static UInt32 execute(Int32 d, Int64 days, const DateLUTImpl & time_zone, Int64)
static UInt32 execute(Int32 d, Int64 days, Int64, UInt8, const DateLUTImpl & time_zone)
{
return static_cast<UInt32>(time_zone.toStartOfDayInterval(ExtendedDayNum(d), days));
}
static UInt32 execute(UInt32 t, Int64 days, const DateLUTImpl & time_zone, Int64)
static UInt32 execute(UInt32 t, Int64 days, Int64, UInt8, const DateLUTImpl & time_zone)
{
return static_cast<UInt32>(time_zone.toStartOfDayInterval(time_zone.toDayNum(t), days));
}
static Int64 execute(Int64 t, Int64 days, const DateLUTImpl & time_zone, Int64 scale_multiplier)
static Int64 execute(Int64 t, Int64 days, Int64 scale_multiplier, UInt8, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfDayInterval(time_zone.toDayNum(t / scale_multiplier), days);
}
@ -683,40 +683,40 @@ struct ToStartOfInterval<IntervalKind::Kind::Day>
template <>
struct ToStartOfInterval<IntervalKind::Kind::Week>
{
static UInt16 execute(UInt16 d, Int64 weeks, const DateLUTImpl & time_zone, Int64)
static UInt16 execute(UInt16 d, Int64 weeks, Int64, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfWeekInterval(DayNum(d), weeks);
return time_zone.toStartOfWeekInterval(DayNum(d), weeks, week_mode);
}
static UInt16 execute(Int32 d, Int64 weeks, const DateLUTImpl & time_zone, Int64)
static UInt16 execute(Int32 d, Int64 weeks, Int64, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfWeekInterval(ExtendedDayNum(d), weeks);
return time_zone.toStartOfWeekInterval(ExtendedDayNum(d), weeks, week_mode);
}
static UInt16 execute(UInt32 t, Int64 weeks, const DateLUTImpl & time_zone, Int64)
static UInt16 execute(UInt32 t, Int64 weeks, Int64, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t), weeks);
return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t), weeks, week_mode);
}
static UInt16 execute(Int64 t, Int64 weeks, const DateLUTImpl & time_zone, Int64 scale_multiplier)
static UInt16 execute(Int64 t, Int64 weeks, Int64 scale_multiplier, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t / scale_multiplier), weeks);
return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t / scale_multiplier), weeks, week_mode);
}
};
template <>
struct ToStartOfInterval<IntervalKind::Kind::Month>
{
static UInt16 execute(UInt16 d, Int64 months, const DateLUTImpl & time_zone, Int64)
static UInt16 execute(UInt16 d, Int64 months, Int64, UInt8, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfMonthInterval(DayNum(d), months);
}
static UInt16 execute(Int32 d, Int64 months, const DateLUTImpl & time_zone, Int64)
static UInt16 execute(Int32 d, Int64 months, Int64, UInt8, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfMonthInterval(ExtendedDayNum(d), months);
}
static UInt16 execute(UInt32 t, Int64 months, const DateLUTImpl & time_zone, Int64)
static UInt16 execute(UInt32 t, Int64 months, Int64, UInt8, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t), months);
}
static UInt16 execute(Int64 t, Int64 months, const DateLUTImpl & time_zone, Int64 scale_multiplier)
static UInt16 execute(Int64 t, Int64 months, Int64 scale_multiplier, UInt8, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t / scale_multiplier), months);
}
@ -725,19 +725,19 @@ struct ToStartOfInterval<IntervalKind::Kind::Month>
template <>
struct ToStartOfInterval<IntervalKind::Kind::Quarter>
{
static UInt16 execute(UInt16 d, Int64 quarters, const DateLUTImpl & time_zone, Int64)
static UInt16 execute(UInt16 d, Int64 quarters, Int64, UInt8, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfQuarterInterval(DayNum(d), quarters);
}
static UInt16 execute(Int32 d, Int64 quarters, const DateLUTImpl & time_zone, Int64)
static UInt16 execute(Int32 d, Int64 quarters, Int64, UInt8, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfQuarterInterval(ExtendedDayNum(d), quarters);
}
static UInt16 execute(UInt32 t, Int64 quarters, const DateLUTImpl & time_zone, Int64)
static UInt16 execute(UInt32 t, Int64 quarters, Int64, UInt8, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t), quarters);
}
static UInt16 execute(Int64 t, Int64 quarters, const DateLUTImpl & time_zone, Int64 scale_multiplier)
static UInt16 execute(Int64 t, Int64 quarters, Int64 scale_multiplier, UInt8, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t / scale_multiplier), quarters);
}
@ -746,19 +746,19 @@ struct ToStartOfInterval<IntervalKind::Kind::Quarter>
template <>
struct ToStartOfInterval<IntervalKind::Kind::Year>
{
static UInt16 execute(UInt16 d, Int64 years, const DateLUTImpl & time_zone, Int64)
static UInt16 execute(UInt16 d, Int64 years, Int64, UInt8, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfYearInterval(DayNum(d), years);
}
static UInt16 execute(Int32 d, Int64 years, const DateLUTImpl & time_zone, Int64)
static UInt16 execute(Int32 d, Int64 years, Int64, UInt8, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfYearInterval(ExtendedDayNum(d), years);
}
static UInt16 execute(UInt32 t, Int64 years, const DateLUTImpl & time_zone, Int64)
static UInt16 execute(UInt32 t, Int64 years, Int64, UInt8, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfYearInterval(time_zone.toDayNum(t), years);
}
static UInt16 execute(Int64 t, Int64 years, const DateLUTImpl & time_zone, Int64 scale_multiplier)
static UInt16 execute(Int64 t, Int64 years, Int64 scale_multiplier, UInt8, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfYearInterval(time_zone.toDayNum(t / scale_multiplier), years);
}

View File

@ -39,9 +39,17 @@ struct ToStartOfTransform;
TRANSFORM_DATE(Year)
TRANSFORM_DATE(Quarter)
TRANSFORM_DATE(Month)
TRANSFORM_DATE(Week)
#undef TRANSFORM_DATE
template <>
struct ToStartOfTransform<IntervalKind::Kind::Week>
{
static auto execute(UInt32 t, UInt64 delta, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t), delta, /*week_mode*/ 1);
}
};
template <>
struct ToStartOfTransform<IntervalKind::Kind::Day>
{

View File

@ -2,6 +2,7 @@
#include <Columns/ColumnsNumber.h>
#include <Common/DateLUTImpl.h>
#include <Common/IntervalKind.h>
#include <Core/SettingsEnums.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
@ -9,6 +10,7 @@
#include <Functions/DateTimeTransforms.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
#include <Interpreters/Context.h>
#include <IO/WriteHelpers.h>
@ -26,9 +28,13 @@ namespace ErrorCodes
class FunctionToStartOfInterval : public IFunction
{
public:
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionToStartOfInterval>(); }
static constexpr auto name = "toStartOfInterval";
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionToStartOfInterval>(context); }
explicit FunctionToStartOfInterval(ContextPtr context)
: first_day_of_week(context->getSettingsRef().first_day_of_week)
{
}
String getName() const override { return name; }
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
@ -247,13 +253,16 @@ private:
auto & result_data = col_to->getData();
result_data.resize(size);
Int64 scale_multiplier = DecimalUtils::scaleMultiplier<DateTime64>(scale);
const Int64 scale_multiplier = DecimalUtils::scaleMultiplier<DateTime64>(scale);
const UInt8 week_mode = (first_day_of_week == FirstDayOfWeek::Monday) ? 1 : 0;
for (size_t i = 0; i != size; ++i)
result_data[i] = static_cast<ResultFieldType>(ToStartOfInterval<unit>::execute(time_data[i], num_units, time_zone, scale_multiplier));
result_data[i] = static_cast<ResultFieldType>(ToStartOfInterval<unit>::execute(time_data[i], num_units, scale_multiplier, week_mode, time_zone));
return result_col;
}
const FirstDayOfWeek first_day_of_week;
};
REGISTER_FUNCTION(ToStartOfInterval)

View File

@ -0,0 +1,4 @@
-- toStartOfInterval
2024-01-02 00:00:00 2024-01-01 2023-12-25
2024-01-02 00:00:00 2024-01-01 2023-12-25
2024-01-02 00:00:00 2023-12-31 2023-12-24

View File

@ -0,0 +1,21 @@
-- Testing behavior of date/time functions under setting 'first_day_of_week'.
SELECT '-- toStartOfInterval';
-- default behavior
SELECT
toDateTime('2024-01-02 00:00:00', 'UTC') dt,
toStartOfInterval(dt, INTERVAL 1 WEEK), -- Monday, Jan 01
toStartOfInterval(dt, INTERVAL 2 WEEK); -- Monday, Dec 25
SELECT
toDateTime('2024-01-02 00:00:00', 'UTC') dt,
toStartOfInterval(dt, INTERVAL 1 WEEK), -- Monday, Jan 01
toStartOfInterval(dt, INTERVAL 2 WEEK) -- Monday, Dec 25
SETTINGS first_day_of_week = 'Monday';
SELECT
toDateTime('2024-01-02 00:00:00', 'UTC') dt,
toStartOfInterval(dt, INTERVAL 1 WEEK), -- Sunday, Dec 31
toStartOfInterval(dt, INTERVAL 2 WEEK) -- Sunday, Dec 24
SETTINGS first_day_of_week = 'Sunday';