add week(), yearweek()

This commit is contained in:
Andy Yang 2019-06-09 23:19:15 +08:00
parent cb3a371756
commit 365d85e1f9
11 changed files with 509 additions and 487 deletions

View File

@ -1,154 +0,0 @@
#pragma once
#include <regex>
#include <Columns/ColumnString.h>
#include <Columns/ColumnVector.h>
#include <Core/Types.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/extractTimeZoneFromFunctionArguments.h>
#include <Common/Exception.h>
#include <common/DateLUTImpl.h>
/// Custom date defaults to January 1 ( 01-01 )
#define DEFAULT_CUSTOM_MONTH 1
#define DEFAULT_CUSTOM_DAY 1
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
/** CustomDate Transformations.
* Represents two functions - from datetime (UInt32) and from date (UInt16), both with custom_month and custom_day.
*/
static inline UInt32 dateIsNotSupported(const char * name)
{
throw Exception("Illegal type Date of argument for function " + std::string(name), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
/// This factor transformation will say that the function is monotone everywhere.
struct ZeroTransform
{
static inline UInt16 execute(UInt32, UInt8, UInt8, const DateLUTImpl &) { return 0; }
static inline UInt16 execute(UInt16, UInt8, UInt8, const DateLUTImpl &) { return 0; }
};
struct ToStartOfCustomYearImpl
{
static constexpr auto name = "toStartOfCustomYear";
static inline UInt16 execute(UInt32 t, UInt8 custom_month, UInt8 custom_day, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfCustomYear(time_zone.toDayNum(t), custom_month, custom_day);
}
static inline UInt16 execute(UInt16 d, UInt8 custom_month, UInt8 custom_day, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfCustomYear(DayNum(d), custom_month, custom_day);
}
using FactorTransform = ZeroTransform;
};
struct ToCustomYearImpl
{
static constexpr auto name = "toCustomYear";
static inline UInt16 execute(UInt32 t, UInt8 custom_month, UInt8 custom_day, const DateLUTImpl & time_zone)
{
return time_zone.toCustomYear(time_zone.toDayNum(t), custom_month, custom_day);
}
static inline UInt16 execute(UInt16 d, UInt8 custom_month, UInt8 custom_day, const DateLUTImpl & time_zone)
{
return time_zone.toCustomYear(DayNum(d), custom_month, custom_day);
}
using FactorTransform = ZeroTransform;
};
struct ToCustomWeekImpl
{
static constexpr auto name = "toCustomWeek";
static inline UInt8 execute(UInt32 t, UInt8 custom_month, UInt8 custom_day, const DateLUTImpl & time_zone)
{
return time_zone.toCustomWeek(time_zone.toDayNum(t), custom_month, custom_day);
}
static inline UInt8 execute(UInt16 d, UInt8 custom_month, UInt8 custom_day, const DateLUTImpl & time_zone)
{
return time_zone.toCustomWeek(DayNum(d), custom_month, custom_day);
}
using FactorTransform = ToCustomYearImpl;
};
template <typename FromType, typename ToType, typename Transform>
struct Transformer
{
static void vector(
const PaddedPODArray<FromType> & vec_from,
PaddedPODArray<ToType> & vec_to,
UInt8 custom_month,
UInt8 custom_day,
const DateLUTImpl & time_zone)
{
size_t size = vec_from.size();
vec_to.resize(size);
for (size_t i = 0; i < size; ++i)
vec_to[i] = Transform::execute(vec_from[i], custom_month, custom_day, time_zone);
}
};
template <typename FromType, typename ToType, typename Transform>
struct CustomDateTransformImpl
{
static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
{
using Op = Transformer<FromType, ToType, Transform>;
UInt8 custom_month = DEFAULT_CUSTOM_MONTH;
UInt8 custom_day = DEFAULT_CUSTOM_DAY;
// With custom date parameter, fommat: MM-DD
if (arguments.size() > 1)
{
auto * custom_date_column = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
if (custom_date_column)
{
String custom_date = custom_date_column->getValue<String>();
std::regex regex_mmdd("((1[0-2])|(0[1-9]))-(([12][0-9])|(3[01])|(0[1-9]))");
if (custom_date.length() == 5 && std::regex_match(custom_date, regex_mmdd))
{
custom_month = std::stoi(custom_date.substr(0, 2));
custom_day = std::stoi(custom_date.substr(3, 2));
}
else
throw Exception(
String("The second argument for function ") + Transform::name
+ " must be a constant string with custom date(MM-DD)",
ErrorCodes::ILLEGAL_COLUMN);
}
}
const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(block, arguments, 2, 0);
const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
if (const auto * sources = checkAndGetColumn<ColumnVector<FromType>>(source_col.get()))
{
auto col_to = ColumnVector<ToType>::create();
Op::vector(sources->getData(), col_to->getData(), custom_month, custom_day, time_zone);
block.getByPosition(result).column = std::move(col_to);
}
else
{
throw Exception(
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function "
+ Transform::name,
ErrorCodes::ILLEGAL_COLUMN);
}
}
};
}

View File

@ -0,0 +1,119 @@
#pragma once
#include <regex>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnsNumber.h>
#include <Core/Types.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/extractTimeZoneFromFunctionArguments.h>
#include <Common/Exception.h>
#include <common/DateLUTImpl.h>
/// The default mode value to use for the WEEK() function
#define DEFAULT_WEEK_MODE 0
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
/**
* CustomWeek Transformations.
*/
static inline UInt32 dateIsNotSupported(const char * name)
{
throw Exception("Illegal type Date of argument for function " + std::string(name), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
/// This factor transformation will say that the function is monotone everywhere.
struct ZeroTransform
{
static inline UInt16 execute(UInt32, UInt8, const DateLUTImpl &) { return 0; }
static inline UInt16 execute(UInt16, UInt8, const DateLUTImpl &) { return 0; }
};
struct WeekImpl
{
static constexpr auto name = "week";
static inline UInt8 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
UInt32 year = 0;
return time_zone.calc_week(time_zone.toDayNum(t), week_mode, &year);
}
static inline UInt8 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
UInt32 year = 0;
return time_zone.calc_week(DayNum(d), week_mode, &year);
}
using FactorTransform = ZeroTransform;
};
struct YearWeekImpl
{
static constexpr auto name = "yearWeek";
static inline UInt32 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.calc_yearWeek(time_zone.toDayNum(t), week_mode);
}
static inline UInt32 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.calc_yearWeek(DayNum(d), week_mode);
}
using FactorTransform = ZeroTransform;
};
template <typename FromType, typename ToType, typename Transform>
struct Transformer
{
static void
vector(const PaddedPODArray<FromType> & vec_from, PaddedPODArray<ToType> & vec_to, UInt8 week_mode, const DateLUTImpl & time_zone)
{
size_t size = vec_from.size();
vec_to.resize(size);
for (size_t i = 0; i < size; ++i)
vec_to[i] = Transform::execute(vec_from[i], week_mode, time_zone);
}
};
template <typename FromType, typename ToType, typename Transform>
struct CustomWeekTransformImpl
{
static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
{
using Op = Transformer<FromType, ToType, Transform>;
UInt8 week_mode = DEFAULT_WEEK_MODE;
if (arguments.size() > 1)
{
if (const auto week_mode_column = checkAndGetColumnConst<ColumnUInt8>(block.getByPosition(arguments[1]).column.get()))
week_mode = week_mode_column->getValue<UInt8>();
}
const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(block, arguments, 2, 0);
const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
if (const auto * sources = checkAndGetColumn<ColumnVector<FromType>>(source_col.get()))
{
auto col_to = ColumnVector<ToType>::create();
Op::vector(sources->getData(), col_to->getData(), week_mode, time_zone);
block.getByPosition(result).column = std::move(col_to);
}
else
{
throw Exception(
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function "
+ Transform::name,
ErrorCodes::ILLEGAL_COLUMN);
}
}
};
}

View File

@ -1,6 +1,6 @@
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <Functions/CustomDateTransforms.h>
#include <Functions/CustomWeekTransforms.h>
#include <Functions/IFunction.h>
#include <Functions/extractTimeZoneFromFunctionArguments.h>
#include <IO/WriteHelpers.h>
@ -15,13 +15,13 @@ namespace ErrorCodes
}
/// See DateTimeTransforms.h
/// See CustomWeekTransforms.h
template <typename ToDataType, typename Transform>
class FunctionCustomDateToSomething : public IFunction
class FunctionCustomWeekToSomething : public IFunction
{
public:
static constexpr auto name = Transform::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionCustomDateToSomething>(); }
static FunctionPtr create(const Context &) { return std::make_shared<FunctionCustomWeekToSomething>(); }
String getName() const override { return name; }
@ -45,12 +45,12 @@ public:
"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);
if (!isString(arguments[1].type))
if (!isUInt8(arguments[1].type))
throw Exception(
"Function " + getName()
+ " supports 1 or 2 or 3 arguments. The 1st argument "
"must be of type Date or DateTime. The 2nd argument (optional) must be "
"a constant string with custom date(MM-DD). The 3nd argument (optional) must be "
"a constant UInt8 with week mode. The 3nd argument (optional) must be "
"a constant string with timezone name",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
@ -61,12 +61,12 @@ public:
"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);
if (!isString(arguments[1].type))
if (!isUInt8(arguments[1].type))
throw Exception(
"Function " + getName()
+ " supports 1 or 2 or 3 arguments. The 1st argument "
"must be of type Date or DateTime. The 2nd argument (optional) must be "
"a constant string with custom date(MM-DD). The 3nd argument (optional) must be "
"a constant UInt8 with week mode. The 3nd argument (optional) must be "
"a constant string with timezone name",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (!isString(arguments[2].type))
@ -74,7 +74,7 @@ public:
"Function " + getName()
+ " supports 1 or 2 or 3 arguments. The 1st argument "
"must be of type Date or DateTime. The 2nd argument (optional) must be "
"a constant string with custom date(MM-DD). The 3nd argument (optional) must be "
"a constant UInt8 with week mode. The 3nd argument (optional) must be "
"a constant string with timezone name",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (isDate(arguments[0].type) && std::is_same_v<ToDataType, DataTypeDate>)
@ -104,10 +104,10 @@ public:
WhichDataType which(from_type);
if (which.isDate())
CustomDateTransformImpl<DataTypeDate::FieldType, typename ToDataType::FieldType, Transform>::execute(
CustomWeekTransformImpl<DataTypeDate::FieldType, typename ToDataType::FieldType, Transform>::execute(
block, arguments, result, input_rows_count);
else if (which.isDateTime())
CustomDateTransformImpl<DataTypeDateTime::FieldType, typename ToDataType::FieldType, Transform>::execute(
CustomWeekTransformImpl<DataTypeDateTime::FieldType, typename ToDataType::FieldType, Transform>::execute(
block, arguments, result, input_rows_count);
else
throw Exception(
@ -139,15 +139,15 @@ public:
if (checkAndGetDataType<DataTypeDate>(&type))
{
return Transform::FactorTransform::execute(UInt16(left.get<UInt64>()), DEFAULT_CUSTOM_MONTH, DEFAULT_CUSTOM_DAY, date_lut)
== Transform::FactorTransform::execute(UInt16(right.get<UInt64>()), DEFAULT_CUSTOM_MONTH, DEFAULT_CUSTOM_DAY, date_lut)
return Transform::FactorTransform::execute(UInt16(left.get<UInt64>()), DEFAULT_WEEK_MODE, date_lut)
== Transform::FactorTransform::execute(UInt16(right.get<UInt64>()), DEFAULT_WEEK_MODE, date_lut)
? is_monotonic
: is_not_monotonic;
}
else
{
return Transform::FactorTransform::execute(UInt32(left.get<UInt64>()), DEFAULT_CUSTOM_MONTH, DEFAULT_CUSTOM_DAY, date_lut)
== Transform::FactorTransform::execute(UInt32(right.get<UInt64>()), DEFAULT_CUSTOM_MONTH, DEFAULT_CUSTOM_DAY, date_lut)
return Transform::FactorTransform::execute(UInt32(left.get<UInt64>()), DEFAULT_WEEK_MODE, date_lut)
== Transform::FactorTransform::execute(UInt32(right.get<UInt64>()), DEFAULT_WEEK_MODE, date_lut)
? is_monotonic
: is_not_monotonic;
}

View File

@ -17,7 +17,6 @@ void registerFunctionToMonday(FunctionFactory &);
void registerFunctionToISOWeek(FunctionFactory &);
void registerFunctionToISOYear(FunctionFactory &);
void registerFunctionToCustomWeek(FunctionFactory &);
void registerFunctionToCustomYear(FunctionFactory &);
void registerFunctionToStartOfMonth(FunctionFactory &);
void registerFunctionToStartOfQuarter(FunctionFactory &);
void registerFunctionToStartOfYear(FunctionFactory &);
@ -28,7 +27,6 @@ void registerFunctionToStartOfFifteenMinutes(FunctionFactory &);
void registerFunctionToStartOfHour(FunctionFactory &);
void registerFunctionToStartOfInterval(FunctionFactory &);
void registerFunctionToStartOfISOYear(FunctionFactory &);
void registerFunctionToStartOfCustomYear(FunctionFactory &);
void registerFunctionToRelativeYearNum(FunctionFactory &);
void registerFunctionToRelativeQuarterNum(FunctionFactory &);
void registerFunctionToRelativeMonthNum(FunctionFactory &);
@ -83,7 +81,6 @@ void registerFunctionsDateTime(FunctionFactory & factory)
registerFunctionToISOWeek(factory);
registerFunctionToISOYear(factory);
registerFunctionToCustomWeek(factory);
registerFunctionToCustomYear(factory);
registerFunctionToStartOfMonth(factory);
registerFunctionToStartOfQuarter(factory);
registerFunctionToStartOfYear(factory);
@ -94,7 +91,6 @@ void registerFunctionsDateTime(FunctionFactory & factory)
registerFunctionToStartOfHour(factory);
registerFunctionToStartOfInterval(factory);
registerFunctionToStartOfISOYear(factory);
registerFunctionToStartOfCustomYear(factory);
registerFunctionToRelativeYearNum(factory);
registerFunctionToRelativeQuarterNum(factory);
registerFunctionToRelativeMonthNum(factory);

View File

@ -1,17 +1,19 @@
#include <DataTypes/DataTypesNumber.h>
#include <Functions/CustomDateTransforms.h>
#include <Functions/FunctionCustomDateToSomething.h>
#include <Functions/CustomWeekTransforms.h>
#include <Functions/FunctionCustomWeekToSomething.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
namespace DB
{
using FunctionToCustomWeek = FunctionCustomDateToSomething<DataTypeUInt8, ToCustomWeekImpl>;
using FunctionWeek = FunctionCustomWeekToSomething<DataTypeUInt8, WeekImpl>;
using FunctionYearWeek = FunctionCustomWeekToSomething<DataTypeUInt32, YearWeekImpl>;
void registerFunctionToCustomWeek(FunctionFactory & factory)
{
factory.registerFunction<FunctionToCustomWeek>();
factory.registerFunction<FunctionWeek>();
factory.registerFunction<FunctionYearWeek>();
}
}

View File

@ -1,20 +0,0 @@
#include <Functions/IFunction.h>
#include <Functions/FunctionFactory.h>
#include <Functions/CustomDateTransforms.h>
#include <Functions/FunctionCustomDateToSomething.h>
#include <DataTypes/DataTypesNumber.h>
namespace DB
{
using FunctionToCustomYear = FunctionCustomDateToSomething<DataTypeUInt16, ToCustomYearImpl>;
void registerFunctionToCustomYear(FunctionFactory & factory)
{
factory.registerFunction<FunctionToCustomYear>();
}
}

View File

@ -1,20 +0,0 @@
#include <Functions/IFunction.h>
#include <Functions/FunctionFactory.h>
#include <Functions/CustomDateTransforms.h>
#include <Functions/FunctionCustomDateToSomething.h>
#include <DataTypes/DataTypesNumber.h>
namespace DB
{
using FunctionToStartOfCustomYear = FunctionCustomDateToSomething<DataTypeDate, ToStartOfCustomYearImpl>;
void registerFunctionToStartOfCustomYear(FunctionFactory & factory)
{
factory.registerFunction<FunctionToStartOfCustomYear>();
}
}

View File

@ -1,45 +1,63 @@
2016-12-22 52 2016 2015-12-28
2016-12-23 52 2016 2015-12-28
2016-12-24 52 2016 2015-12-28
2016-12-25 52 2016 2015-12-28
2016-12-26 1 2017 2016-12-26
2016-12-27 1 2017 2016-12-26
2016-12-28 1 2017 2016-12-26
2016-12-29 1 2017 2016-12-26
2016-12-30 1 2017 2016-12-26
2016-12-31 1 2017 2016-12-26
2017-01-01 1 2017 2016-12-26
2017-01-02 2 2017 2016-12-26
2017-01-03 2 2017 2016-12-26
2017-01-04 2 2017 2016-12-26
2017-01-05 2 2017 2016-12-26
2017-01-20 00:00:00 52 2016 2016-01-25
2017-01-21 00:00:00 52 2016 2016-01-25
2017-01-22 00:00:00 52 2016 2016-01-25
2017-01-23 00:00:00 1 2017 2017-01-23
2017-01-24 00:00:00 1 2017 2017-01-23
2017-01-25 00:00:00 1 2017 2017-01-23
2017-01-26 00:00:00 1 2017 2017-01-23
2017-01-27 00:00:00 1 2017 2017-01-23
2017-01-28 00:00:00 1 2017 2017-01-23
2017-01-29 00:00:00 1 2017 2017-01-23
2017-01-30 00:00:00 2 2017 2017-01-23
2017-01-31 00:00:00 2 2017 2017-01-23
2017-02-01 00:00:00 2 2017 2017-01-23
2017-02-02 00:00:00 2 2017 2017-01-23
2017-02-03 00:00:00 2 2017 2017-01-23
2017-01-20 00:00:00 52 2016 2016-01-25
2017-01-21 00:00:00 52 2016 2016-01-25
2017-01-22 00:00:00 52 2016 2016-01-25
2017-01-23 00:00:00 1 2017 2017-01-23
2017-01-24 00:00:00 1 2017 2017-01-23
2017-01-25 00:00:00 1 2017 2017-01-23
2017-01-26 00:00:00 1 2017 2017-01-23
2017-01-27 00:00:00 1 2017 2017-01-23
2017-01-28 00:00:00 1 2017 2017-01-23
2017-01-29 00:00:00 1 2017 2017-01-23
2017-01-30 00:00:00 2 2017 2017-01-23
2017-01-31 00:00:00 2 2017 2017-01-23
2017-02-01 00:00:00 2 2017 2017-01-23
2017-02-02 00:00:00 2 2017 2017-01-23
2017-02-03 00:00:00 2 2017 2017-01-23
0 0 1 1
52 52 53 53
1 0
198153 198153 198252 198252
198701 198652
0 0 0 0 0 0 1
1 0 1 1 1 1 1
0 1 1 1 1 0 0
1 1 1 2 2 1 1
199952 200053 200152 200252 200352 200452 200601
200001 200053 200201 200301 200401 200501 200601
199952 200101 200201 200301 200401 200453 200552
200001 200101 200201 200302 200402 200501 200601
52 53 52 52
53 52
52 53 52 53 52 52 52 52
0 0 52 52 0 0 52 52
1 1 1 1 1 1 1 1
53 52 53 52 53 52 1 52
0 1 53 1 1 1 1 1
200053 200052 200053 200052 200101 200052 200101 200052
2016-12-21 52 52 201652 201652
2016-12-22 52 52 201652 201652
2016-12-23 52 52 201652 201652
2016-12-24 52 52 201652 201652
2016-12-25 53 52 201653 201652
2016-12-26 53 1 201653 201701
2016-12-27 53 1 201653 201701
2016-12-28 53 1 201653 201701
2016-12-29 53 1 201653 201701
2016-12-30 53 1 201653 201701
2016-12-31 53 1 201653 201701
2017-01-01 1 1 201701 201701
2017-01-02 1 2 201701 201702
2017-01-03 1 2 201701 201702
2017-01-04 1 2 201701 201702
2017-01-05 1 2 201701 201702
2017-01-06 1 2 201701 201702
2017-01-07 1 2 201701 201702
2017-01-08 2 2 201702 201702
2017-01-09 2 3 201702 201703
2017-01-10 2 3 201702 201703
2016-12-22 00:00:00 52 52 201652 201652
2016-12-23 00:00:00 52 52 201652 201652
2016-12-24 00:00:00 52 52 201652 201652
2016-12-25 00:00:00 53 52 201653 201652
2016-12-26 00:00:00 53 1 201653 201701
2016-12-27 00:00:00 53 1 201653 201701
2016-12-28 00:00:00 53 1 201653 201701
2016-12-29 00:00:00 53 1 201653 201701
2016-12-30 00:00:00 53 1 201653 201701
2016-12-31 00:00:00 53 1 201653 201701
2017-01-01 00:00:00 1 1 201701 201701
2017-01-02 00:00:00 1 2 201701 201702
2017-01-03 00:00:00 1 2 201701 201702
2017-01-04 00:00:00 1 2 201701 201702
2017-01-05 00:00:00 1 2 201701 201702
2017-01-06 00:00:00 1 2 201701 201702
2017-01-07 00:00:00 1 2 201701 201702
2017-01-08 00:00:00 2 2 201702 201702
2017-01-09 00:00:00 2 3 201702 201703
2017-01-10 00:00:00 2 3 201702 201703
2017-01-11 00:00:00 2 3 201702 201703

View File

@ -1,6 +1,42 @@
-- By default, the week contains January 1 is the first week:
SELECT toDate('2016-12-22') + number AS d, toCustomWeek(d) as week, toCustomYear(d) AS year, toStartOfCustomYear(d) AS startday FROM numbers(15);
-- The week contains January 28 is the first week:
SELECT toDateTime(toDate('2017-01-20') + number, 'Europe/Moscow' ) AS d, toCustomWeek(d,'01-28') as week, toCustomYear(d,'01-28') AS year, toStartOfCustomYear(d,'01-28') AS startday FROM numbers(15);
-- and with timezone
SELECT toDateTime(toDate('2017-01-20') + number, 'Europe/Moscow' ) AS d, toCustomWeek(d,'01-28','Europe/Moscow') as week, toCustomYear(d,'01-28','Europe/Moscow') AS year, toStartOfCustomYear(d,'01-28','Europe/Moscow') AS startday FROM numbers(15);
-- week mode [0,7], week test case. refer to the mysql test case
SELECT week(toDate('1998-01-01')), week(toDate('1997-01-01')), week(toDate('1998-01-01'), 1), week(toDate('1997-01-01'), 1);
SELECT week(toDate('1998-12-31')), week(toDate('1997-12-31')), week(toDate('1998-12-31'), 1), week(toDate('1997-12-31'), 1);
SELECT week(toDate('1995-01-01')), week(toDate('1995-01-01'), 1);
SELECT yearWeek(toDate('1981-12-31'), 1), yearWeek(toDate('1982-01-01'), 1), yearWeek(toDate('1982-12-31'), 1), yearWeek(toDate('1983-01-01'), 1);
SELECT yearWeek(toDate('1987-01-01'), 1), yearWeek(toDate('1987-01-01'));
SELECT week(toDate('2000-01-01'),0) AS w2000, week(toDate('2001-01-01'),0) AS w2001, week(toDate('2002-01-01'),0) AS w2002,week(toDate('2003-01-01'),0) AS w2003, week(toDate('2004-01-01'),0) AS w2004, week(toDate('2005-01-01'),0) AS w2005, week(toDate('2006-01-01'),0) AS w2006;
SELECT week(toDate('2000-01-06'),0) AS w2000, week(toDate('2001-01-06'),0) AS w2001, week(toDate('2002-01-06'),0) AS w2002,week(toDate('2003-01-06'),0) AS w2003, week(toDate('2004-01-06'),0) AS w2004, week(toDate('2005-01-06'),0) AS w2005, week(toDate('2006-01-06'),0) AS w2006;
SELECT week(toDate('2000-01-01'),1) AS w2000, week(toDate('2001-01-01'),1) AS w2001, week(toDate('2002-01-01'),1) AS w2002,week(toDate('2003-01-01'),1) AS w2003, week(toDate('2004-01-01'),1) AS w2004, week(toDate('2005-01-01'),1) AS w2005, week(toDate('2006-01-01'),1) AS w2006;
SELECT week(toDate('2000-01-06'),1) AS w2000, week(toDate('2001-01-06'),1) AS w2001, week(toDate('2002-01-06'),1) AS w2002,week(toDate('2003-01-06'),1) AS w2003, week(toDate('2004-01-06'),1) AS w2004, week(toDate('2005-01-06'),1) AS w2005, week(toDate('2006-01-06'),1) AS w2006;
SELECT yearWeek(toDate('2000-01-01'),0) AS w2000, yearWeek(toDate('2001-01-01'),0) AS w2001, yearWeek(toDate('2002-01-01'),0) AS w2002,yearWeek(toDate('2003-01-01'),0) AS w2003, yearWeek(toDate('2004-01-01'),0) AS w2004, yearWeek(toDate('2005-01-01'),0) AS w2005, yearWeek(toDate('2006-01-01'),0) AS w2006;
SELECT yearWeek(toDate('2000-01-06'),0) AS w2000, yearWeek(toDate('2001-01-06'),0) AS w2001, yearWeek(toDate('2002-01-06'),0) AS w2002,yearWeek(toDate('2003-01-06'),0) AS w2003, yearWeek(toDate('2004-01-06'),0) AS w2004, yearWeek(toDate('2005-01-06'),0) AS w2005, yearWeek(toDate('2006-01-06'),0) AS w2006;
SELECT yearWeek(toDate('2000-01-01'),1) AS w2000, yearWeek(toDate('2001-01-01'),1) AS w2001, yearWeek(toDate('2002-01-01'),1) AS w2002,yearWeek(toDate('2003-01-01'),1) AS w2003, yearWeek(toDate('2004-01-01'),1) AS w2004, yearWeek(toDate('2005-01-01'),1) AS w2005, yearWeek(toDate('2006-01-01'),1) AS w2006;
SELECT yearWeek(toDate('2000-01-06'),1) AS w2000, yearWeek(toDate('2001-01-06'),1) AS w2001, yearWeek(toDate('2002-01-06'),1) AS w2002,yearWeek(toDate('2003-01-06'),1) AS w2003, yearWeek(toDate('2004-01-06'),1) AS w2004, yearWeek(toDate('2005-01-06'),1) AS w2005, yearWeek(toDate('2006-01-06'),1) AS w2006;
SELECT week(toDate('1998-12-31'),2),week(toDate('1998-12-31'),3), week(toDate('2000-01-01'),2), week(toDate('2000-01-01'),3);
SELECT week(toDate('2000-12-31'),2),week(toDate('2000-12-31'),3);
SELECT week(toDate('1998-12-31'),0) AS w0, week(toDate('1998-12-31'),1) AS w1, week(toDate('1998-12-31'),2) AS w2, week(toDate('1998-12-31'),3) AS w3, week(toDate('1998-12-31'),4) AS w4, week(toDate('1998-12-31'),5) AS w5, week(toDate('1998-12-31'),6) AS w6, week(toDate('1998-12-31'),7) AS w7;
SELECT week(toDate('2000-01-01'),0) AS w0, week(toDate('2000-01-01'),1) AS w1, week(toDate('2000-01-01'),2) AS w2, week(toDate('2000-01-01'),3) AS w3, week(toDate('2000-01-01'),4) AS w4, week(toDate('2000-01-01'),5) AS w5, week(toDate('2000-01-01'),6) AS w6, week(toDate('2000-01-01'),7) AS w7;
SELECT week(toDate('2000-01-06'),0) AS w0, week(toDate('2000-01-06'),1) AS w1, week(toDate('2000-01-06'),2) AS w2, week(toDate('2000-01-06'),3) AS w3, week(toDate('2000-01-06'),4) AS w4, week(toDate('2000-01-06'),5) AS w5, week(toDate('2000-01-06'),6) AS w6, week(toDate('2000-01-06'),7) AS w7;
SELECT week(toDate('2000-12-31'),0) AS w0, week(toDate('2000-12-31'),1) AS w1, week(toDate('2000-12-31'),2) AS w2, week(toDate('2000-12-31'),3) AS w3, week(toDate('2000-12-31'),4) AS w4, week(toDate('2000-12-31'),5) AS w5, week(toDate('2000-12-31'),6) AS w6, week(toDate('2000-12-31'),7) AS w7;
SELECT week(toDate('2001-01-01'),0) AS w0, week(toDate('2001-01-01'),1) AS w1, week(toDate('2001-01-01'),2) AS w2, week(toDate('2001-01-01'),3) AS w3, week(toDate('2001-01-01'),4) AS w4, week(toDate('2001-01-01'),5) AS w5, week(toDate('2001-01-01'),6) AS w6, week(toDate('2001-01-01'),7) AS w7;
SELECT yearWeek(toDate('2000-12-31'),0), yearWeek(toDate('2000-12-31'),1), yearWeek(toDate('2000-12-31'),2), yearWeek(toDate('2000-12-31'),3), yearWeek(toDate('2000-12-31'),4), yearWeek(toDate('2000-12-31'),5), yearWeek(toDate('2000-12-31'),6), yearWeek(toDate('2000-12-31'),7);
-- week mode 8,9
SELECT
toDate('2016-12-21') + number AS d,
week(d, 8) AS week8,
week(d, 9) AS week9,
yearWeek(d, 8) AS yearWeek8,
yearWeek(d, 9) AS yearWeek9
FROM numbers(21);
SELECT toDateTime(toDate('2016-12-22') + number, 'Europe/Moscow' ) AS d,
week(d, 8, 'Europe/Moscow') AS week8,
week(d, 9, 'Europe/Moscow') AS week9,
yearWeek(d, 8, 'Europe/Moscow') AS yearWeek8,
yearWeek(d, 9, 'Europe/Moscow') AS yearWeek9
FROM numbers(21);

View File

@ -76,14 +76,6 @@ Returns the date.
Rounds down a date or date with time to the first day of ISO year.
Returns the date.
## toStartOfCustomYear
Rounds down a date or date with time to the first day of Custom year.
Returns the date.
```
toStartOfCustomYear(DateTime1, [, CustomDate][, Timezone])
```
## toStartOfQuarter
Rounds down a date or date with time to the first day of the quarter.
@ -184,46 +176,70 @@ Converts a date or date with time to a UInt16 number containing the Custom Year
toCustomYear(DateTime, [, CustomDate][, Timezone])
```
## toCustomWeek
## week(date[,mode])
This function returns the week number for date or datetime. The two-argument form of week() enables you to specify whether the week starts on Sunday or Monday and whether the return value should be in the range from 0 to 53 or from 1 to 53. If the mode argument is omitted, the default mode is 0.
The following table describes how the mode argument works.
| Mode | First day of week | Range | Week 1 is the first week … |
| ----------- | -------- | -------- | ------------------ |
|0|Sunday|0-53|with a Sunday in this year
|1|Monday|0-53|with 4 or more days this year
|2|Sunday|1-53|with a Sunday in this year
|3|Monday|1-53|with 4 or more days this year
|4|Sunday|0-53|with 4 or more days this year
|5|Monday|0-53|with a Monday in this year
|6|Sunday|1-53|with 4 or more days this year
|7|Monday|1-53|with a Monday in this year
|8|Sunday|1-53|contains January 1
|9|Monday|1-53|contains January 1
For mode values with a meaning of “with 4 or more days this year,” weeks are numbered according to ISO 8601:1988:
- If the week containing January 1 has 4 or more days in the new year, it is week 1.
- Otherwise, it is the last week of the previous year, and the next week is week 1.
For mode values with a meaning of “contains January 1”, the week contains January 1 is week 1. It doesn't matter how many days in the new year the week contained, even if it contained only one day.
Converts a date or date with time to a UInt8 number containing the Custom Week number.
The week number 1 is the first week in year that contains some day, which default to January 1, it can be modify by parameter of function.
Week begins at monday.
```
toCustomWeek(DateTime1, [, CustomDate][, Timezone])
week(DateTime1, [, mode][, Timezone])
```
**Parameters**
- `DateTime1` Date or DateTime.
- `CustomDate` Optional parameter, default is January 1, format: MM-DD.
- `mode` Optional parameter, Range of values is [0,9], default is 0.
- `Timezone` Optional parameter, it behaves like any other conversion function.
**Example**
By default, the week contains January 1 is the first week:
``` sql
SELECT toDate('2016-12-27') AS date, toCustomYear(date) AS year, toCustomWeek(date) AS week, toStartOfCustomYear(date) AS startday;
SELECT toDate('2016-12-27') AS date, week(date) AS week0, week(date,1) AS week1, week(date,9) AS week9;
```
```
┌───────date─┬─year─┬─week─┬───startday─┐
│ 2016-12-27 │ 2017 │ 1 │ 2016-12-26 │
└───────────┴─────┴──────┴─────────┘
┌───────date─┬─week0─┬─week1─┬─week9─┐
│ 2016-12-27 │ 52 │ 52 │ 1
└────────────┴───────┴───────┴───────┘
```
The week contains January 28 is the first week:
## yearWeek(date[,mode])
Returns year and week for a date. The year in the result may be different from the year in the date argument for the first and the last week of the year.
The mode argument works exactly like the mode argument to week(). For the single-argument syntax, a mode value of 0 is used.
**Example**
``` sql
SELECT toDate('2016-12-27') AS date, toCustomYear(date, '01-28') AS year, toCustomWeek(date,'01-28') AS week, toStartOfCustomYear(date,'01-28') AS startday;
SELECT toDate('2016-12-27') AS date, yearWeek(date) AS yearWeek0, yearWeek(date,1) AS yearWeek1, yearWeek(date,9) AS yearWeek9;
```
```
┌───────date─┬─year─┬─week─┬───startday─┐
│ 2016-12-27 │ 2016 │ 49 │ 2016-01-25
└───────────┴─────┴──────┴─────────┘
┌───────date─┬─yearWeek0─┬─yearWeek1─┬─yearWeek9─┐
│ 2016-12-27 │ 201652 │ 201652 │ 201701
└──────────────────────┴──────────────────────┘
```
## now
Accepts zero arguments and returns the current time at one of the moments of request execution.

View File

@ -1,10 +1,10 @@
#pragma once
#include <common/Types.h>
#include <common/DayNum.h>
#include <common/likely.h>
#include <ctime>
#include <string>
#include <common/DayNum.h>
#include <common/Types.h>
#include <common/likely.h>
#define DATE_LUT_MAX (0xFFFFFFFFU - 86400)
#define DATE_LUT_MAX_DAY_NUM (0xFFFFFFFFU / 86400)
@ -14,10 +14,17 @@
#define DATE_LUT_MAX_YEAR 2105 /// Last supported year
#define DATE_LUT_YEARS (1 + DATE_LUT_MAX_YEAR - DATE_LUT_MIN_YEAR) /// Number of years in lookup table
/// Flags for calc_week() function.
#define WEEK_MONDAY_FIRST 1
#define WEEK_YEAR 2
#define WEEK_FIRST_WEEKDAY 4
#define WEEK_NEWYEAR_DAY 8
#if defined(__PPC__)
#if !__clang__
#pragma GCC diagnostic ignored "-Wmaybe-uninitialized"
#endif
# if !__clang__
# pragma GCC diagnostic ignored "-Wmaybe-uninitialized"
# endif
#endif
/** Lookup table to conversion of time to date, and to month / year / day of week / day of month and so on.
@ -93,10 +100,7 @@ private:
return DayNum(guess - 1);
}
inline const Values & find(time_t t) const
{
return lut[findIndex(t)];
}
inline const Values & find(time_t t) const { return lut[findIndex(t)]; }
public:
const std::string & getTimeZone() const { return time_zone; }
@ -117,15 +121,9 @@ public:
return lut[DayNum(index - (lut[index].day_of_week - 1))].date;
}
inline DayNum toFirstDayNumOfWeek(DayNum d) const
{
return DayNum(d - (lut[d].day_of_week - 1));
}
inline DayNum toFirstDayNumOfWeek(DayNum d) const { return DayNum(d - (lut[d].day_of_week - 1)); }
inline DayNum toFirstDayNumOfWeek(time_t t) const
{
return toFirstDayNumOfWeek(toDayNum(t));
}
inline DayNum toFirstDayNumOfWeek(time_t t) const { return toFirstDayNumOfWeek(toDayNum(t)); }
/// Round down to start of month.
inline time_t toFirstDayOfMonth(time_t t) const
@ -134,15 +132,9 @@ public:
return lut[index - (lut[index].day_of_month - 1)].date;
}
inline DayNum toFirstDayNumOfMonth(DayNum d) const
{
return DayNum(d - (lut[d].day_of_month - 1));
}
inline DayNum toFirstDayNumOfMonth(DayNum d) const { return DayNum(d - (lut[d].day_of_month - 1)); }
inline DayNum toFirstDayNumOfMonth(time_t t) const
{
return toFirstDayNumOfMonth(toDayNum(t));
}
inline DayNum toFirstDayNumOfMonth(time_t t) const { return toFirstDayNumOfMonth(toDayNum(t)); }
/// Round down to start of quarter.
inline DayNum toFirstDayNumOfQuarter(DayNum d) const
@ -160,31 +152,16 @@ public:
return DayNum(index + 1);
}
inline DayNum toFirstDayNumOfQuarter(time_t t) const
{
return toFirstDayNumOfQuarter(toDayNum(t));
}
inline DayNum toFirstDayNumOfQuarter(time_t t) const { return toFirstDayNumOfQuarter(toDayNum(t)); }
inline time_t toFirstDayOfQuarter(time_t t) const
{
return fromDayNum(toFirstDayNumOfQuarter(t));
}
inline time_t toFirstDayOfQuarter(time_t t) const { return fromDayNum(toFirstDayNumOfQuarter(t)); }
/// Round down to start of year.
inline time_t toFirstDayOfYear(time_t t) const
{
return lut[years_lut[lut[findIndex(t)].year - DATE_LUT_MIN_YEAR]].date;
}
inline time_t toFirstDayOfYear(time_t t) const { return lut[years_lut[lut[findIndex(t)].year - DATE_LUT_MIN_YEAR]].date; }
inline DayNum toFirstDayNumOfYear(DayNum d) const
{
return years_lut[lut[d].year - DATE_LUT_MIN_YEAR];
}
inline DayNum toFirstDayNumOfYear(DayNum d) const { return years_lut[lut[d].year - DATE_LUT_MIN_YEAR]; }
inline DayNum toFirstDayNumOfYear(time_t t) const
{
return toFirstDayNumOfYear(toDayNum(t));
}
inline DayNum toFirstDayNumOfYear(time_t t) const { return toFirstDayNumOfYear(toDayNum(t)); }
inline time_t toFirstDayOfNextMonth(time_t t) const
{
@ -200,15 +177,9 @@ public:
return lut[index - (lut[index].day_of_month - 1)].date;
}
inline UInt8 daysInMonth(DayNum d) const
{
return lut[d].days_in_month;
}
inline UInt8 daysInMonth(DayNum d) const { return lut[d].days_in_month; }
inline UInt8 daysInMonth(time_t t) const
{
return find(t).days_in_month;
}
inline UInt8 daysInMonth(time_t t) const { return find(t).days_in_month; }
inline UInt8 daysInMonth(UInt16 year, UInt8 month) const
{
@ -219,10 +190,7 @@ public:
/** Round to start of day, then shift for specified amount of days.
*/
inline time_t toDateAndShift(time_t t, Int32 days) const
{
return lut[DayNum(findIndex(t) + days)].date;
}
inline time_t toDateAndShift(time_t t, Int32 days) const { return lut[DayNum(findIndex(t) + days)].date; }
inline time_t toTime(time_t t) const
{
@ -324,10 +292,7 @@ public:
return (d + 8 - toDayOfWeek(d)) / 7;
}
inline unsigned toRelativeWeekNum(time_t t) const
{
return toRelativeWeekNum(toDayNum(t));
}
inline unsigned toRelativeWeekNum(time_t t) const { return toRelativeWeekNum(toDayNum(t)); }
/// Get year that contains most of the current week. Week begins at monday.
inline unsigned toISOYear(DayNum d) const
@ -336,10 +301,7 @@ public:
return toYear(DayNum(d + 4 - toDayOfWeek(d)));
}
inline unsigned toISOYear(time_t t) const
{
return toISOYear(toDayNum(t));
}
inline unsigned toISOYear(time_t t) const { return toISOYear(toDayNum(t)); }
/// ISO year begins with a monday of the week that is contained more than by half in the corresponding calendar year.
/// Example: ISO year 2019 begins at 2018-12-31. And ISO year 2017 begins at 2017-01-02.
@ -351,100 +313,204 @@ public:
DayNum first_day_of_year = years_lut[iso_year - DATE_LUT_MIN_YEAR];
auto first_day_of_week_of_year = lut[first_day_of_year].day_of_week;
return DayNum(first_day_of_week_of_year <= 4
? first_day_of_year + 1 - first_day_of_week_of_year
: first_day_of_year + 8 - first_day_of_week_of_year);
return DayNum(
first_day_of_week_of_year <= 4 ? first_day_of_year + 1 - first_day_of_week_of_year
: first_day_of_year + 8 - first_day_of_week_of_year);
}
inline DayNum toFirstDayNumOfISOYear(time_t t) const
{
return toFirstDayNumOfISOYear(toDayNum(t));
}
inline DayNum toFirstDayNumOfISOYear(time_t t) const { return toFirstDayNumOfISOYear(toDayNum(t)); }
inline time_t toFirstDayOfISOYear(time_t t) const
{
return fromDayNum(toFirstDayNumOfISOYear(t));
}
inline time_t toFirstDayOfISOYear(time_t t) const { return fromDayNum(toFirstDayNumOfISOYear(t)); }
/// ISO 8601 week number. Week begins at monday.
/// The week number 1 is the first week in year that contains 4 or more days (that's more than half).
inline unsigned toISOWeek(DayNum d) const
inline unsigned toISOWeek(DayNum d) const { return 1 + DayNum(toFirstDayNumOfWeek(d) - toFirstDayNumOfISOYear(d)) / 7; }
inline unsigned toISOWeek(time_t t) const { return toISOWeek(toDayNum(t)); }
/*
The bits in week_mode has the following meaning:
WEEK_MONDAY_FIRST (0) If not set Sunday is first day of week
If set Monday is first day of week
WEEK_YEAR (1) If not set Week is in range 0-53
Week 0 is returned for the the last week of the previous year (for
a date at start of january) In this case one can get 53 for the
first week of next year. This flag ensures that the week is
relevant for the given year. Note that this flag is only
releveant if WEEK_JANUARY is not set.
If set Week is in range 1-53.
In this case one may get week 53 for a date in January (when
the week is that last week of previous year) and week 1 for a
date in December.
WEEK_FIRST_WEEKDAY (2) If not set Weeks are numbered according
to ISO 8601:1988
If set The week that contains the first
'first-day-of-week' is week 1.
WEEK_NEWYEAR_DAY (3)
If set The week that contains the January 1 is week 1.
Week is in range 1-53.
And ignore WEEK_YEAR, WEEK_FIRST_WEEKDAY
ISO 8601:1988 means that if the week containing January 1 has
four or more days in the new year, then it is week 1;
Otherwise it is the last week of the previous year, and the
next week is week 1.
*/
inline unsigned calc_week(DayNum d, UInt32 week_mode, UInt32 * year) const
{
return 1 + DayNum(toFirstDayNumOfWeek(d) - toFirstDayNumOfISOYear(d)) / 7;
bool newyear_day_mode = week_mode & WEEK_NEWYEAR_DAY;
week_mode = check_week_mode(week_mode);
bool monday_first_mode = week_mode & WEEK_MONDAY_FIRST;
bool week_year_mode = week_mode & WEEK_YEAR;
bool first_weekday_mode = week_mode & WEEK_FIRST_WEEKDAY;
// Calculate week number of WEEK_NEWYEAR_DAY mode
if (newyear_day_mode)
{
return calc_newyear_week(d, monday_first_mode, year);
}
UInt32 days = 0;
UInt64 daynr = calc_daynr(toYear(d), toMonth(d), toDayOfMonth(d));
UInt64 first_daynr = calc_daynr(toYear(d), 1, 1);
// 0 for monday, 1 for tuesday ...
// get weekday from first day in year.
UInt32 weekday = calc_weekday(first_daynr, !monday_first_mode);
*year = toYear(d);
if (toMonth(d) == 1 && toDayOfMonth(d) <= 7 - weekday)
{
if (!week_year_mode && ((first_weekday_mode && weekday != 0) || (!first_weekday_mode && weekday >= 4)))
return 0;
week_year_mode = 1;
(*year)--;
first_daynr -= (days = calc_days_in_year(*year));
weekday = (weekday + 53 * 7 - days) % 7;
}
if ((first_weekday_mode && weekday != 0) || (!first_weekday_mode && weekday >= 4))
days = daynr - (first_daynr + (7 - weekday));
else
days = daynr - (first_daynr - weekday);
if (week_year_mode && days >= 52 * 7)
{
weekday = (weekday + calc_days_in_year(*year)) % 7;
if ((!first_weekday_mode && weekday < 4) || (first_weekday_mode && weekday == 0))
{
(*year)++;
return 1;
}
}
return days / 7 + 1;
}
inline unsigned toISOWeek(time_t t) const
inline unsigned calc_yearWeek(DayNum d, UInt32 week_mode) const
{
return toISOWeek(toDayNum(t));
UInt32 year = 0;
UInt8 week = calc_week(d, week_mode | WEEK_YEAR, &year);
return week + year * 100;
}
/// Get year that contains the custom week.
inline unsigned toCustomYear(DayNum d, UInt8 custom_month, UInt8 custom_day) const
{
// Checking the week across the year
auto year = toYear(DayNum(d + 7 - toDayOfWeek(d)));
auto day_of_year = makeDayNum(year, custom_month, custom_day);
// Checking greater than or equal to Monday, If true take current year, else take last year.
return toFirstDayNumOfWeek(day_of_year) <= d ? year : year - 1;
}
inline unsigned toCustomYear(time_t t, UInt8 custom_month, UInt8 custom_day) const
{
return toCustomYear(toDayNum(t),custom_month,custom_day);
}
/// Custom year begins with a monday of the week that is contained January 1,
/// which day can be modified through custom_month and custom_day of parameters.
/// Example: Custom year 2019 begins at 2018-12-31. And Custom year 2017 begins at 2016-12-26.
inline DayNum toFirstDayNumOfCustomYear(DayNum d, UInt8 custom_month, UInt8 custom_day) const
{
auto custom_year = toCustomYear(d, custom_month, custom_day);
return toFirstDayNumOfWeek(makeDayNum(custom_year, custom_month, custom_day));
}
inline DayNum toFirstDayNumOfCustomYear(time_t t, UInt8 custom_month, UInt8 custom_day) const
{
return toFirstDayNumOfCustomYear(toDayNum(t), custom_month, custom_day);
}
inline time_t toFirstDayOfCustomYear(time_t t, UInt8 custom_month, UInt8 custom_day) const
{
return fromDayNum(toFirstDayNumOfCustomYear(t, custom_month, custom_day));
}
/// Custom week number. Week begins at monday.
/// Calculate week number of WEEK_NEWYEAR_DAY mode
/// The week number 1 is the first week in year that contains January 1,
/// which day can be modified through custom_month and custom_day of parameters.
inline unsigned toCustomWeek(DayNum d, UInt8 custom_month, UInt8 custom_day) const
inline unsigned calc_newyear_week(DayNum d, bool monday_first_mode, UInt32 * year) const
{
return 1 + (toFirstDayNumOfWeek(d) - toFirstDayNumOfCustomYear(d, custom_month, custom_day)) / 7;
UInt16 offset_day = monday_first_mode ? 0U : 1U;
// Checking the week across the year
*year = toYear(DayNum(d + 7 - toDayOfWeek(DayNum(d + offset_day))));
DayNum first_day = makeDayNum(*year, 1, 1);
DayNum this_day = d;
if (monday_first_mode)
{
// Rounds down a date to the nearest Monday.
first_day = toFirstDayNumOfWeek(first_day);
this_day = toFirstDayNumOfWeek(d);
}
else
{
// Rounds down a date to the nearest Sunday.
if (toDayOfWeek(first_day) != 7)
first_day = DayNum(first_day - toDayOfWeek(first_day));
if (toDayOfWeek(d) != 7)
this_day = DayNum(d - toDayOfWeek(d));
}
return (this_day - first_day) / 7 + 1;
}
inline unsigned toCustomWeek(time_t t, UInt8 custom_month, UInt8 custom_day) const
inline unsigned check_week_mode(UInt32 mode) const
{
return toCustomWeek(toDayNum(t), custom_month, custom_day);
UInt32 week_format = (mode & 7);
if (!(week_format & WEEK_MONDAY_FIRST))
week_format ^= WEEK_FIRST_WEEKDAY;
return week_format;
}
/*
Calculate nr of day since year 0 in new date-system (from 1615)
SYNOPSIS
calc_daynr()
year Year (exact 4 digit year, no year conversions)
month Month
day Day
NOTES: 0000-00-00 is a valid date, and will return 0
RETURN
Days since 0000-00-00
*/
inline UInt64 calc_daynr(UInt32 year, UInt32 month, UInt32 day) const
{
UInt64 delsum;
int temp;
int y = year; /* may be < 0 temporarily */
if (y == 0 && month == 0)
return 0; /* Skip errors */
/* Cast to int to be able to handle month == 0 */
delsum = static_cast<UInt64>(365 * y + 31 * (month - 1) + day);
delsum = static_cast<UInt64>(365 * y + 31 * (static_cast<int>(month) - 1) + static_cast<int>(day));
if (month <= 2)
y--;
else
delsum -= static_cast<UInt64>(static_cast<int>(month) * 4 + 23) / 10;
temp = ((y / 100 + 1) * 3) / 4;
return delsum + y / 4 - temp;
} /* calc_daynr */
/*
Calc weekday from daynr
Returns 0 for monday, 1 for tuesday ...
*/
inline unsigned calc_weekday(UInt64 daynr, bool sunday_first_day_of_week) const
{
return (static_cast<UInt32>((daynr + 5L + (sunday_first_day_of_week ? 1L : 0L)) % 7));
}
/* Calc days in one year. */
inline unsigned calc_days_in_year(UInt32 year) const
{
return ((year & 3) == 0 && (year % 100 || (year % 400 == 0 && year)) ? 366 : 365);
}
/// Number of month from some fixed moment in the past (year * 12 + month)
inline unsigned toRelativeMonthNum(DayNum d) const
{
return lut[d].year * 12 + lut[d].month;
}
inline unsigned toRelativeMonthNum(DayNum d) const { return lut[d].year * 12 + lut[d].month; }
inline unsigned toRelativeMonthNum(time_t t) const
{
return toRelativeMonthNum(toDayNum(t));
}
inline unsigned toRelativeMonthNum(time_t t) const { return toRelativeMonthNum(toDayNum(t)); }
inline unsigned toRelativeQuarterNum(DayNum d) const
{
return lut[d].year * 4 + (lut[d].month - 1) / 3;
}
inline unsigned toRelativeQuarterNum(DayNum d) const { return lut[d].year * 4 + (lut[d].month - 1) / 3; }
inline unsigned toRelativeQuarterNum(time_t t) const
{
return toRelativeQuarterNum(toDayNum(t));
}
inline unsigned toRelativeQuarterNum(time_t t) const { return toRelativeQuarterNum(toDayNum(t)); }
/// We count all hour-length intervals, unrelated to offset changes.
inline time_t toRelativeHourNum(time_t t) const
@ -457,20 +523,11 @@ public:
return (t + 86400 - offset_at_start_of_epoch) / 3600;
}
inline time_t toRelativeHourNum(DayNum d) const
{
return toRelativeHourNum(lut[d].date);
}
inline time_t toRelativeHourNum(DayNum d) const { return toRelativeHourNum(lut[d].date); }
inline time_t toRelativeMinuteNum(time_t t) const
{
return t / 60;
}
inline time_t toRelativeMinuteNum(time_t t) const { return t / 60; }
inline time_t toRelativeMinuteNum(DayNum d) const
{
return toRelativeMinuteNum(lut[d].date);
}
inline time_t toRelativeMinuteNum(DayNum d) const { return toRelativeMinuteNum(lut[d].date); }
inline DayNum toStartOfYearInterval(DayNum d, UInt64 years) const
{
@ -540,16 +597,14 @@ public:
/// Create DayNum from year, month, day of month.
inline DayNum makeDayNum(UInt16 year, UInt8 month, UInt8 day_of_month) const
{
if (unlikely(year < DATE_LUT_MIN_YEAR || year > DATE_LUT_MAX_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31))
if (unlikely(
year < DATE_LUT_MIN_YEAR || year > DATE_LUT_MAX_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31))
return DayNum(0);
return DayNum(years_months_lut[(year - DATE_LUT_MIN_YEAR) * 12 + month - 1] + day_of_month - 1);
}
inline time_t makeDate(UInt16 year, UInt8 month, UInt8 day_of_month) const
{
return lut[makeDayNum(year, month, day_of_month)].date;
}
inline time_t makeDate(UInt16 year, UInt8 month, UInt8 day_of_month) const { return lut[makeDayNum(year, month, day_of_month)].date; }
/** Does not accept daylight saving time as argument: in case of ambiguity, it choose greater timestamp.
*/
@ -591,38 +646,21 @@ public:
return values.year * 10000 + values.month * 100 + values.day_of_month;
}
inline time_t YYYYMMDDToDate(UInt32 num) const
{
return makeDate(num / 10000, num / 100 % 100, num % 100);
}
inline time_t YYYYMMDDToDate(UInt32 num) const { return makeDate(num / 10000, num / 100 % 100, num % 100); }
inline DayNum YYYYMMDDToDayNum(UInt32 num) const
{
return makeDayNum(num / 10000, num / 100 % 100, num % 100);
}
inline DayNum YYYYMMDDToDayNum(UInt32 num) const { return makeDayNum(num / 10000, num / 100 % 100, num % 100); }
inline UInt64 toNumYYYYMMDDhhmmss(time_t t) const
{
const Values & values = find(t);
return
toSecond(t)
+ toMinute(t) * 100
+ toHour(t) * 10000
+ UInt64(values.day_of_month) * 1000000
+ UInt64(values.month) * 100000000
+ UInt64(values.year) * 10000000000;
return toSecond(t) + toMinute(t) * 100 + toHour(t) * 10000 + UInt64(values.day_of_month) * 1000000
+ UInt64(values.month) * 100000000 + UInt64(values.year) * 10000000000;
}
inline time_t YYYYMMDDhhmmssToTime(UInt64 num) const
{
return makeDateTime(
num / 10000000000,
num / 100000000 % 100,
num / 1000000 % 100,
num / 10000 % 100,
num / 100 % 100,
num % 100);
return makeDateTime(num / 10000000000, num / 100000000 % 100, num / 1000000 % 100, num / 10000 % 100, num / 100 % 100, num % 100);
}
/// Adding calendar intervals.
@ -641,10 +679,7 @@ public:
return lut[index].date + time_offset;
}
inline time_t addWeeks(time_t t, Int64 delta) const
{
return addDays(t, delta * 7);
}
inline time_t addWeeks(time_t t, Int64 delta) const { return addDays(t, delta * 7); }
inline UInt8 saturateDayOfMonth(UInt16 year, UInt8 month, UInt8 day_of_month) const
{
@ -697,15 +732,9 @@ public:
}
}
inline time_t addQuarters(time_t t, Int64 delta) const
{
return addMonths(t, delta * 3);
}
inline time_t addQuarters(time_t t, Int64 delta) const { return addMonths(t, delta * 3); }
inline DayNum addQuarters(DayNum d, Int64 delta) const
{
return addMonths(d, delta * 3);
}
inline DayNum addQuarters(DayNum d, Int64 delta) const { return addMonths(d, delta * 3); }
/// Saturation can occur if 29 Feb is mapped to non-leap year.
inline time_t addYears(time_t t, Int64 delta) const
@ -740,7 +769,7 @@ public:
{
const Values & values = find(t);
std::string s {"0000-00-00 00:00:00"};
std::string s{"0000-00-00 00:00:00"};
s[0] += values.year / 1000;
s[1] += (values.year / 100) % 10;
@ -769,7 +798,7 @@ public:
{
const Values & values = find(t);
std::string s {"0000-00-00"};
std::string s{"0000-00-00"};
s[0] += values.year / 1000;
s[1] += (values.year / 100) % 10;
@ -787,7 +816,7 @@ public:
{
const Values & values = lut[d];
std::string s {"0000-00-00"};
std::string s{"0000-00-00"};
s[0] += values.year / 1000;
s[1] += (values.year / 100) % 10;
@ -803,7 +832,7 @@ public:
};
#if defined(__PPC__)
#if !__clang__
#pragma GCC diagnostic pop
#endif
# if !__clang__
# pragma GCC diagnostic pop
# endif
#endif