mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge
This commit is contained in:
parent
ba715bdde0
commit
992e7dc7a5
@ -69,7 +69,7 @@ struct ConvertImpl<DataTypeDate, DataTypeDateTime, Name>
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
typedef DataTypeDate::FieldType FromFieldType;
|
||||
auto & date_lut = DateLUT::instance();
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
|
||||
if (const ColumnVector<FromFieldType> * col_from = typeid_cast<const ColumnVector<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
@ -108,7 +108,7 @@ struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name>
|
||||
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
auto & date_lut = DateLUT::instance();
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
|
||||
if (const ColumnVector<FromFieldType> * col_from = typeid_cast<const ColumnVector<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
@ -158,7 +158,7 @@ struct ConvertImpl32Or64ToDate
|
||||
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
auto & date_lut = DateLUT::instance();
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
|
||||
if (const ColumnVector<FromFieldType> * col_from
|
||||
= typeid_cast<const ColumnVector<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
||||
@ -248,7 +248,7 @@ namespace details { namespace {
|
||||
* объекту from_date_lut. Эта функция возвращает timestamp представлящий те же дату и время
|
||||
* в часовом поясе соответствующем объекту to_date_lut.
|
||||
*/
|
||||
time_t convertTimestamp(time_t source_timestamp, DateLUTImpl & from_date_lut, DateLUTImpl & to_date_lut)
|
||||
time_t convertTimestamp(time_t source_timestamp, const DateLUTImpl & from_date_lut, const DateLUTImpl & to_date_lut)
|
||||
{
|
||||
if (&from_date_lut == &to_date_lut)
|
||||
return source_timestamp;
|
||||
@ -271,7 +271,7 @@ struct DateTimeToStringConverter
|
||||
static void vector_vector(const PODArray<FromFieldType> & vec_from, const ColumnString::Chars_t & data,
|
||||
const ColumnString::Offsets_t & offsets, ColumnString & vec_to)
|
||||
{
|
||||
auto & local_date_lut = DateLUT::instance();
|
||||
const auto & local_date_lut = DateLUT::instance();
|
||||
|
||||
ColumnString::Chars_t & data_to = vec_to.getChars();
|
||||
ColumnString::Offsets_t & offsets_to = vec_to.getOffsets();
|
||||
@ -287,7 +287,7 @@ struct DateTimeToStringConverter
|
||||
{
|
||||
ColumnString::Offset_t cur_offset = offsets[i];
|
||||
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||
auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
const auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
|
||||
auto ti = convertTimestamp(vec_from[i], remote_date_lut, local_date_lut);
|
||||
formatImpl<DataTypeDateTime>(ti, write_buffer);
|
||||
@ -302,8 +302,8 @@ struct DateTimeToStringConverter
|
||||
static void vector_constant(const PODArray<FromFieldType> & vec_from, const std::string & data,
|
||||
ColumnString & vec_to)
|
||||
{
|
||||
auto & local_date_lut = DateLUT::instance();
|
||||
auto & remote_date_lut = DateLUT::instance(data);
|
||||
const auto & local_date_lut = DateLUT::instance();
|
||||
const auto & remote_date_lut = DateLUT::instance(data);
|
||||
|
||||
ColumnString::Chars_t & data_to = vec_to.getChars();
|
||||
ColumnString::Offsets_t & offsets_to = vec_to.getOffsets();
|
||||
@ -327,7 +327,7 @@ struct DateTimeToStringConverter
|
||||
const ColumnString::Offsets_t & offsets,
|
||||
ColumnString & vec_to)
|
||||
{
|
||||
auto & local_date_lut = DateLUT::instance();
|
||||
const auto & local_date_lut = DateLUT::instance();
|
||||
|
||||
ColumnString::Chars_t & data_to = vec_to.getChars();
|
||||
ColumnString::Offsets_t & offsets_to = vec_to.getOffsets();
|
||||
@ -343,7 +343,7 @@ struct DateTimeToStringConverter
|
||||
{
|
||||
ColumnString::Offset_t cur_offset = offsets[i];
|
||||
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||
auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
const auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
|
||||
auto ti = convertTimestamp(from, remote_date_lut, local_date_lut);
|
||||
formatImpl<DataTypeDateTime>(ti, write_buffer);
|
||||
@ -357,8 +357,8 @@ struct DateTimeToStringConverter
|
||||
|
||||
static void constant_constant(FromFieldType from, const std::string & data, std::string & to)
|
||||
{
|
||||
auto & local_date_lut = DateLUT::instance();
|
||||
auto & remote_date_lut = DateLUT::instance(data);
|
||||
const auto & local_date_lut = DateLUT::instance();
|
||||
const auto & remote_date_lut = DateLUT::instance(data);
|
||||
|
||||
std::vector<char> buf;
|
||||
WriteBufferFromVector<std::vector<char> > write_buffer(buf);
|
||||
@ -533,7 +533,7 @@ struct StringToTimestampConverter
|
||||
static void vector_vector(const ColumnString::Chars_t & vec_from, const ColumnString::Chars_t & data,
|
||||
const ColumnString::Offsets_t & offsets, PODArray<ToFieldType> & vec_to)
|
||||
{
|
||||
auto & local_date_lut = DateLUT::instance();
|
||||
const auto & local_date_lut = DateLUT::instance();
|
||||
ReadBuffer read_buffer(const_cast<char *>(reinterpret_cast<const char *>(&vec_from[0])), vec_from.size(), 0);
|
||||
|
||||
ColumnString::Offset_t prev_offset = 0;
|
||||
@ -546,7 +546,7 @@ struct StringToTimestampConverter
|
||||
|
||||
ColumnString::Offset_t cur_offset = offsets[i];
|
||||
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||
auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
const auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
|
||||
auto ti = convertTimestamp(x, local_date_lut, remote_date_lut);
|
||||
|
||||
@ -562,8 +562,8 @@ struct StringToTimestampConverter
|
||||
static void vector_constant(const ColumnString::Chars_t & vec_from, const std::string & data,
|
||||
PODArray<ToFieldType> & vec_to)
|
||||
{
|
||||
auto & local_date_lut = DateLUT::instance();
|
||||
auto & remote_date_lut = DateLUT::instance(data);
|
||||
const auto & local_date_lut = DateLUT::instance();
|
||||
const auto & remote_date_lut = DateLUT::instance(data);
|
||||
ReadBuffer read_buffer(const_cast<char *>(reinterpret_cast<const char *>(&vec_from[0])), vec_from.size(), 0);
|
||||
|
||||
char zero = 0;
|
||||
@ -584,7 +584,7 @@ struct StringToTimestampConverter
|
||||
static void constant_vector(const std::string & from, const ColumnString::Chars_t & data,
|
||||
const ColumnString::Offsets_t & offsets, PODArray<ToFieldType> & vec_to)
|
||||
{
|
||||
auto & local_date_lut = DateLUT::instance();
|
||||
const auto & local_date_lut = DateLUT::instance();
|
||||
|
||||
ReadBufferFromString read_buffer(from);
|
||||
DataTypeDateTime::FieldType x = 0;
|
||||
@ -596,7 +596,7 @@ struct StringToTimestampConverter
|
||||
{
|
||||
ColumnString::Offset_t cur_offset = offsets[i];
|
||||
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||
auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
const auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
|
||||
auto ti = convertTimestamp(x, local_date_lut, remote_date_lut);
|
||||
|
||||
@ -607,8 +607,8 @@ struct StringToTimestampConverter
|
||||
|
||||
static void constant_constant(const std::string & from, const std::string & data, ToFieldType & to)
|
||||
{
|
||||
auto & local_date_lut = DateLUT::instance();
|
||||
auto & remote_date_lut = DateLUT::instance(data);
|
||||
const auto & local_date_lut = DateLUT::instance();
|
||||
const auto & remote_date_lut = DateLUT::instance(data);
|
||||
|
||||
ReadBufferFromString read_buffer(from);
|
||||
DataTypeDateTime::FieldType x = 0;
|
||||
|
@ -51,32 +51,32 @@ namespace DB
|
||||
|
||||
struct ToYearImpl
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toYear(t); }
|
||||
static inline UInt16 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toYear(DayNum_t(d)); }
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toYear(t); }
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toYear(DayNum_t(d)); }
|
||||
};
|
||||
|
||||
struct ToMonthImpl
|
||||
{
|
||||
static inline UInt8 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toMonth(t); }
|
||||
static inline UInt8 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toMonth(DayNum_t(d)); }
|
||||
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toMonth(t); }
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toMonth(DayNum_t(d)); }
|
||||
};
|
||||
|
||||
struct ToDayOfMonthImpl
|
||||
{
|
||||
static inline UInt8 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toDayOfMonth(t); }
|
||||
static inline UInt8 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toDayOfMonth(DayNum_t(d)); }
|
||||
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toDayOfMonth(t); }
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toDayOfMonth(DayNum_t(d)); }
|
||||
};
|
||||
|
||||
struct ToDayOfWeekImpl
|
||||
{
|
||||
static inline UInt8 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toDayOfWeek(t); }
|
||||
static inline UInt8 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toDayOfWeek(DayNum_t(d)); }
|
||||
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toDayOfWeek(t); }
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toDayOfWeek(DayNum_t(d)); }
|
||||
};
|
||||
|
||||
struct ToHourImpl
|
||||
{
|
||||
static inline UInt8 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toHourInaccurate(t); }
|
||||
static inline UInt8 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
|
||||
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toHourInaccurate(t); }
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toHour", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -84,8 +84,8 @@ struct ToHourImpl
|
||||
|
||||
struct ToMinuteImpl
|
||||
{
|
||||
static inline UInt8 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toMinuteInaccurate(t); }
|
||||
static inline UInt8 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
|
||||
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toMinuteInaccurate(t); }
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -93,8 +93,8 @@ struct ToMinuteImpl
|
||||
|
||||
struct ToSecondImpl
|
||||
{
|
||||
static inline UInt8 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toSecondInaccurate(t); }
|
||||
static inline UInt8 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
|
||||
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toSecondInaccurate(t); }
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toSecond", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -102,33 +102,33 @@ struct ToSecondImpl
|
||||
|
||||
struct ToMondayImpl
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfWeek(remote_date_lut.toDayNum(t)); }
|
||||
static inline UInt16 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfWeek(DayNum_t(d)); }
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfWeek(remote_date_lut.toDayNum(t)); }
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfWeek(DayNum_t(d)); }
|
||||
};
|
||||
|
||||
struct ToStartOfMonthImpl
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfMonth(remote_date_lut.toDayNum(t)); }
|
||||
static inline UInt16 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfMonth(DayNum_t(d)); }
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfMonth(remote_date_lut.toDayNum(t)); }
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfMonth(DayNum_t(d)); }
|
||||
};
|
||||
|
||||
struct ToStartOfQuarterImpl
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfQuarter(remote_date_lut.toDayNum(t)); }
|
||||
static inline UInt16 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfQuarter(DayNum_t(d)); }
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfQuarter(remote_date_lut.toDayNum(t)); }
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfQuarter(DayNum_t(d)); }
|
||||
};
|
||||
|
||||
struct ToStartOfYearImpl
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfYear(remote_date_lut.toDayNum(t)); }
|
||||
static inline UInt16 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfYear(DayNum_t(d)); }
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfYear(remote_date_lut.toDayNum(t)); }
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfYear(DayNum_t(d)); }
|
||||
};
|
||||
|
||||
|
||||
struct ToTimeImpl
|
||||
{
|
||||
/// При переводе во время, дату будем приравнивать к 1970-01-02.
|
||||
static inline UInt32 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
{
|
||||
time_t remote_t = remote_date_lut.toTimeInaccurate(t) + 86400;
|
||||
|
||||
@ -144,7 +144,7 @@ struct ToTimeImpl
|
||||
}
|
||||
}
|
||||
|
||||
static inline UInt32 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -152,8 +152,8 @@ struct ToTimeImpl
|
||||
|
||||
struct ToStartOfMinuteImpl
|
||||
{
|
||||
static inline UInt32 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toStartOfMinuteInaccurate(t); }
|
||||
static inline UInt32 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toStartOfMinuteInaccurate(t); }
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toStartOfMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -161,8 +161,8 @@ struct ToStartOfMinuteImpl
|
||||
|
||||
struct ToStartOfFiveMinuteImpl
|
||||
{
|
||||
static inline UInt32 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toStartOfFiveMinuteInaccurate(t); }
|
||||
static inline UInt32 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toStartOfFiveMinuteInaccurate(t); }
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toStartOfFiveMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -170,8 +170,8 @@ struct ToStartOfFiveMinuteImpl
|
||||
|
||||
struct ToStartOfHourImpl
|
||||
{
|
||||
static inline UInt32 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toStartOfHourInaccurate(t); }
|
||||
static inline UInt32 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toStartOfHourInaccurate(t); }
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toStartOfHour", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -179,33 +179,33 @@ struct ToStartOfHourImpl
|
||||
|
||||
struct ToRelativeYearNumImpl
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toYear(t); }
|
||||
static inline UInt16 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toYear(DayNum_t(d)); }
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toYear(t); }
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toYear(DayNum_t(d)); }
|
||||
};
|
||||
|
||||
struct ToRelativeMonthNumImpl
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeMonthNum(t); }
|
||||
static inline UInt16 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeMonthNum(DayNum_t(d)); }
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeMonthNum(t); }
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeMonthNum(DayNum_t(d)); }
|
||||
};
|
||||
|
||||
struct ToRelativeWeekNumImpl
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeWeekNum(t); }
|
||||
static inline UInt16 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeWeekNum(DayNum_t(d)); }
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeWeekNum(t); }
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeWeekNum(DayNum_t(d)); }
|
||||
};
|
||||
|
||||
struct ToRelativeDayNumImpl
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toDayNum(t); }
|
||||
static inline UInt16 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return static_cast<DayNum_t>(d); }
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toDayNum(t); }
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return static_cast<DayNum_t>(d); }
|
||||
};
|
||||
|
||||
|
||||
struct ToRelativeHourNumImpl
|
||||
{
|
||||
static inline UInt32 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeHourNum(t); }
|
||||
static inline UInt32 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeHourNum(t); }
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toRelativeHourNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -213,8 +213,8 @@ struct ToRelativeHourNumImpl
|
||||
|
||||
struct ToRelativeMinuteNumImpl
|
||||
{
|
||||
static inline UInt32 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeMinuteNum(t); }
|
||||
static inline UInt32 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeMinuteNum(t); }
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toRelativeMinuteNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -222,8 +222,8 @@ struct ToRelativeMinuteNumImpl
|
||||
|
||||
struct ToRelativeSecondNumImpl
|
||||
{
|
||||
static inline UInt32 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return t; }
|
||||
static inline UInt32 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return t; }
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toRelativeSecondNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -235,14 +235,14 @@ struct Transformer
|
||||
static void vector_vector(const PODArray<FromType> & vec_from, const ColumnString::Chars_t & data,
|
||||
const ColumnString::Offsets_t & offsets, PODArray<ToType> & vec_to)
|
||||
{
|
||||
auto & local_date_lut = DateLUT::instance();
|
||||
const auto & local_date_lut = DateLUT::instance();
|
||||
ColumnString::Offset_t prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < vec_from.size(); ++i)
|
||||
{
|
||||
ColumnString::Offset_t cur_offset = offsets[i];
|
||||
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||
auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
const auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
vec_to[i] = Transform::execute(vec_from[i], remote_date_lut, local_date_lut);
|
||||
prev_offset = cur_offset;
|
||||
}
|
||||
@ -251,8 +251,8 @@ struct Transformer
|
||||
static void vector_constant(const PODArray<FromType> & vec_from, const std::string & data,
|
||||
PODArray<ToType> & vec_to)
|
||||
{
|
||||
auto & local_date_lut = DateLUT::instance();
|
||||
auto & remote_date_lut = DateLUT::instance(data);
|
||||
const auto & local_date_lut = DateLUT::instance();
|
||||
const auto & remote_date_lut = DateLUT::instance(data);
|
||||
for (size_t i = 0; i < vec_from.size(); ++i)
|
||||
vec_to[i] = Transform::execute(vec_from[i], remote_date_lut, local_date_lut);
|
||||
}
|
||||
@ -260,14 +260,14 @@ struct Transformer
|
||||
static void constant_vector(const FromType & from, const ColumnString::Chars_t & data,
|
||||
const ColumnString::Offsets_t & offsets, PODArray<ToType> & vec_to)
|
||||
{
|
||||
auto & local_date_lut = DateLUT::instance();
|
||||
const auto & local_date_lut = DateLUT::instance();
|
||||
ColumnString::Offset_t prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
ColumnString::Offset_t cur_offset = offsets[i];
|
||||
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||
auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
const auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
vec_to[i] = Transform::execute(from, remote_date_lut, local_date_lut);
|
||||
prev_offset = cur_offset;
|
||||
}
|
||||
@ -275,8 +275,8 @@ struct Transformer
|
||||
|
||||
static void constant_constant(const FromType & from, const std::string & data, ToType & to)
|
||||
{
|
||||
auto & local_date_lut = DateLUT::instance();
|
||||
auto & remote_date_lut = DateLUT::instance(data);
|
||||
const auto & local_date_lut = DateLUT::instance();
|
||||
const auto & remote_date_lut = DateLUT::instance(data);
|
||||
to = Transform::execute(from, remote_date_lut, local_date_lut);
|
||||
}
|
||||
};
|
||||
|
@ -375,7 +375,7 @@ inline void writeDateTimeText(time_t datetime, WriteBuffer & buf)
|
||||
return;
|
||||
}
|
||||
|
||||
auto & date_lut = DateLUT::instance();
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
const auto & values = date_lut.getValues(datetime);
|
||||
|
||||
s[0] += values.year / 1000;
|
||||
|
@ -210,7 +210,7 @@ void QueryLog::flush()
|
||||
{
|
||||
LOG_TRACE(log, "Flushing query log");
|
||||
|
||||
auto & date_lut = DateLUT::instance();
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
|
||||
Block block = createBlock();
|
||||
|
||||
|
@ -95,7 +95,7 @@ QueryParseResult QueryParser::parse(std::istream & s)
|
||||
if (result.date_first > result.date_last)
|
||||
throw Exception("First date is bigger than last date.", ErrorCodes::FIRST_DATE_IS_BIGGER_THAN_LAST_DATE);
|
||||
|
||||
auto & date_lut = DateLUT::instance();
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
result.days = 1 + date_lut.toDayNum(result.date_last) - date_lut.toDayNum(result.date_first);
|
||||
|
||||
result.cut_date_last = false;
|
||||
|
@ -105,7 +105,7 @@ size_t ActiveDataPartSet::size() const
|
||||
|
||||
String ActiveDataPartSet::getPartName(DayNum_t left_date, DayNum_t right_date, UInt64 left_id, UInt64 right_id, UInt64 level)
|
||||
{
|
||||
auto & date_lut = DateLUT::instance();
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
|
||||
/// Имя директории для куска иммет вид: YYYYMMDD_YYYYMMDD_N_N_L.
|
||||
String res;
|
||||
@ -153,7 +153,7 @@ void ActiveDataPartSet::parsePartName(const String & file_name, Part & part, con
|
||||
|
||||
const Poco::RegularExpression::MatchVec & matches = *matches_p;
|
||||
|
||||
auto & date_lut = DateLUT::instance();
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
|
||||
part.left_date = date_lut.YYYYMMDDToDayNum(parse<UInt32>(file_name.substr(matches[1].offset, matches[1].length)));
|
||||
part.right_date = date_lut.YYYYMMDDToDayNum(parse<UInt32>(file_name.substr(matches[2].offset, matches[2].length)));
|
||||
|
@ -46,7 +46,7 @@ bool MergeTreeDataMerger::selectPartsToMerge(MergeTreeData::DataPartsVector & pa
|
||||
{
|
||||
MergeTreeData::DataParts data_parts = data.getDataParts();
|
||||
|
||||
auto & date_lut = DateLUT::instance();
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
|
||||
size_t min_max = -1U;
|
||||
size_t min_min = -1U;
|
||||
|
@ -12,7 +12,7 @@ BlocksWithDateIntervals MergeTreeDataWriter::splitBlockIntoParts(const Block & b
|
||||
{
|
||||
data.check(block, true);
|
||||
|
||||
auto & date_lut = DateLUT::instance();
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
|
||||
size_t rows = block.rows();
|
||||
size_t columns = block.columns();
|
||||
@ -77,7 +77,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithDa
|
||||
UInt16 min_date = block_with_dates.min_date;
|
||||
UInt16 max_date = block_with_dates.max_date;
|
||||
|
||||
auto & date_lut = DateLUT::instance();
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
|
||||
size_t part_size = (block.rows() + data.index_granularity - 1) / data.index_granularity;
|
||||
|
||||
|
@ -2214,7 +2214,7 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params,
|
||||
static String getFakePartNameForDrop(const String & month_name, UInt64 left, UInt64 right)
|
||||
{
|
||||
/// Диапазон дат - весь месяц.
|
||||
auto & lut = DateLUT::instance();
|
||||
const auto & lut = DateLUT::instance();
|
||||
time_t start_time = lut.YYYYMMDDToDate(parse<UInt32>(month_name + "01"));
|
||||
DayNum_t left_date = lut.toDayNum(start_time);
|
||||
DayNum_t right_date = DayNum_t(static_cast<size_t>(left_date) + lut.daysInMonth(start_time) - 1);
|
||||
|
@ -1,11 +1,8 @@
|
||||
#include <Yandex/DateLUT.h>
|
||||
#include <Poco/Exception.h>
|
||||
|
||||
#include <unicode/timezone.h>
|
||||
#include <unicode/unistr.h>
|
||||
|
||||
std::string DateLUT::default_time_zone;
|
||||
|
||||
DateLUT::DateLUT()
|
||||
{
|
||||
using namespace icu;
|
||||
@ -16,6 +13,7 @@ DateLUT::DateLUT()
|
||||
|
||||
UnicodeString u_out;
|
||||
tz->getID(u_out);
|
||||
std::string default_time_zone;
|
||||
u_out.toUTF8String(default_time_zone);
|
||||
|
||||
std::unique_ptr<StringEnumeration> time_zone_ids(TimeZone::createEnumeration());
|
||||
@ -79,37 +77,35 @@ DateLUT::DateLUT()
|
||||
throw Poco::Exception("Could not find any time zone information.");
|
||||
|
||||
date_lut_impl_list = std::make_unique<DateLUTImplList>(group_id);
|
||||
}
|
||||
|
||||
DateLUTImpl & DateLUT::instance(const std::string & time_zone)
|
||||
{
|
||||
auto & date_lut = Singleton<DateLUT>::instance();
|
||||
return date_lut.get(time_zone);
|
||||
}
|
||||
|
||||
DateLUTImpl & DateLUT::get(const std::string & time_zone)
|
||||
{
|
||||
const std::string & actual_time_zone = time_zone.empty() ? default_time_zone : time_zone;
|
||||
|
||||
auto it = time_zone_to_group.find(actual_time_zone);
|
||||
/// Инициализация указателя на реализацию для часового пояса по умолчанию.
|
||||
auto it = time_zone_to_group.find(default_time_zone);
|
||||
if (it == time_zone_to_group.end())
|
||||
throw Poco::Exception("Invalid time zone " + actual_time_zone);
|
||||
throw Poco::Exception("Failed to get default time zone information.");
|
||||
default_group_id = it->second;
|
||||
|
||||
const auto & group_id = it->second;
|
||||
auto & wrapper = (*date_lut_impl_list)[group_id];
|
||||
default_date_lut_impl = new DateLUTImpl(default_time_zone);
|
||||
auto & wrapper = (*date_lut_impl_list)[default_group_id];
|
||||
wrapper.store(default_date_lut_impl, std::memory_order_seq_cst);
|
||||
}
|
||||
|
||||
const DateLUTImpl & DateLUT::getImplementation(const std::string & time_zone, size_t group_id) const
|
||||
{
|
||||
static auto & date_lut_table = *date_lut_impl_list;
|
||||
|
||||
auto & wrapper = date_lut_table[group_id];
|
||||
|
||||
DateLUTImpl * tmp = wrapper.load(std::memory_order_acquire);
|
||||
if (tmp == nullptr)
|
||||
{
|
||||
std::lock_guard<std::mutex> guard(mutex);
|
||||
tmp = wrapper.load(std::memory_order_acquire);
|
||||
tmp = wrapper.load(std::memory_order_relaxed);
|
||||
if (tmp == nullptr)
|
||||
{
|
||||
tmp = new DateLUTImpl(actual_time_zone);
|
||||
tmp = new DateLUTImpl(time_zone);
|
||||
wrapper.store(tmp, std::memory_order_release);
|
||||
}
|
||||
}
|
||||
|
||||
return *tmp;
|
||||
}
|
||||
|
||||
|
@ -33,7 +33,7 @@ static time_t orderedIdentifierToDate(unsigned value)
|
||||
|
||||
void loop(time_t begin, time_t end, int step)
|
||||
{
|
||||
auto & date_lut = DateLUT::instance();
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
|
||||
for (time_t t = begin; t < end; t += step)
|
||||
std::cout << toString(t)
|
||||
|
@ -35,7 +35,7 @@ static time_t orderedIdentifierToDate(unsigned value)
|
||||
|
||||
void loop(time_t begin, time_t end, int step)
|
||||
{
|
||||
auto & date_lut = DateLUT::instance();
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
|
||||
for (time_t t = begin; t < end; t += step)
|
||||
{
|
||||
|
@ -7,7 +7,7 @@ int main(int argc, char ** argv)
|
||||
/** В DateLUT был глюк - для времён из дня 1970-01-01, возвращался номер часа больше 23. */
|
||||
static const time_t TIME = 66130;
|
||||
|
||||
auto & date_lut = DateLUT::instance();
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
|
||||
std::cerr << date_lut.toHourInaccurate(TIME) << std::endl;
|
||||
std::cerr << date_lut.toDayNum(TIME) << std::endl;
|
||||
|
@ -29,7 +29,7 @@ private:
|
||||
|
||||
void init(time_t time)
|
||||
{
|
||||
auto & date_lut = DateLUT::instance();
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
const auto & values = date_lut.getValues(time);
|
||||
|
||||
m_year = values.year;
|
||||
|
@ -43,7 +43,7 @@ private:
|
||||
return;
|
||||
}
|
||||
|
||||
auto & date_lut = DateLUT::instance();
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
const auto & values = date_lut.getValues(time);
|
||||
|
||||
m_year = values.year;
|
||||
|
@ -145,7 +145,7 @@ private:
|
||||
|
||||
time_t getDateTimeImpl() const
|
||||
{
|
||||
auto & date_lut = DateLUT::instance();
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
|
||||
if (m_length == 10)
|
||||
{
|
||||
@ -173,7 +173,7 @@ private:
|
||||
|
||||
time_t getDateImpl() const
|
||||
{
|
||||
auto & date_lut = DateLUT::instance();
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
|
||||
if (m_length == 10 || m_length == 19)
|
||||
{
|
||||
@ -216,7 +216,7 @@ private:
|
||||
return getDateImpl();
|
||||
else
|
||||
{
|
||||
auto & date_lut = DateLUT::instance();
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
return date_lut.toDate(getIntImpl());
|
||||
}
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user