提交 992e7dc7 编写于 作者: A Alexey Arno

Merge

上级 ba715bdd
......@@ -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);
......
#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);
/// Инициализация указателя на реализацию для часового пояса по умолчанию.
auto it = time_zone_to_group.find(default_time_zone);
if (it == time_zone_to_group.end())
throw Poco::Exception("Failed to get default time zone information.");
default_group_id = it->second;
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);
}
DateLUTImpl & DateLUT::get(const std::string & time_zone)
const DateLUTImpl & DateLUT::getImplementation(const std::string & time_zone, size_t group_id) const
{
const std::string & actual_time_zone = time_zone.empty() ? default_time_zone : time_zone;
auto it = time_zone_to_group.find(actual_time_zone);
if (it == time_zone_to_group.end())
throw Poco::Exception("Invalid time zone " + actual_time_zone);
static auto & date_lut_table = *date_lut_impl_list;
const auto & group_id = it->second;
auto & wrapper = (*date_lut_impl_list)[group_id];
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());
}
}
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册