mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Add time function toStartOfInterval().
This commit is contained in:
parent
a95d2d153d
commit
0dd1dc242a
@ -24,6 +24,7 @@ void registerFunctionToStartOfFiveMinute(FunctionFactory &);
|
||||
void registerFunctionToStartOfTenMinutes(FunctionFactory &);
|
||||
void registerFunctionToStartOfFifteenMinutes(FunctionFactory &);
|
||||
void registerFunctionToStartOfHour(FunctionFactory &);
|
||||
void registerFunctionToStartOfInterval(FunctionFactory &);
|
||||
void registerFunctionToStartOfISOYear(FunctionFactory &);
|
||||
void registerFunctionToRelativeYearNum(FunctionFactory &);
|
||||
void registerFunctionToRelativeQuarterNum(FunctionFactory &);
|
||||
@ -86,6 +87,7 @@ void registerFunctionsDateTime(FunctionFactory & factory)
|
||||
registerFunctionToStartOfTenMinutes(factory);
|
||||
registerFunctionToStartOfFifteenMinutes(factory);
|
||||
registerFunctionToStartOfHour(factory);
|
||||
registerFunctionToStartOfInterval(factory);
|
||||
registerFunctionToStartOfISOYear(factory);
|
||||
registerFunctionToRelativeYearNum(factory);
|
||||
registerFunctionToRelativeQuarterNum(factory);
|
||||
|
301
dbms/src/Functions/toStartOfInterval.cpp
Normal file
301
dbms/src/Functions/toStartOfInterval.cpp
Normal file
@ -0,0 +1,301 @@
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeInterval.h>
|
||||
#include <Functions/DateTimeTransforms.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
}
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
static constexpr auto function_name = "toStartOfInterval";
|
||||
|
||||
template <DataTypeInterval::Kind unit>
|
||||
struct Transform;
|
||||
|
||||
template <>
|
||||
struct Transform<DataTypeInterval::Year>
|
||||
{
|
||||
static UInt16 execute(UInt16 d, UInt64 years, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toStartOfYearInterval(DayNum(d), years);
|
||||
}
|
||||
|
||||
static UInt16 execute(UInt32 t, UInt64 years, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toStartOfYearInterval(time_zone.toDayNum(t), years);
|
||||
}
|
||||
};
|
||||
|
||||
template <>
|
||||
struct Transform<DataTypeInterval::Quarter>
|
||||
{
|
||||
static UInt16 execute(UInt16 d, UInt64 quarters, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toStartOfQuarterInterval(DayNum(d), quarters);
|
||||
}
|
||||
|
||||
static UInt16 execute(UInt32 t, UInt64 quarters, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t), quarters);
|
||||
}
|
||||
};
|
||||
|
||||
template <>
|
||||
struct Transform<DataTypeInterval::Month>
|
||||
{
|
||||
static UInt16 execute(UInt16 d, UInt64 months, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toStartOfMonthInterval(DayNum(d), months);
|
||||
}
|
||||
|
||||
static UInt16 execute(UInt32 t, UInt64 months, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t), months);
|
||||
}
|
||||
};
|
||||
|
||||
template <>
|
||||
struct Transform<DataTypeInterval::Week>
|
||||
{
|
||||
static UInt16 execute(UInt16 d, UInt64 weeks, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toStartOfWeekInterval(DayNum(d), weeks);
|
||||
}
|
||||
|
||||
static UInt16 execute(UInt32 t, UInt64 weeks, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t), weeks);
|
||||
}
|
||||
};
|
||||
|
||||
template <>
|
||||
struct Transform<DataTypeInterval::Day>
|
||||
{
|
||||
static UInt32 execute(UInt16 d, UInt64 days, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toStartOfDayInterval(DayNum(d), days);
|
||||
}
|
||||
|
||||
static UInt32 execute(UInt32 t, UInt64 days, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toStartOfDayInterval(time_zone.toDayNum(t), days);
|
||||
}
|
||||
};
|
||||
|
||||
template <>
|
||||
struct Transform<DataTypeInterval::Hour>
|
||||
{
|
||||
static UInt32 execute(UInt16, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); }
|
||||
|
||||
static UInt32 execute(UInt32 t, UInt64 hours, const DateLUTImpl & time_zone) { return time_zone.toStartOfHourInterval(t, hours); }
|
||||
};
|
||||
|
||||
template <>
|
||||
struct Transform<DataTypeInterval::Minute>
|
||||
{
|
||||
static UInt32 execute(UInt16, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); }
|
||||
|
||||
static UInt32 execute(UInt32 t, UInt64 minutes, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toStartOfMinuteInterval(t, minutes);
|
||||
}
|
||||
};
|
||||
|
||||
template <>
|
||||
struct Transform<DataTypeInterval::Second>
|
||||
{
|
||||
static UInt32 execute(UInt16, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); }
|
||||
|
||||
static UInt32 execute(UInt32 t, UInt64 seconds, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toStartOfSecondInterval(t, seconds);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
||||
class FunctionToStartOfInterval : public IFunction
|
||||
{
|
||||
public:
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionToStartOfInterval>(); }
|
||||
|
||||
static constexpr auto name = function_name;
|
||||
String getName() const override { return name; }
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
auto check_date_time_argument = [&] {
|
||||
if (!isDateOrDateTime(arguments[0].type))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName()
|
||||
+ ". Should be a date or a date with time",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
};
|
||||
|
||||
const DataTypeInterval * interval_type = nullptr;
|
||||
auto check_interval_argument = [&] {
|
||||
interval_type = checkAndGetDataType<DataTypeInterval>(arguments[1].type.get());
|
||||
if (!interval_type)
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[1].type->getName() + " of argument of function " + getName()
|
||||
+ ". Should be an interval of time",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
};
|
||||
|
||||
auto check_timezone_argument = [&] {
|
||||
if (!WhichDataType(arguments[2].type).isString())
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[2].type->getName() + " of argument of function " + getName()
|
||||
+ ". This argument is optional and must be a constant string with timezone name"
|
||||
". This argument is allowed only when the 1st argument has the type DateTime",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
};
|
||||
|
||||
if (arguments.size() == 2)
|
||||
{
|
||||
check_date_time_argument();
|
||||
check_interval_argument();
|
||||
}
|
||||
else if (arguments.size() == 3)
|
||||
{
|
||||
check_date_time_argument();
|
||||
check_interval_argument();
|
||||
check_timezone_argument();
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(
|
||||
"Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size())
|
||||
+ ", should be 2 or 3",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
}
|
||||
|
||||
if ((interval_type->getKind() == DataTypeInterval::Second) || (interval_type->getKind() == DataTypeInterval::Minute)
|
||||
|| (interval_type->getKind() == DataTypeInterval::Hour) || (interval_type->getKind() == DataTypeInterval::Day))
|
||||
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0));
|
||||
else
|
||||
return std::make_shared<DataTypeDate>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /* input_rows_count */) override
|
||||
{
|
||||
const auto & time_column = block.getByPosition(arguments[0]);
|
||||
const auto & interval_column = block.getByPosition(arguments[1]);
|
||||
const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(block, arguments, 2, 0);
|
||||
auto result_column = dispatchForColumns(time_column, interval_column, time_zone);
|
||||
block.getByPosition(result).column = std::move(result_column);
|
||||
}
|
||||
|
||||
bool hasInformationAboutMonotonicity() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
Monotonicity getMonotonicityForRange(const IDataType &, const Field &, const Field &) const override
|
||||
{
|
||||
return { true, true, true };
|
||||
}
|
||||
|
||||
private:
|
||||
ColumnPtr dispatchForColumns(
|
||||
const ColumnWithTypeAndName & time_column, const ColumnWithTypeAndName & interval_column, const DateLUTImpl & time_zone)
|
||||
{
|
||||
if (WhichDataType(time_column.type.get()).isDateTime())
|
||||
{
|
||||
const auto * time_column_vec = checkAndGetColumn<ColumnUInt32>(time_column.column.get());
|
||||
if (time_column_vec)
|
||||
return dispatchForIntervalColumn(*time_column_vec, interval_column, time_zone);
|
||||
}
|
||||
if (WhichDataType(time_column.type.get()).isDate())
|
||||
{
|
||||
const auto * time_column_vec = checkAndGetColumn<ColumnUInt16>(time_column.column.get());
|
||||
if (time_column_vec)
|
||||
return dispatchForIntervalColumn(*time_column_vec, interval_column, time_zone);
|
||||
}
|
||||
throw Exception(
|
||||
"Illegal column for first argument of function " + getName() + ". Must contain dates or dates with time",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
template <typename FromType>
|
||||
ColumnPtr dispatchForIntervalColumn(
|
||||
const ColumnVector<FromType> & time_column, const ColumnWithTypeAndName & interval_column, const DateLUTImpl & time_zone)
|
||||
{
|
||||
const auto * interval_type = checkAndGetDataType<DataTypeInterval>(interval_column.type.get());
|
||||
if (!interval_type)
|
||||
throw Exception(
|
||||
"Illegal column for second argument of function " + getName() + ", must be an interval of time.",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
const auto * interval_column_const_int64 = checkAndGetColumnConst<ColumnInt64>(interval_column.column.get());
|
||||
if (!interval_column_const_int64)
|
||||
throw Exception(
|
||||
"Illegal column for second argument of function " + getName() + ", must be a const interval of time.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
Int64 num_units = interval_column_const_int64->getValue<Int64>();
|
||||
if (num_units <= 0)
|
||||
throw Exception("Value for second argument of function " + getName() + " must be positive.", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
|
||||
switch (interval_type->getKind())
|
||||
{
|
||||
case DataTypeInterval::Second:
|
||||
return execute<FromType, UInt32, DataTypeInterval::Second>(time_column, num_units, time_zone);
|
||||
case DataTypeInterval::Minute:
|
||||
return execute<FromType, UInt32, DataTypeInterval::Minute>(time_column, num_units, time_zone);
|
||||
case DataTypeInterval::Hour:
|
||||
return execute<FromType, UInt32, DataTypeInterval::Hour>(time_column, num_units, time_zone);
|
||||
case DataTypeInterval::Day:
|
||||
return execute<FromType, UInt32, DataTypeInterval::Day>(time_column, num_units, time_zone);
|
||||
case DataTypeInterval::Week:
|
||||
return execute<FromType, UInt16, DataTypeInterval::Week>(time_column, num_units, time_zone);
|
||||
case DataTypeInterval::Month:
|
||||
return execute<FromType, UInt16, DataTypeInterval::Month>(time_column, num_units, time_zone);
|
||||
case DataTypeInterval::Quarter:
|
||||
return execute<FromType, UInt16, DataTypeInterval::Quarter>(time_column, num_units, time_zone);
|
||||
case DataTypeInterval::Year:
|
||||
return execute<FromType, UInt16, DataTypeInterval::Year>(time_column, num_units, time_zone);
|
||||
}
|
||||
|
||||
__builtin_unreachable();
|
||||
}
|
||||
|
||||
template <typename FromType, typename ToType, DataTypeInterval::Kind unit>
|
||||
ColumnPtr execute(const ColumnVector<FromType> & time_column, UInt64 num_units, const DateLUTImpl & time_zone)
|
||||
{
|
||||
const auto & time_data = time_column.getData();
|
||||
size_t size = time_column.size();
|
||||
auto result = ColumnVector<ToType>::create();
|
||||
auto & result_data = result->getData();
|
||||
result_data.resize(size);
|
||||
for (size_t i = 0; i != size; ++i)
|
||||
result_data[i] = Transform<unit>::execute(time_data[i], num_units, time_zone);
|
||||
return result;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
void registerFunctionToStartOfInterval(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionToStartOfInterval>();
|
||||
}
|
||||
|
||||
}
|
@ -83,6 +83,48 @@
|
||||
2019-02-06 19:00:00
|
||||
2019-02-07 04:00:00
|
||||
2019-02-06 11:00:00
|
||||
2019-01-01
|
||||
2018-01-01
|
||||
2015-01-01
|
||||
2019-01-01
|
||||
2019-01-01
|
||||
2018-10-01
|
||||
2019-02-01
|
||||
2019-01-01
|
||||
2018-10-01
|
||||
2019-02-04
|
||||
2019-01-28
|
||||
2018-12-31
|
||||
2019-02-06 00:00:00
|
||||
2019-02-05 00:00:00
|
||||
2019-02-03 00:00:00
|
||||
2019-02-06 22:00:00
|
||||
2019-02-06 21:00:00
|
||||
2019-02-06 21:00:00
|
||||
2019-02-06 03:00:00
|
||||
2019-02-06 22:57:00
|
||||
2019-02-06 22:56:00
|
||||
2019-02-06 22:55:00
|
||||
2019-02-06 22:40:00
|
||||
2019-02-06 22:30:00
|
||||
2019-02-06 22:57:35
|
||||
2019-02-06 22:57:34
|
||||
2019-02-06 22:57:35
|
||||
2019-01-01
|
||||
2018-01-01
|
||||
2015-01-01
|
||||
2019-01-01
|
||||
2019-01-01
|
||||
2018-10-01
|
||||
2019-02-01
|
||||
2019-01-01
|
||||
2018-10-01
|
||||
2019-02-04
|
||||
2019-01-28
|
||||
2018-12-31
|
||||
2019-02-06 00:00:00
|
||||
2019-02-05 00:00:00
|
||||
2019-02-03 00:00:00
|
||||
44
|
||||
44
|
||||
44
|
||||
|
@ -142,6 +142,50 @@ SELECT toString(toStartOfHour(toDateTime(1549483055), 'Europe/London'), 'Europe/
|
||||
SELECT toString(toStartOfHour(toDateTime(1549483055), 'Asia/Tokyo'), 'Asia/Tokyo');
|
||||
SELECT toString(toStartOfHour(toDateTime(1549483055), 'Pacific/Pitcairn'), 'Pacific/Pitcairn');
|
||||
|
||||
/* toStartOfInterval */
|
||||
SELECT toStartOfInterval(toDateTime(1549483055), INTERVAL 1 year, 'Europe/Moscow');
|
||||
SELECT toStartOfInterval(toDateTime(1549483055), INTERVAL 2 year, 'Europe/Moscow');
|
||||
SELECT toStartOfInterval(toDateTime(1549483055), INTERVAL 5 year, 'Europe/Moscow');
|
||||
SELECT toStartOfInterval(toDateTime(1549483055), INTERVAL 1 quarter, 'Europe/Moscow');
|
||||
SELECT toStartOfInterval(toDateTime(1549483055), INTERVAL 2 quarter, 'Europe/Moscow');
|
||||
SELECT toStartOfInterval(toDateTime(1549483055), INTERVAL 3 quarter, 'Europe/Moscow');
|
||||
SELECT toStartOfInterval(toDateTime(1549483055), INTERVAL 1 month, 'Europe/Moscow');
|
||||
SELECT toStartOfInterval(toDateTime(1549483055), INTERVAL 2 month, 'Europe/Moscow');
|
||||
SELECT toStartOfInterval(toDateTime(1549483055), INTERVAL 5 month, 'Europe/Moscow');
|
||||
SELECT toStartOfInterval(toDateTime(1549483055), INTERVAL 1 week, 'Europe/Moscow');
|
||||
SELECT toStartOfInterval(toDateTime(1549483055), INTERVAL 2 week, 'Europe/Moscow');
|
||||
SELECT toStartOfInterval(toDateTime(1549483055), INTERVAL 6 week, 'Europe/Moscow');
|
||||
SELECT toString(toStartOfInterval(toDateTime(1549483055), INTERVAL 1 day, 'Europe/Moscow'), 'Europe/Moscow');
|
||||
SELECT toString(toStartOfInterval(toDateTime(1549483055), INTERVAL 2 day, 'Europe/Moscow'), 'Europe/Moscow');
|
||||
SELECT toString(toStartOfInterval(toDateTime(1549483055), INTERVAL 5 day, 'Europe/Moscow'), 'Europe/Moscow');
|
||||
SELECT toString(toStartOfInterval(toDateTime(1549483055), INTERVAL 1 hour, 'Europe/Moscow'), 'Europe/Moscow');
|
||||
SELECT toString(toStartOfInterval(toDateTime(1549483055), INTERVAL 2 hour, 'Europe/Moscow'), 'Europe/Moscow');
|
||||
SELECT toString(toStartOfInterval(toDateTime(1549483055), INTERVAL 6 hour, 'Europe/Moscow'), 'Europe/Moscow');
|
||||
SELECT toString(toStartOfInterval(toDateTime(1549483055), INTERVAL 24 hour, 'Europe/Moscow'), 'Europe/Moscow');
|
||||
SELECT toString(toStartOfInterval(toDateTime(1549483055), INTERVAL 1 minute, 'Europe/Moscow'), 'Europe/Moscow');
|
||||
SELECT toString(toStartOfInterval(toDateTime(1549483055), INTERVAL 2 minute, 'Europe/Moscow'), 'Europe/Moscow');
|
||||
SELECT toString(toStartOfInterval(toDateTime(1549483055), INTERVAL 5 minute, 'Europe/Moscow'), 'Europe/Moscow');
|
||||
SELECT toString(toStartOfInterval(toDateTime(1549483055), INTERVAL 20 minute, 'Europe/Moscow'), 'Europe/Moscow');
|
||||
SELECT toString(toStartOfInterval(toDateTime(1549483055), INTERVAL 90 minute, 'Europe/Moscow'), 'Europe/Moscow');
|
||||
SELECT toString(toStartOfInterval(toDateTime(1549483055), INTERVAL 1 second, 'Europe/Moscow'), 'Europe/Moscow');
|
||||
SELECT toString(toStartOfInterval(toDateTime(1549483055), INTERVAL 2 second, 'Europe/Moscow'), 'Europe/Moscow');
|
||||
SELECT toString(toStartOfInterval(toDateTime(1549483055), INTERVAL 5 second, 'Europe/Moscow'), 'Europe/Moscow');
|
||||
SELECT toStartOfInterval(toDate(17933), INTERVAL 1 year);
|
||||
SELECT toStartOfInterval(toDate(17933), INTERVAL 2 year);
|
||||
SELECT toStartOfInterval(toDate(17933), INTERVAL 5 year);
|
||||
SELECT toStartOfInterval(toDate(17933), INTERVAL 1 quarter);
|
||||
SELECT toStartOfInterval(toDate(17933), INTERVAL 2 quarter);
|
||||
SELECT toStartOfInterval(toDate(17933), INTERVAL 3 quarter);
|
||||
SELECT toStartOfInterval(toDate(17933), INTERVAL 1 month);
|
||||
SELECT toStartOfInterval(toDate(17933), INTERVAL 2 month);
|
||||
SELECT toStartOfInterval(toDate(17933), INTERVAL 5 month);
|
||||
SELECT toStartOfInterval(toDate(17933), INTERVAL 1 week);
|
||||
SELECT toStartOfInterval(toDate(17933), INTERVAL 2 week);
|
||||
SELECT toStartOfInterval(toDate(17933), INTERVAL 6 week);
|
||||
SELECT toString(toStartOfInterval(toDate(17933), INTERVAL 1 day, 'Europe/Moscow'), 'Europe/Moscow');
|
||||
SELECT toString(toStartOfInterval(toDate(17933), INTERVAL 2 day, 'Europe/Moscow'), 'Europe/Moscow');
|
||||
SELECT toString(toStartOfInterval(toDate(17933), INTERVAL 5 day, 'Europe/Moscow'), 'Europe/Moscow');
|
||||
|
||||
/* toRelativeYearNum */
|
||||
|
||||
SELECT toRelativeYearNum(toDateTime(1412106600), 'Europe/Moscow') - toRelativeYearNum(toDateTime(0), 'Europe/Moscow');
|
||||
|
@ -283,15 +283,7 @@ public:
|
||||
inline time_t toStartOfMinute(time_t t) const { return t / 60 * 60; }
|
||||
inline time_t toStartOfFiveMinute(time_t t) const { return t / 300 * 300; }
|
||||
inline time_t toStartOfFifteenMinutes(time_t t) const { return t / 900 * 900; }
|
||||
|
||||
inline time_t toStartOfTenMinutes(time_t t) const
|
||||
{
|
||||
if (offset_is_whole_number_of_hours_everytime)
|
||||
return t / 600 * 600;
|
||||
|
||||
time_t date = find(t).date;
|
||||
return date + (t - date) / 600 * 600;
|
||||
}
|
||||
inline time_t toStartOfTenMinutes(time_t t) const { return t / 600 * 600; }
|
||||
|
||||
inline time_t toStartOfHour(time_t t) const
|
||||
{
|
||||
@ -434,6 +426,71 @@ public:
|
||||
return toRelativeMinuteNum(lut[d].date);
|
||||
}
|
||||
|
||||
inline DayNum toStartOfYearInterval(DayNum d, UInt64 years) const
|
||||
{
|
||||
if (years == 1)
|
||||
return toFirstDayNumOfYear(d);
|
||||
return years_lut[(lut[d].year - DATE_LUT_MIN_YEAR) / years * years];
|
||||
}
|
||||
|
||||
inline DayNum toStartOfQuarterInterval(DayNum d, UInt64 quarters) const
|
||||
{
|
||||
if (quarters == 1)
|
||||
return toFirstDayNumOfQuarter(d);
|
||||
return toStartOfMonthInterval(d, quarters * 3);
|
||||
}
|
||||
|
||||
inline DayNum toStartOfMonthInterval(DayNum d, UInt64 months) const
|
||||
{
|
||||
if (months == 1)
|
||||
return toFirstDayNumOfMonth(d);
|
||||
const auto & date = lut[d];
|
||||
UInt32 month_total_index = (date.year - DATE_LUT_MIN_YEAR) * 12 + date.month - 1;
|
||||
return years_months_lut[month_total_index / months * months];
|
||||
}
|
||||
|
||||
inline DayNum toStartOfWeekInterval(DayNum d, UInt64 weeks) const
|
||||
{
|
||||
if (weeks == 1)
|
||||
return toFirstDayNumOfWeek(d);
|
||||
UInt64 days = weeks * 7;
|
||||
// January 1st 1970 was Thursday so we need this 4-days offset to make weeks start on Monday.
|
||||
return DayNum(4 + (d - 4) / days * days);
|
||||
}
|
||||
|
||||
inline time_t toStartOfDayInterval(DayNum d, UInt64 days) const
|
||||
{
|
||||
if (days == 1)
|
||||
return toDate(d);
|
||||
return lut[d / days * days].date;
|
||||
}
|
||||
|
||||
inline time_t toStartOfHourInterval(time_t t, UInt64 hours) const
|
||||
{
|
||||
if (hours == 1)
|
||||
return toStartOfHour(t);
|
||||
UInt64 seconds = hours * 3600;
|
||||
t = t / seconds * seconds;
|
||||
if (offset_is_whole_number_of_hours_everytime)
|
||||
return t;
|
||||
return toStartOfHour(t);
|
||||
}
|
||||
|
||||
inline time_t toStartOfMinuteInterval(time_t t, UInt64 minutes) const
|
||||
{
|
||||
if (minutes == 1)
|
||||
return toStartOfMinute(t);
|
||||
UInt64 seconds = 60 * minutes;
|
||||
return t / seconds * seconds;
|
||||
}
|
||||
|
||||
inline time_t toStartOfSecondInterval(time_t t, UInt64 seconds) const
|
||||
{
|
||||
if (seconds == 1)
|
||||
return t;
|
||||
return t / seconds * seconds;
|
||||
}
|
||||
|
||||
/// Create DayNum from year, month, day of month.
|
||||
inline DayNum makeDayNum(UInt16 year, UInt8 month, UInt8 day_of_month) const
|
||||
{
|
||||
|
Loading…
Reference in New Issue
Block a user