From ca2b79330d5dd53eb7c358c52da398e1bf2e441c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 9 Jul 2014 03:52:53 +0400 Subject: [PATCH] Tiny improvement [#METR-2807]. --- .../DB/Functions/FunctionsConversion.h | 4 +- dbms/include/DB/Functions/FunctionsDateTime.h | 86 +++++++++---------- dbms/include/DB/IO/ReadHelpers.h | 4 +- dbms/include/DB/IO/WriteHelpers.h | 4 +- .../ReplicatedMergeTreeBlockOutputStream.h | 2 +- dbms/src/Client/Client.cpp | 2 +- dbms/src/Interpreters/tests/select_query.cpp | 2 +- dbms/src/Server/OLAPQueryConverter.cpp | 4 +- dbms/src/Server/OLAPQueryParser.cpp | 2 +- dbms/src/Server/Server.cpp | 2 +- .../Storages/MergeTree/ActiveDataPartSet.cpp | 4 +- .../MergeTree/MergeTreeDataMerger.cpp | 2 +- .../MergeTree/MergeTreeDataWriter.cpp | 4 +- libs/libcommon/src/DateLUT.cpp | 11 ++- libs/libcommon/src/tests/date_lut2.cpp | 2 +- libs/libcommon/src/tests/date_lut3.cpp | 2 +- libs/libmysqlxx/include/mysqlxx/Date.h | 8 +- libs/libmysqlxx/include/mysqlxx/DateTime.h | 4 +- libs/libmysqlxx/include/mysqlxx/Value.h | 6 +- 19 files changed, 80 insertions(+), 75 deletions(-) diff --git a/dbms/include/DB/Functions/FunctionsConversion.h b/dbms/include/DB/Functions/FunctionsConversion.h index 70ace0ae499..e15d6eafa31 100644 --- a/dbms/include/DB/Functions/FunctionsConversion.h +++ b/dbms/include/DB/Functions/FunctionsConversion.h @@ -68,7 +68,7 @@ struct ConvertImpl static void execute(Block & block, const ColumnNumbers & arguments, size_t result) { typedef DataTypeDate::FieldType FromFieldType; - DateLUTSingleton & date_lut = DateLUTSingleton::instance(); + DateLUT & date_lut = DateLUT::instance(); if (const ColumnVector * col_from = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) { @@ -107,7 +107,7 @@ struct ConvertImpl static void execute(Block & block, const ColumnNumbers & arguments, size_t result) { - DateLUTSingleton & date_lut = DateLUTSingleton::instance(); + DateLUT & date_lut = DateLUT::instance(); if (const ColumnVector * col_from = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) { diff --git a/dbms/include/DB/Functions/FunctionsDateTime.h b/dbms/include/DB/Functions/FunctionsDateTime.h index cb22b903acb..54ffc423e60 100644 --- a/dbms/include/DB/Functions/FunctionsDateTime.h +++ b/dbms/include/DB/Functions/FunctionsDateTime.h @@ -48,32 +48,32 @@ namespace DB struct ToYearImpl { - static inline UInt16 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toYear(t); } - static inline UInt16 execute(UInt16 d, DateLUTSingleton & date_lut) { return date_lut.toYear(DayNum_t(d)); } + static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toYear(t); } + static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toYear(DayNum_t(d)); } }; struct ToMonthImpl { - static inline UInt8 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toMonth(t); } - static inline UInt8 execute(UInt16 d, DateLUTSingleton & date_lut) { return date_lut.toMonth(DayNum_t(d)); } + static inline UInt8 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toMonth(t); } + static inline UInt8 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toMonth(DayNum_t(d)); } }; struct ToDayOfMonthImpl { - static inline UInt8 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toDayOfMonth(t); } - static inline UInt8 execute(UInt16 d, DateLUTSingleton & date_lut) { return date_lut.toDayOfMonth(DayNum_t(d)); } + static inline UInt8 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toDayOfMonth(t); } + static inline UInt8 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toDayOfMonth(DayNum_t(d)); } }; struct ToDayOfWeekImpl { - static inline UInt8 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toDayOfWeek(t); } - static inline UInt8 execute(UInt16 d, DateLUTSingleton & date_lut) { return date_lut.toDayOfWeek(DayNum_t(d)); } + static inline UInt8 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toDayOfWeek(t); } + static inline UInt8 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toDayOfWeek(DayNum_t(d)); } }; struct ToHourImpl { - static inline UInt8 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toHourInaccurate(t); } - static inline UInt8 execute(UInt16 d, DateLUTSingleton & date_lut) + static inline UInt8 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toHourInaccurate(t); } + static inline UInt8 execute(UInt16 d, DateLUT & date_lut) { throw Exception("Illegal type Date of argument for function toHour", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -81,8 +81,8 @@ struct ToHourImpl struct ToMinuteImpl { - static inline UInt8 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toMinuteInaccurate(t); } - static inline UInt8 execute(UInt16 d, DateLUTSingleton & date_lut) + static inline UInt8 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toMinuteInaccurate(t); } + static inline UInt8 execute(UInt16 d, DateLUT & date_lut) { throw Exception("Illegal type Date of argument for function toMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -90,8 +90,8 @@ struct ToMinuteImpl struct ToSecondImpl { - static inline UInt8 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toSecondInaccurate(t); } - static inline UInt8 execute(UInt16 d, DateLUTSingleton & date_lut) + static inline UInt8 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toSecondInaccurate(t); } + static inline UInt8 execute(UInt16 d, DateLUT & date_lut) { throw Exception("Illegal type Date of argument for function toSecond", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -99,34 +99,34 @@ struct ToSecondImpl struct ToMondayImpl { - static inline UInt16 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toFirstDayNumOfWeek(date_lut.toDayNum(t)); } - static inline UInt16 execute(UInt16 d, DateLUTSingleton & date_lut) { return date_lut.toFirstDayNumOfWeek(DayNum_t(d)); } + static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toFirstDayNumOfWeek(date_lut.toDayNum(t)); } + static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toFirstDayNumOfWeek(DayNum_t(d)); } }; struct ToStartOfMonthImpl { - static inline UInt16 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toFirstDayNumOfMonth(date_lut.toDayNum(t)); } - static inline UInt16 execute(UInt16 d, DateLUTSingleton & date_lut) { return date_lut.toFirstDayNumOfMonth(DayNum_t(d)); } + static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toFirstDayNumOfMonth(date_lut.toDayNum(t)); } + static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toFirstDayNumOfMonth(DayNum_t(d)); } }; struct ToStartOfQuarterImpl { - static inline UInt16 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toFirstDayNumOfQuarter(date_lut.toDayNum(t)); } - static inline UInt16 execute(UInt16 d, DateLUTSingleton & date_lut) { return date_lut.toFirstDayNumOfQuarter(DayNum_t(d)); } + static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toFirstDayNumOfQuarter(date_lut.toDayNum(t)); } + static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toFirstDayNumOfQuarter(DayNum_t(d)); } }; struct ToStartOfYearImpl { - static inline UInt16 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toFirstDayNumOfYear(date_lut.toDayNum(t)); } - static inline UInt16 execute(UInt16 d, DateLUTSingleton & date_lut) { return date_lut.toFirstDayNumOfYear(DayNum_t(d)); } + static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toFirstDayNumOfYear(date_lut.toDayNum(t)); } + static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toFirstDayNumOfYear(DayNum_t(d)); } }; struct ToTimeImpl { /// При переводе во время, дату будем приравнивать к 1970-01-02. - static inline UInt32 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toTimeInaccurate(t) + 86400; } - static inline UInt32 execute(UInt16 d, DateLUTSingleton & date_lut) + static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toTimeInaccurate(t) + 86400; } + static inline UInt32 execute(UInt16 d, DateLUT & date_lut) { throw Exception("Illegal type Date of argument for function toTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -134,8 +134,8 @@ struct ToTimeImpl struct ToStartOfMinuteImpl { - static inline UInt32 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toStartOfMinuteInaccurate(t); } - static inline UInt32 execute(UInt16 d, DateLUTSingleton & date_lut) + static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toStartOfMinuteInaccurate(t); } + static inline UInt32 execute(UInt16 d, DateLUT & date_lut) { throw Exception("Illegal type Date of argument for function toStartOfMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -143,8 +143,8 @@ struct ToStartOfMinuteImpl struct ToStartOfHourImpl { - static inline UInt32 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toStartOfHourInaccurate(t); } - static inline UInt32 execute(UInt16 d, DateLUTSingleton & date_lut) + static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toStartOfHourInaccurate(t); } + static inline UInt32 execute(UInt16 d, DateLUT & date_lut) { throw Exception("Illegal type Date of argument for function toStartOfHour", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -152,33 +152,33 @@ struct ToStartOfHourImpl struct ToRelativeYearNumImpl { - static inline UInt16 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toYear(t); } - static inline UInt16 execute(UInt16 d, DateLUTSingleton & date_lut) { return date_lut.toYear(DayNum_t(d)); } + static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toYear(t); } + static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toYear(DayNum_t(d)); } }; struct ToRelativeMonthNumImpl { - static inline UInt16 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toRelativeMonthNum(t); } - static inline UInt16 execute(UInt16 d, DateLUTSingleton & date_lut) { return date_lut.toRelativeMonthNum(DayNum_t(d)); } + static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toRelativeMonthNum(t); } + static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toRelativeMonthNum(DayNum_t(d)); } }; struct ToRelativeWeekNumImpl { - static inline UInt16 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toRelativeWeekNum(t); } - static inline UInt16 execute(UInt16 d, DateLUTSingleton & date_lut) { return date_lut.toRelativeWeekNum(DayNum_t(d)); } + static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toRelativeWeekNum(t); } + static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toRelativeWeekNum(DayNum_t(d)); } }; struct ToRelativeDayNumImpl { - static inline UInt16 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toDayNum(t); } - static inline UInt16 execute(UInt16 d, DateLUTSingleton & date_lut) { return static_cast(d); } + static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toDayNum(t); } + static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return static_cast(d); } }; struct ToRelativeHourNumImpl { - static inline UInt32 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toRelativeHourNum(t); } - static inline UInt32 execute(UInt16 d, DateLUTSingleton & date_lut) + static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toRelativeHourNum(t); } + static inline UInt32 execute(UInt16 d, DateLUT & date_lut) { throw Exception("Illegal type Date of argument for function toRelativeHourNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -186,8 +186,8 @@ struct ToRelativeHourNumImpl struct ToRelativeMinuteNumImpl { - static inline UInt32 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toRelativeMinuteNum(t); } - static inline UInt32 execute(UInt16 d, DateLUTSingleton & date_lut) + static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toRelativeMinuteNum(t); } + static inline UInt32 execute(UInt16 d, DateLUT & date_lut) { throw Exception("Illegal type Date of argument for function toRelativeMinuteNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -195,8 +195,8 @@ struct ToRelativeMinuteNumImpl struct ToRelativeSecondNumImpl { - static inline UInt32 execute(UInt32 t, DateLUTSingleton & date_lut) { return t; } - static inline UInt32 execute(UInt16 d, DateLUTSingleton & date_lut) + static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return t; } + static inline UInt32 execute(UInt16 d, DateLUT & date_lut) { throw Exception("Illegal type Date of argument for function toRelativeSecondNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -208,7 +208,7 @@ struct DateTimeTransformImpl { static void execute(Block & block, const ColumnNumbers & arguments, size_t result) { - DateLUTSingleton & date_lut = DateLUTSingleton::instance(); + DateLUT & date_lut = DateLUT::instance(); if (const ColumnVector * col_from = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) { diff --git a/dbms/include/DB/IO/ReadHelpers.h b/dbms/include/DB/IO/ReadHelpers.h index 0f364c01c29..33dd48136ed 100644 --- a/dbms/include/DB/IO/ReadHelpers.h +++ b/dbms/include/DB/IO/ReadHelpers.h @@ -341,7 +341,7 @@ inline void readDateText(DayNum_t & date, ReadBuffer & buf) UInt8 month = (s[5] - '0') * 10 + (s[6] - '0'); UInt8 day = (s[8] - '0') * 10 + (s[9] - '0'); - date = DateLUTSingleton::instance().makeDayNum(year, month, day); + date = DateLUT::instance().makeDayNum(year, month, day); } inline void readDateText(mysqlxx::Date & date, ReadBuffer & buf) @@ -382,7 +382,7 @@ inline void readDateTimeText(time_t & datetime, ReadBuffer & buf) if (unlikely(year == 0)) datetime = 0; else - datetime = DateLUTSingleton::instance().makeDateTime(year, month, day, hour, minute, second); + datetime = DateLUT::instance().makeDateTime(year, month, day, hour, minute, second); } inline void readDateTimeText(mysqlxx::DateTime & datetime, ReadBuffer & buf) diff --git a/dbms/include/DB/IO/WriteHelpers.h b/dbms/include/DB/IO/WriteHelpers.h index c3f676fd7f6..4b781f61253 100644 --- a/dbms/include/DB/IO/WriteHelpers.h +++ b/dbms/include/DB/IO/WriteHelpers.h @@ -325,7 +325,7 @@ inline void writeDateText(DayNum_t date, WriteBuffer & buf) return; } - const DateLUT::Values & values = DateLUTSingleton::instance().getValues(date); + const DateLUT::Values & values = DateLUT::instance().getValues(date); s[0] += values.year / 1000; s[1] += (values.year / 100) % 10; @@ -367,7 +367,7 @@ inline void writeDateTimeText(time_t datetime, WriteBuffer & buf) return; } - DateLUTSingleton & date_lut = DateLUTSingleton::instance(); + DateLUT & date_lut = DateLUT::instance(); const DateLUT::Values & values = date_lut.getValues(datetime); s[0] += values.year / 1000; diff --git a/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h b/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h index 4bcad4ac96f..6bba39ed01c 100644 --- a/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h +++ b/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h @@ -24,7 +24,7 @@ public: ++block_index; String block_id = insert_id.empty() ? "" : insert_id + "__" + toString(block_index); - time_t min_date_time = DateLUTSingleton::instance().fromDayNum(DayNum_t(current_block.min_date)); + time_t min_date_time = DateLUT::instance().fromDayNum(DayNum_t(current_block.min_date)); String month_name = toString(Date2OrderedIdentifier(min_date_time) / 100); storage.zookeeper->createIfNotExists(storage.zookeeper_path + "/block_numbers/" + month_name, ""); diff --git a/dbms/src/Client/Client.cpp b/dbms/src/Client/Client.cpp index 09e89fd1dec..d44786ab023 100644 --- a/dbms/src/Client/Client.cpp +++ b/dbms/src/Client/Client.cpp @@ -247,7 +247,7 @@ private: } /// Инициализируем DateLUT, чтобы потраченное время не отображалось, как время, потраченное на запрос. - DateLUTSingleton::instance(); + DateLUT::instance(); loop(); diff --git a/dbms/src/Interpreters/tests/select_query.cpp b/dbms/src/Interpreters/tests/select_query.cpp index 81a1c4f38ab..f1165de9df1 100644 --- a/dbms/src/Interpreters/tests/select_query.cpp +++ b/dbms/src/Interpreters/tests/select_query.cpp @@ -28,7 +28,7 @@ int main(int argc, char ** argv) Logger::root().setLevel("trace"); /// Заранее инициализируем DateLUT, чтобы первая инициализация потом не влияла на измеряемую скорость выполнения. - DateLUTSingleton::instance(); + DateLUT::instance(); DB::Context context; diff --git a/dbms/src/Server/OLAPQueryConverter.cpp b/dbms/src/Server/OLAPQueryConverter.cpp index 0662fea21cc..70c6080323a 100644 --- a/dbms/src/Server/OLAPQueryConverter.cpp +++ b/dbms/src/Server/OLAPQueryConverter.cpp @@ -401,8 +401,8 @@ std::string QueryConverter::convertDateRange(time_t date_first, time_t date_last { WriteBufferFromString first_buf(first_str); WriteBufferFromString last_buf(last_str); - writeDateText(DateLUTSingleton::instance().toDayNum(date_first), first_buf); - writeDateText(DateLUTSingleton::instance().toDayNum(date_last), last_buf); + writeDateText(DateLUT::instance().toDayNum(date_first), first_buf); + writeDateText(DateLUT::instance().toDayNum(date_last), last_buf); } return "StartDate >= toDate('" + first_str + "') AND StartDate <= toDate('" + last_str + "')"; } diff --git a/dbms/src/Server/OLAPQueryParser.cpp b/dbms/src/Server/OLAPQueryParser.cpp index 27dc221a4de..30f6491d6ba 100644 --- a/dbms/src/Server/OLAPQueryParser.cpp +++ b/dbms/src/Server/OLAPQueryParser.cpp @@ -96,7 +96,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); - DateLUTSingleton & date_lut = DateLUTSingleton::instance(); + DateLUT & date_lut = DateLUT::instance(); result.days = 1 + date_lut.toDayNum(result.date_last) - date_lut.toDayNum(result.date_first); result.cut_date_last = false; diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 396c0d06783..b403220f05b 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -225,7 +225,7 @@ int Server::main(const std::vector & args) /// Заранее инициализируем DateLUT, чтобы первая инициализация потом не влияла на измеряемую скорость выполнения. LOG_DEBUG(log, "Initializing DateLUT."); - DateLUTSingleton::instance(); + DateLUT::instance(); LOG_TRACE(log, "Initialized DateLUT."); global_context = new Context; diff --git a/dbms/src/Storages/MergeTree/ActiveDataPartSet.cpp b/dbms/src/Storages/MergeTree/ActiveDataPartSet.cpp index 54c471f5497..e6a116543df 100644 --- a/dbms/src/Storages/MergeTree/ActiveDataPartSet.cpp +++ b/dbms/src/Storages/MergeTree/ActiveDataPartSet.cpp @@ -86,7 +86,7 @@ Strings ActiveDataPartSet::getParts() const String ActiveDataPartSet::getPartName(DayNum_t left_date, DayNum_t right_date, UInt64 left_id, UInt64 right_id, UInt64 level) { - DateLUTSingleton & date_lut = DateLUTSingleton::instance(); + DateLUT & date_lut = DateLUT::instance(); /// Имя директории для куска иммет вид: YYYYMMDD_YYYYMMDD_N_N_L. String res; @@ -128,7 +128,7 @@ void ActiveDataPartSet::parsePartName(const String & file_name, Part & part, con const Poco::RegularExpression::MatchVec & matches = *matches_p; - DateLUTSingleton & date_lut = DateLUTSingleton::instance(); + DateLUT & date_lut = DateLUT::instance(); part.left_date = date_lut.toDayNum(OrderedIdentifier2Date(file_name.substr(matches[1].offset, matches[1].length))); part.right_date = date_lut.toDayNum(OrderedIdentifier2Date(file_name.substr(matches[2].offset, matches[2].length))); diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp index 7f7a30083c8..630811ddff9 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp @@ -41,7 +41,7 @@ bool MergeTreeDataMerger::selectPartsToMerge(MergeTreeData::DataPartsVector & pa { MergeTreeData::DataParts data_parts = data.getDataParts(); - DateLUTSingleton & date_lut = DateLUTSingleton::instance(); + DateLUT & date_lut = DateLUT::instance(); size_t min_max = -1U; size_t min_min = -1U; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 8ee9cb69378..078a99f5a26 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -12,7 +12,7 @@ BlocksWithDateIntervals MergeTreeDataWriter::splitBlockIntoParts(const Block & b { data.check(block, true); - DateLUTSingleton & date_lut = DateLUTSingleton::instance(); + DateLUT & 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; - DateLUTSingleton & date_lut = DateLUTSingleton::instance(); + DateLUT & date_lut = DateLUT::instance(); size_t part_size = (block.rows() + data.index_granularity - 1) / data.index_granularity; diff --git a/libs/libcommon/src/DateLUT.cpp b/libs/libcommon/src/DateLUT.cpp index 2983d3224b0..8085e8e1e91 100644 --- a/libs/libcommon/src/DateLUT.cpp +++ b/libs/libcommon/src/DateLUT.cpp @@ -1,5 +1,6 @@ #include #include +#include DateLUT::DateLUT() @@ -7,11 +8,15 @@ DateLUT::DateLUT() /** Дополнительный вызов Time2Date для случая, когда в 1981-1984 году в России, * 1 апреля начиналось в час ночи, не в полночь. */ + size_t i = 0; for (time_t t = Time2Date(DATE_LUT_MIN); t <= DATE_LUT_MAX; t = Time2Date(TimeDayShift(t))) { - Values values; + if (i > DATE_LUT_MAX_DAY_NUM) + throw Poco::Exception("Cannot create DateLUT: i > DATE_LUT_MAX_DAY_NUM."); + + Values & values = lut[i]; struct tm tm; localtime_r(&t, &tm); @@ -28,12 +33,12 @@ DateLUT::DateLUT() values.date = mktime(&tm); - lut.push_back(values); + ++i; } /// Заполняем lookup таблицу для годов memset(years_lut, 0, DATE_LUT_YEARS * sizeof(years_lut[0])); - for (size_t day = 0; day < lut.size() && lut[day].year <= DATE_LUT_MAX_YEAR; ++day) + for (size_t day = 0; day < i && lut[day].year <= DATE_LUT_MAX_YEAR; ++day) { if (lut[day].month == 1 && lut[day].day_of_month == 1) years_lut[lut[day].year - DATE_LUT_MIN_YEAR] = day; diff --git a/libs/libcommon/src/tests/date_lut2.cpp b/libs/libcommon/src/tests/date_lut2.cpp index 161c5143b97..4f8b167767b 100644 --- a/libs/libcommon/src/tests/date_lut2.cpp +++ b/libs/libcommon/src/tests/date_lut2.cpp @@ -6,7 +6,7 @@ void loop(time_t begin, time_t end, int step) { - DateLUTSingleton & date_lut = DateLUTSingleton::instance(); + DateLUT & date_lut = DateLUT::instance(); for (time_t t = begin; t < end; t += step) std::cout << Time2Sql(t) diff --git a/libs/libcommon/src/tests/date_lut3.cpp b/libs/libcommon/src/tests/date_lut3.cpp index 832b0d9f624..b6460af93f0 100644 --- a/libs/libcommon/src/tests/date_lut3.cpp +++ b/libs/libcommon/src/tests/date_lut3.cpp @@ -8,7 +8,7 @@ void loop(time_t begin, time_t end, int step) { - DateLUTSingleton & date_lut = DateLUTSingleton::instance(); + DateLUT & date_lut = DateLUT::instance(); for (time_t t = begin; t < end; t += step) { diff --git a/libs/libmysqlxx/include/mysqlxx/Date.h b/libs/libmysqlxx/include/mysqlxx/Date.h index c2c465474a7..d90dd4a7b1d 100644 --- a/libs/libmysqlxx/include/mysqlxx/Date.h +++ b/libs/libmysqlxx/include/mysqlxx/Date.h @@ -29,7 +29,7 @@ private: void init(time_t time) { - DateLUTSingleton & date_lut = DateLUTSingleton::instance(); + DateLUT & date_lut = DateLUT::instance(); const DateLUT::Values & values = date_lut.getValues(time); m_year = values.year; @@ -66,7 +66,7 @@ public: Date(DayNum_t day_num) { - const DateLUT::Values & values = DateLUTSingleton::instance().getValues(day_num); + const DateLUT::Values & values = DateLUT::instance().getValues(day_num); m_year = values.year; m_month = values.month; m_day = values.day_of_month; @@ -113,12 +113,12 @@ public: operator time_t() const { - return DateLUTSingleton::instance().makeDate(m_year, m_month, m_day); + return DateLUT::instance().makeDate(m_year, m_month, m_day); } DayNum_t getDayNum() const { - return DateLUTSingleton::instance().makeDayNum(m_year, m_month, m_day); + return DateLUT::instance().makeDayNum(m_year, m_month, m_day); } operator DayNum_t() const diff --git a/libs/libmysqlxx/include/mysqlxx/DateTime.h b/libs/libmysqlxx/include/mysqlxx/DateTime.h index 5ab3ebfb3d4..8429d2256a8 100644 --- a/libs/libmysqlxx/include/mysqlxx/DateTime.h +++ b/libs/libmysqlxx/include/mysqlxx/DateTime.h @@ -42,7 +42,7 @@ private: return; } - DateLUTSingleton & date_lut = DateLUTSingleton::instance(); + DateLUT & date_lut = DateLUT::instance(); const DateLUT::Values & values = date_lut.getValues(time); m_year = values.year; @@ -123,7 +123,7 @@ public: { return m_year == 0 ? 0 - : DateLUTSingleton::instance().makeDateTime(m_year, m_month, m_day, m_hour, m_minute, m_second); + : DateLUT::instance().makeDateTime(m_year, m_month, m_day, m_hour, m_minute, m_second); } unsigned short year() const { return m_year; } diff --git a/libs/libmysqlxx/include/mysqlxx/Value.h b/libs/libmysqlxx/include/mysqlxx/Value.h index a56c523824a..2d128db7e72 100644 --- a/libs/libmysqlxx/include/mysqlxx/Value.h +++ b/libs/libmysqlxx/include/mysqlxx/Value.h @@ -145,7 +145,7 @@ private: time_t getDateTimeImpl() const { - DateLUTSingleton & date_lut = DateLUTSingleton::instance(); + DateLUT & date_lut = DateLUT::instance(); if (m_length == 10) { @@ -173,7 +173,7 @@ private: time_t getDateImpl() const { - DateLUTSingleton & date_lut = DateLUTSingleton::instance(); + DateLUT & date_lut = DateLUT::instance(); if (m_length == 10 || m_length == 19) { @@ -216,7 +216,7 @@ private: return getDateImpl(); else { - DateLUTSingleton & date_lut = DateLUTSingleton::instance(); + DateLUT & date_lut = DateLUT::instance(); return date_lut.toDate(getIntImpl()); } }